From 5df66e733338d34c55f738a23bb3e27fad4d6d42 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 15:22:30 +0300 Subject: [PATCH 001/167] introduce Soft & Hard RequiredInputOrderings remove usage of prefer_existing_sort as default set requirements Hard set soft on AggregateExec and BoundedWindowExec since they have InputOrderMode functionalities --- .../examples/custom_file_format.rs | 4 +- .../core/src/datasource/file_format/arrow.rs | 4 +- .../core/src/datasource/file_format/csv.rs | 6 +- .../core/src/datasource/file_format/json.rs | 4 +- .../core/src/datasource/file_format/mod.rs | 4 +- .../src/datasource/file_format/parquet.rs | 6 +- .../core/src/datasource/listing/table.rs | 9 +- datafusion/core/src/datasource/stream.rs | 5 +- .../enforce_distribution.rs | 133 +++++------------- .../physical_optimizer/projection_pushdown.rs | 9 +- .../tests/physical_optimizer/test_utils.rs | 7 +- .../src/enforce_distribution.rs | 20 +-- .../src/enforce_sorting/mod.rs | 9 +- .../replace_with_order_preserving_variants.rs | 6 +- .../src/enforce_sorting/sort_pushdown.rs | 126 ++++++++++------- .../src/output_requirements.rs | 22 +-- .../physical-optimizer/src/sanity_checker.rs | 4 +- datafusion/physical-optimizer/src/utils.rs | 7 +- .../physical-plan/src/aggregates/mod.rs | 14 +- .../physical-plan/src/execution_plan.rs | 65 ++++++++- datafusion/physical-plan/src/insert.rs | 12 +- .../src/joins/sort_merge_join.rs | 12 +- .../src/joins/symmetric_hash_join.rs | 12 +- .../src/sorts/sort_preserving_merge.rs | 7 +- .../src/windows/bounded_window_agg_exec.rs | 10 +- datafusion/physical-plan/src/windows/mod.rs | 52 ++++--- .../src/windows/window_agg_exec.rs | 10 +- datafusion/proto/src/physical_plan/mod.rs | 5 + .../tests/cases/roundtrip_physical_plan.rs | 21 +-- datafusion/sqllogictest/test_files/joins.slt | 29 ++-- 30 files changed, 350 insertions(+), 284 deletions(-) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index c44210e553182..55a1df4c266ef 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -23,8 +23,8 @@ use arrow::{ }; use datafusion::datasource::data_source::FileSource; use datafusion::execution::session_state::SessionStateBuilder; -use datafusion::physical_expr::LexRequirement; use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::{ catalog::Session, common::{GetExt, Statistics}, @@ -123,7 +123,7 @@ impl FileFormat for TSVFileFormat { input: Arc, state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { self.csv_file_format .create_writer_physical_plan(input, state, conf, order_requirements) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 5a4bf103e7ce4..339199310ca68 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -51,7 +51,7 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use crate::datasource::data_source::FileSource; @@ -182,7 +182,7 @@ impl FileFormat for ArrowFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Arrow format"); diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index cd8a4dedc1695..2e30d2a3b1968 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -27,6 +27,7 @@ use super::{ Decoder, DecoderDeserializer, FileFormat, FileFormatFactory, DEFAULT_SCHEMA_INFER_MAX_RECORD, }; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; use crate::datasource::file_format::write::BatchSerializer; @@ -54,9 +55,8 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use futures::stream::BoxStream; @@ -444,7 +444,7 @@ impl FileFormat for CsvFormat { input: Arc, state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for CSV"); diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 3aae692151f6b..6648e48159ea3 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -61,7 +61,7 @@ use datafusion_physical_plan::ExecutionPlan; use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; #[derive(Default)] @@ -265,7 +265,7 @@ impl FileFormat for JsonFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Json"); diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 383d2b14b31c4..0caf363f106d1 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -53,7 +53,7 @@ use datafusion_physical_expr::PhysicalExpr; use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use file_compression_type::FileCompressionType; use futures::stream::BoxStream; use futures::{ready, Stream, StreamExt}; @@ -140,7 +140,7 @@ pub trait FileFormat: Send + Sync + Debug { _input: Arc, _state: &dyn Session, _conf: FileSinkConfig, - _order_requirements: Option, + _order_requirements: Option, ) -> Result> { not_impl_err!("Writer not implemented for this format") } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 0b23fd8da6124..13a57278c9811 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -32,6 +32,7 @@ use super::{ }; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::get_writer_schema; use crate::datasource::physical_plan::parquet::can_expr_be_pushed_down_with_schemas; @@ -63,9 +64,8 @@ use datafusion_expr::dml::InsertOp; use datafusion_expr::Expr; use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::Bytes; use futures::future::BoxFuture; @@ -433,7 +433,7 @@ impl FileFormat for ParquetFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Parquet"); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 5d3a0f886fe49..4d898e6a24a46 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -41,6 +41,7 @@ use datafusion_physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; +use datafusion_catalog::Session; use datafusion_common::{ config_datafusion_err, internal_err, plan_err, project_schema, Constraints, SchemaExt, ToDFSchema, @@ -51,10 +52,10 @@ use datafusion_execution::cache::{ use datafusion_physical_expr::{ create_physical_expr, LexOrdering, PhysicalSortRequirement, }; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use async_trait::async_trait; -use datafusion_catalog::Session; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; @@ -1067,13 +1068,13 @@ impl TableProvider for ListingTable { ); }; // Converts Vec> into type required by execution plan to specify its required input ordering - Some(LexRequirement::new( + Some(RequiredInputOrdering::Hard(LexRequirement::new( ordering .into_iter() .cloned() .map(PhysicalSortRequirement::from) .collect::>(), - )) + ))) } else { None }; diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index ff2e4436e94d6..eab192362b275 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -35,6 +35,7 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; @@ -357,7 +358,9 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, std::slice::from_ref(x))?; let ordering = orders.into_iter().next().unwrap(); - Some(ordering.into_iter().map(Into::into).collect()) + Some(RequiredInputOrdering::Hard( + ordering.into_iter().map(Into::into).collect(), + )) } None => None, }; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 855550dc748a6..4ed5d1471c2b7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -40,7 +40,6 @@ use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::enforce_distribution::*; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_optimizer::output_requirements::OutputRequirements; @@ -49,7 +48,7 @@ use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::execution_plan::ExecutionPlan; +use datafusion_physical_plan::execution_plan::{ExecutionPlan, RequiredInputOrdering}; use datafusion_physical_plan::expressions::col; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::JoinOn; @@ -128,11 +127,11 @@ impl ExecutionPlan for SortRequiredExec { } // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { if self.expr.is_empty() { vec![None] } else { - vec![Some(LexRequirement::from(self.expr.clone()))] + vec![Some(RequiredInputOrdering::from(self.expr.clone()))] } } @@ -334,7 +333,6 @@ fn sort_required_exec_with_req( fn ensure_distribution_helper( plan: Arc, target_partitions: usize, - prefer_existing_sort: bool, ) -> Result> { let distribution_context = DistributionContext::new_default(plan); let mut config = ConfigOptions::new(); @@ -342,7 +340,6 @@ fn ensure_distribution_helper( config.optimizer.enable_round_robin_repartition = true; config.optimizer.repartition_file_scans = false; config.optimizer.repartition_file_min_size = 1024; - config.optimizer.prefer_existing_sort = prefer_existing_sort; ensure_distribution(distribution_context, &config).map(|item| item.data.plan) } @@ -377,29 +374,24 @@ macro_rules! plans_matches_expected { /// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave macro_rules! assert_optimized { ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, 10, false, 1024, false); }; - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_UNION: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, 10, false, 1024, $PREFER_EXISTING_UNION); }; - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); }; - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); let mut config = ConfigOptions::new(); config.execution.target_partitions = $TARGET_PARTITIONS; config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; // Use a small batch size, to trigger RoundRobin in tests config.execution.batch_size = 1; @@ -1048,7 +1040,6 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let bottom_left_join = ensure_distribution_helper( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, - true, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1079,7 +1070,6 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let bottom_right_join = ensure_distribution_helper( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, - true, )?; // Join on (B == b1 and C == c and AA = a1) @@ -1182,7 +1172,6 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let bottom_left_join = ensure_distribution_helper( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, - true, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1213,7 +1202,6 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let bottom_right_join = ensure_distribution_helper( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, - true, )?; // Join on (B == b1 and C == c and AA = a1) @@ -1385,7 +1373,7 @@ fn multi_smj_joins() -> Result<()> { "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected, top_join.clone(), true, true); + assert_optimized!(expected, top_join.clone(), true); let expected_first_sort_enforcement = match join_type { // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs @@ -1439,7 +1427,7 @@ fn multi_smj_joins() -> Result<()> { "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + assert_optimized!(expected_first_sort_enforcement, top_join, false); match join_type { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { @@ -1496,7 +1484,7 @@ fn multi_smj_joins() -> Result<()> { // this match arm cannot be reached _ => unreachable!() }; - assert_optimized!(expected, top_join.clone(), true, true); + assert_optimized!(expected, top_join.clone(), true); let expected_first_sort_enforcement = match join_type { // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs @@ -1542,7 +1530,7 @@ fn multi_smj_joins() -> Result<()> { // this match arm cannot be reached _ => unreachable!() }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + assert_optimized!(expected_first_sort_enforcement, top_join, false); } _ => {} } @@ -1616,7 +1604,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected, join.clone(), true, true); + assert_optimized!(expected, join.clone(), true); let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", @@ -1642,7 +1630,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - assert_optimized!(expected_first_sort_enforcement, join, false, true); + assert_optimized!(expected_first_sort_enforcement, join, false); Ok(()) } @@ -1674,17 +1662,6 @@ fn merge_does_not_need_sort() -> Result<()> { "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, true); - - // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: PREFER_EXISTING_SORT) - // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with - // SortExec at the top. - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "CoalesceBatchesExec: target_batch_size=4096", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; assert_optimized!(expected, exec, false); Ok(()) @@ -1769,8 +1746,6 @@ fn union_not_to_interleave() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; - // no sort in the plan but since we need it as a parameter, make it default false - let prefer_existing_sort = false; let first_enforce_distribution = true; let prefer_existing_union = true; @@ -1778,14 +1753,12 @@ fn union_not_to_interleave() -> Result<()> { expected, plan.clone(), first_enforce_distribution, - prefer_existing_sort, prefer_existing_union ); assert_optimized!( expected, plan, !first_enforce_distribution, - prefer_existing_sort, prefer_existing_union ); @@ -1992,12 +1965,6 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { ]; assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, plan, false); Ok(()) @@ -2023,14 +1990,6 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { ]; assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, plan, false); Ok(()) @@ -2058,8 +2017,8 @@ fn repartition_does_not_destroy_sort() -> Result<()> { "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; - assert_optimized!(expected, plan.clone(), true, true); - assert_optimized!(expected, plan, false, true); + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); Ok(()) } @@ -2312,8 +2271,8 @@ fn parallelization_single_partition() -> Result<()> { "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, true, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, 2, true, 10); Ok(()) } @@ -2342,7 +2301,6 @@ fn parallelization_multiple_files() -> Result<()> { expected, plan, true, - true, target_partitions, true, repartition_size, @@ -2360,7 +2318,6 @@ fn parallelization_multiple_files() -> Result<()> { expected, plan, true, - true, target_partitions, true, repartition_size, @@ -2415,7 +2372,7 @@ fn parallelization_compressed_csv() -> Result<()> { .new_exec(), vec![("a".to_string(), "a".to_string())], ); - assert_optimized!(expected, plan, true, false, 2, true, 10, false); + assert_optimized!(expected, plan, true, 2, true, 10, false); } Ok(()) } @@ -2440,8 +2397,8 @@ fn parallelization_two_partitions() -> Result<()> { // Plan already has two partitions "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + assert_optimized!(expected_parquet, plan_parquet, true, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, 2, true, 10); Ok(()) } @@ -2465,8 +2422,8 @@ fn parallelization_two_partitions_into_four() -> Result<()> { // Multiple source files splitted across partitions "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); + assert_optimized!(expected_parquet, plan_parquet, true, 4, true, 10); + assert_optimized!(expected_csv, plan_csv, true, 4, true, 10); Ok(()) } @@ -2838,9 +2795,8 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); Ok(()) } @@ -2861,9 +2817,8 @@ fn preserve_ordering_through_repartition() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); Ok(()) } @@ -2880,21 +2835,12 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC", "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; assert_optimized!(expected, physical_plan, false); Ok(()) @@ -2915,7 +2861,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { "CoalescePartitionsExec", // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2948,15 +2894,6 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { ]; assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, physical_plan, false); Ok(()) @@ -3013,7 +2950,6 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, dist_plan); @@ -3050,7 +2986,6 @@ fn put_sort_when_input_is_valid() -> Result<()> { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, dist_plan); @@ -3074,8 +3009,8 @@ fn do_not_add_unnecessary_hash() -> Result<()> { "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary - assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); + assert_optimized!(expected, physical_plan.clone(), true, 1, false, 1024); + assert_optimized!(expected, physical_plan, false, 1, false, 1024); Ok(()) } @@ -3104,8 +3039,8 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). - assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); + assert_optimized!(expected, physical_plan.clone(), true, 4, false, 1024); + assert_optimized!(expected, physical_plan, false, 4, false, 1024); Ok(()) } diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 12b41196ef374..cc005c0aa8892 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -41,6 +41,7 @@ use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion_physical_plan::joins::{ @@ -650,7 +651,7 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv.clone(), - Some(LexRequirement::new(vec![ + Some(RequiredInputOrdering::Hard(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), options: Some(SortOptions::default()), @@ -663,7 +664,7 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ])), + ]))), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -696,7 +697,7 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = LexRequirement::new(vec![ + let expected_reqs = RequiredInputOrdering::Hard(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 2)), options: Some(SortOptions::default()), @@ -709,7 +710,7 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ]); + ])); assert_eq!( after_optimize .as_any() diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 721dfca029b08..fe25f8e19f615 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -38,7 +38,6 @@ use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{expressions, PhysicalExpr}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; use datafusion_physical_optimizer::PhysicalOptimizerRule; @@ -47,6 +46,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; @@ -400,8 +400,9 @@ impl ExecutionPlan for RequirementsTestExec { self.input.properties() } - fn required_input_ordering(&self) -> Vec> { - let requirement = LexRequirement::from(self.required_input_ordering.clone()); + fn required_input_ordering(&self) -> Vec> { + let requirement = + RequiredInputOrdering::from(self.required_input_ordering.clone()); vec![Some(requirement)] } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 5e76edad1f569..c3cd86fb1498b 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -47,7 +47,7 @@ use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::execution_plan::{EmissionType, RequiredInputOrdering}; use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, }; @@ -1178,12 +1178,6 @@ pub fn ensure_distribution( dist_context.plan.pipeline_behavior(), EmissionType::Incremental | EmissionType::Both ); - // Use order preserving variants either of the conditions true - // - it is desired according to config - // - when plan is unbounded - // - when it is pipeline friendly (can incrementally produce results) - let order_preserving_variants_desirable = - unbounded_and_pipeline_friendly || config.optimizer.prefer_existing_sort; // Remove unnecessary repartition from the physical plan if any let DistributionContext { @@ -1286,7 +1280,17 @@ pub fn ensure_distribution( let ordering_satisfied = child .plan .equivalence_properties() - .ordering_satisfy_requirement(&required_input_ordering); + .ordering_satisfy_requirement( + required_input_ordering.lex_requirement(), + ); + + // Use order preserving variants either of the conditions true + // - it is desired according to requirement + // - when plan is unbounded + // - when it is pipeline friendly (can incrementally produce results) + // - when the requirement is not optional + let order_preserving_variants_desirable = unbounded_and_pipeline_friendly + || matches!(required_input_ordering, RequiredInputOrdering::Hard(_)); if (!ordering_satisfied || !order_preserving_variants_desirable) && child.data { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index a25e6c6f17ac3..dbeac7a4d6e07 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -59,6 +59,7 @@ use datafusion_common::Result; use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; @@ -274,7 +275,7 @@ pub fn parallelize_sorts( { // Take the initial sort expressions and requirements let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; - let sort_reqs = LexRequirement::from(sort_exprs.clone()); + let sort_reqs = RequiredInputOrdering::from(sort_exprs.clone()); let sort_exprs = sort_exprs.clone(); // If there is a connection between a `CoalescePartitionsExec` and a @@ -347,12 +348,12 @@ pub fn ensure_sorting( if let Some(required) = required_ordering { let eq_properties = child.plan.equivalence_properties(); - if !eq_properties.ordering_satisfy_requirement(&required) { + if !eq_properties.ordering_satisfy_requirement(required.lex_requirement()) { // Make sure we preserve the ordering requirements: if physical_ordering.is_some() { child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; } - child = add_sort_above(child, required, None); + child = add_sort_above(child, required.lex_requirement().clone(), None); child = update_sort_ctx_children(child, true)?; } } else if physical_ordering.is_none() @@ -485,7 +486,7 @@ fn adjust_window_sort_removal( // Satisfy the ordering requirement so that the window can run: let mut child_node = window_tree.children.swap_remove(0); - child_node = add_sort_above(child_node, reqs, None); + child_node = add_sort_above(child_node, reqs.lex_requirement().clone(), None); let child_plan = Arc::clone(&child_node.plan); window_tree.children.push(child_node); diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index c542f9261a247..92012dfa12889 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -171,7 +171,7 @@ fn plan_with_order_breaking_variants( || !required_ordering.is_some_and(|required_ordering| { node.plan .equivalence_properties() - .ordering_satisfy_requirement(&required_ordering) + .ordering_satisfy_requirement(required_ordering.lex_requirement()) })) { plan_with_order_breaking_variants(node) @@ -210,10 +210,6 @@ fn plan_with_order_breaking_variants( /// If this replacement is helpful for removing a `SortExec`, it updates the plan. /// Otherwise, it leaves the plan unchanged. /// -/// NOTE: This optimizer sub-rule will only produce sort-preserving `RepartitionExec`s -/// if the query is bounded or if the config option `prefer_existing_sort` is -/// set to `true`. -/// /// The algorithm flow is simply like this: /// 1. Visit nodes of the physical plan bottom-up and look for `SortExec` nodes. /// During the traversal, keep track of operators that maintain ordering (or diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index a0414af13daf0..6af1e8f326c07 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -32,6 +32,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalSortRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ calculate_join_output_ordering, ColumnIndex, @@ -52,7 +53,7 @@ use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; /// [`EnforceSorting`]: crate::enforce_sorting::EnforceSorting #[derive(Default, Clone)] pub struct ParentRequirements { - ordering_requirement: Option, + ordering_requirement: Option, fetch: Option, } @@ -95,7 +96,7 @@ fn pushdown_sorts_helper( .unwrap_or_default(); let satisfy_parent = plan .equivalence_properties() - .ordering_satisfy_requirement(&parent_reqs); + .ordering_satisfy_requirement(parent_reqs.lex_requirement()); if is_sort(plan) { let sort_fetch = plan.fetch(); @@ -103,6 +104,7 @@ fn pushdown_sorts_helper( .output_ordering() .cloned() .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard) .unwrap_or_default(); if !satisfy_parent { // Make sure this `SortExec` satisfies parent requirements: @@ -116,7 +118,8 @@ fn pushdown_sorts_helper( } let fetch = requirements.data.fetch.or(sort_fetch); requirements = requirements.children.swap_remove(0); - requirements = add_sort_above(requirements, sort_reqs, fetch); + requirements = + add_sort_above(requirements, sort_reqs.lex_requirement().clone(), fetch); }; // We can safely get the 0th index as we are dealing with a `SortExec`. @@ -168,7 +171,8 @@ fn pushdown_sorts_helper( .clone() .unwrap_or_default(); let fetch = requirements.data.fetch; - requirements = add_sort_above(requirements, sort_reqs, fetch); + requirements = + add_sort_above(requirements, sort_reqs.lex_requirement().clone(), fetch); assign_initial_requirements(&mut requirements); } Ok(Transformed::yes(requirements)) @@ -176,8 +180,8 @@ fn pushdown_sorts_helper( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: &LexRequirement, -) -> Result>>> { + parent_required: &RequiredInputOrdering, +) -> Result>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); @@ -187,15 +191,18 @@ fn pushdown_requirement_to_children( match determine_children_requirement(parent_required, &request_child, child_plan) { RequirementsCompatibility::Satisfy => { - let req = (!request_child.is_empty()) - .then(|| LexRequirement::new(request_child.to_vec())); + let req = (!request_child.is_empty()).then(|| { + RequiredInputOrdering::Hard(LexRequirement::new( + request_child.to_vec(), + )) + }); Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), } } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let sort_req = LexRequirement::from( + let sort_req = RequiredInputOrdering::from( sort_exec .properties() .output_ordering() @@ -205,12 +212,13 @@ fn pushdown_requirement_to_children( if sort_exec .properties() .eq_properties - .requirements_compatible(parent_required, &sort_req) + .requirements_compatible( + parent_required.lex_requirement(), + sort_req.lex_requirement(), + ) { debug_assert!(!parent_required.is_empty()); - Ok(Some(vec![Some(LexRequirement::new( - parent_required.to_vec(), - ))])) + Ok(Some(vec![Some(parent_required.clone())])) } else { Ok(None) } @@ -233,10 +241,11 @@ fn pushdown_requirement_to_children( if plan .properties() .eq_properties - .requirements_compatible(parent_required, &output_req) + .requirements_compatible(parent_required.lex_requirement(), &output_req) { - let req = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); + let req = (!parent_required.is_empty()).then(|| { + RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + }); Ok(Some(vec![req])) } else { Ok(None) @@ -249,7 +258,8 @@ fn pushdown_requirement_to_children( } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = LexOrdering::from(parent_required.clone()); + let parent_required_expr = + LexOrdering::from(parent_required.lex_requirement().clone()); match expr_source_side( parent_required_expr.as_ref(), smj.join_type(), @@ -257,19 +267,21 @@ fn pushdown_requirement_to_children( ) { Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, - parent_required, + parent_required.lex_requirement(), parent_required_expr.as_ref(), JoinSide::Left, ), Some(JoinSide::Right) => { let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); - let new_right_required = - shift_right_required(parent_required, right_offset)?; + let new_right_required = shift_right_required( + parent_required.lex_requirement(), + right_offset, + )?; let new_right_required_expr = LexOrdering::from(new_right_required); try_pushdown_requirements_to_join( smj, - parent_required, + parent_required.lex_requirement(), new_right_required_expr.as_ref(), JoinSide::Right, ) @@ -285,14 +297,14 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // TODO: Add support for Projection push down || plan.as_any().is::() - || pushdown_would_violate_requirements(parent_required, plan.as_ref()) + || pushdown_would_violate_requirements(parent_required.lex_requirement(), plan.as_ref()) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) } else if is_sort_preserving_merge(plan) { - let new_ordering = LexOrdering::from(parent_required.clone()); + let new_ordering = LexOrdering::from(parent_required.lex_requirement().clone()); let mut spm_eqs = plan.equivalence_properties().clone(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); @@ -304,8 +316,9 @@ fn pushdown_requirement_to_children( } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - let req = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); + let req = (!parent_required.is_empty()).then(|| { + RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + }); Ok(Some(vec![req])) } } else if let Some(hash_join) = plan.as_any().downcast_ref::() { @@ -333,6 +346,7 @@ fn pushdown_would_violate_requirements( // check if the plan's requirements would still e satisfied if we pushed // down the parent requirements child_required + .lex_requirement() .iter() .zip(parent_required.iter()) .all(|(c, p)| !c.compatible(p)) @@ -345,24 +359,25 @@ fn pushdown_would_violate_requirements( /// - If parent requirements are more specific, push down parent requirements. /// - If they are not compatible, need to add a sort. fn determine_children_requirement( - parent_required: &LexRequirement, - request_child: &LexRequirement, + parent_required: &RequiredInputOrdering, + request_child: &RequiredInputOrdering, child_plan: &Arc, ) -> RequirementsCompatibility { - if child_plan - .equivalence_properties() - .requirements_compatible(request_child, parent_required) - { + if child_plan.equivalence_properties().requirements_compatible( + request_child.lex_requirement(), + parent_required.lex_requirement(), + ) { // Child requirements are more specific, no need to push down. RequirementsCompatibility::Satisfy - } else if child_plan - .equivalence_properties() - .requirements_compatible(parent_required, request_child) - { + } else if child_plan.equivalence_properties().requirements_compatible( + parent_required.lex_requirement(), + request_child.lex_requirement(), + ) { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: - let adjusted = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); + let adjusted = (!parent_required.lex_requirement().is_empty()).then(|| { + RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + }); RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -374,7 +389,7 @@ fn try_pushdown_requirements_to_join( parent_required: &LexRequirement, sort_expr: &LexOrdering, push_side: JoinSide, -) -> Result>>> { +) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); let right_eq_properties = smj.right().equivalence_properties(); let mut smj_required_orderings = smj.required_input_ordering(); @@ -387,9 +402,9 @@ fn try_pushdown_requirements_to_join( JoinSide::Left => { let left_eq_properties = left_eq_properties.clone().with_reorder(sort_expr.clone()); - if left_eq_properties - .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) - { + if left_eq_properties.ordering_satisfy_requirement( + left_requirement.unwrap_or_default().lex_requirement(), + ) { // After re-ordering requirement is still satisfied (sort_expr, right_ordering) } else { @@ -399,9 +414,9 @@ fn try_pushdown_requirements_to_join( JoinSide::Right => { let right_eq_properties = right_eq_properties.clone().with_reorder(sort_expr.clone()); - if right_eq_properties - .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) - { + if right_eq_properties.ordering_satisfy_requirement( + right_requirement.unwrap_or_default().lex_requirement(), + ) { // After re-ordering requirement is still satisfied (left_ordering, sort_expr) } else { @@ -427,7 +442,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(LexRequirement::from(sort_expr.clone())); + let new_req = Some(RequiredInputOrdering::from(sort_expr.clone())); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; @@ -532,9 +547,9 @@ fn shift_right_required( /// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. fn handle_custom_pushdown( plan: &Arc, - parent_required: &LexRequirement, + parent_required: &RequiredInputOrdering, maintains_input_order: Vec, -) -> Result>>> { +) -> Result>>> { // If there's no requirement from the parent or the plan has no children, return early if parent_required.is_empty() || plan.children().is_empty() { return Ok(None); @@ -542,6 +557,7 @@ fn handle_custom_pushdown( // Collect all unique column indices used in the parent-required sorting expression let all_indices: HashSet = parent_required + .lex_requirement() .iter() .flat_map(|order| { collect_columns(&order.expr) @@ -584,6 +600,7 @@ fn handle_custom_pushdown( .sum::(); // Transform the parent-required expression for the child schema by adjusting columns let updated_parent_req = parent_required + .lex_requirement() .iter() .map(|req| { let child_schema = plan.children()[maintained_child_idx].schema(); @@ -609,7 +626,10 @@ fn handle_custom_pushdown( .iter() .map(|&maintains_order| { if maintains_order { - Some(LexRequirement::new(updated_parent_req.clone())) + Some( + parent_required + .with_updated_requirements(updated_parent_req.clone()), + ) } else { None } @@ -626,8 +646,8 @@ fn handle_custom_pushdown( // for join type: Inner, Right, RightSemi, RightAnti fn handle_hash_join( plan: &HashJoinExec, - parent_required: &LexRequirement, -) -> Result>>> { + parent_required: &RequiredInputOrdering, +) -> Result>>> { // If there's no requirement from the parent or the plan has no children // or the join type is not Inner, Right, RightSemi, RightAnti, return early if parent_required.is_empty() || !plan.maintains_input_order()[1] { @@ -636,6 +656,7 @@ fn handle_hash_join( // Collect all unique column indices used in the parent-required sorting expression let all_indices: HashSet = parent_required + .lex_requirement() .iter() .flat_map(|order| { collect_columns(&order.expr) @@ -662,6 +683,7 @@ fn handle_hash_join( if all_from_right_child { // Transform the parent-required expression for the child schema by adjusting columns let updated_parent_req = parent_required + .lex_requirement() .iter() .map(|req| { let child_schema = plan.children()[1].schema(); @@ -685,7 +707,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - Some(LexRequirement::new(updated_parent_req)), + Some(parent_required.with_updated_requirements(updated_parent_req)), ])) } else { Ok(None) @@ -724,7 +746,7 @@ enum RequirementsCompatibility { /// Requirements satisfy Satisfy, /// Requirements compatible - Compatible(Option), + Compatible(Option), /// Requirements not compatible NonCompatible, } diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 90a570894a44d..1043bb62222bb 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -31,6 +31,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::projection::{ make_with_child, update_expr, ProjectionExec, }; @@ -94,7 +95,7 @@ enum RuleMode { #[derive(Debug)] pub struct OutputRequirementExec { input: Arc, - order_requirement: Option, + order_requirement: Option, dist_requirement: Distribution, cache: PlanProperties, } @@ -102,7 +103,7 @@ pub struct OutputRequirementExec { impl OutputRequirementExec { pub fn new( input: Arc, - requirements: Option, + requirements: Option, dist_requirement: Distribution, ) -> Self { let cache = Self::compute_properties(&input); @@ -168,7 +169,7 @@ impl ExecutionPlan for OutputRequirementExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.order_requirement.clone()] } @@ -207,7 +208,7 @@ impl ExecutionPlan for OutputRequirementExec { let mut updated_sort_reqs = LexRequirement::new(vec![]); // None or empty_vec can be treated in the same way. if let Some(reqs) = &self.required_input_ordering()[0] { - for req in &reqs.inner { + for req in reqs.lex_requirement().inner.clone() { let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { return Ok(None); @@ -238,7 +239,8 @@ impl ExecutionPlan for OutputRequirementExec { .map(|input| { OutputRequirementExec::new( input, - (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), + (!updated_sort_reqs.is_empty()) + .then_some(RequiredInputOrdering::Hard(updated_sort_reqs)), dist_req, ) }) @@ -309,13 +311,13 @@ fn require_top_ordering_helper( // Therefore; we check the sort expression field of the SortExec to assign the requirements. let req_ordering = sort_exec.expr(); let req_dist = sort_exec.required_input_distribution()[0].clone(); - let reqs = LexRequirement::from(req_ordering.clone()); + let reqs = RequiredInputOrdering::from(req_ordering.clone()); Ok(( Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, true, )) } else if let Some(spm) = plan.as_any().downcast_ref::() { - let reqs = LexRequirement::from(spm.expr().clone()); + let reqs = RequiredInputOrdering::from(spm.expr().clone()); Ok(( Arc::new(OutputRequirementExec::new( plan, @@ -325,7 +327,11 @@ fn require_top_ordering_helper( true, )) } else if plan.maintains_input_order()[0] - && plan.required_input_ordering()[0].is_none() + && (plan.required_input_ordering()[0].is_none() + || matches!( + plan.required_input_ordering()[0].clone().unwrap(), + RequiredInputOrdering::Soft(_) + )) { // Keep searching for a `SortExec` as long as ordering is maintained, // and on-the-way operators do not themselves require an ordering. diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index 8edbb0f091140..85b0f7f2f4cd6 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -137,12 +137,12 @@ pub fn check_plan_sanity( ) { let child_eq_props = child.equivalence_properties(); if let Some(sort_req) = sort_req { - if !child_eq_props.ordering_satisfy_requirement(&sort_req) { + if !child_eq_props.ordering_satisfy_requirement(sort_req.lex_requirement()) { let plan_str = get_plan_string(&plan); return plan_err!( "Plan: {:?} does not satisfy order requirements: {}. Child-{} order: {}", plan_str, - format_physical_sort_requirement_list(&sort_req), + format_physical_sort_requirement_list(sort_req.lex_requirement()), idx, child_eq_props.oeq_class() ); diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 636e78a06ce7b..8e70c18fdd99d 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; @@ -55,15 +56,15 @@ pub fn add_sort_above( /// requirement is already satisfied no `SortExec` is added. pub fn add_sort_above_with_check( node: PlanContext, - sort_requirements: LexRequirement, + sort_requirements: RequiredInputOrdering, fetch: Option, ) -> PlanContext { if !node .plan .equivalence_properties() - .ordering_satisfy_requirement(&sort_requirements) + .ordering_satisfy_requirement(sort_requirements.lex_requirement()) { - add_sort_above(node, sort_requirements, fetch) + add_sort_above(node, sort_requirements.lex_requirement().clone(), fetch) } else { node } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 85b41da85742b..bc1d5af3f429e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -25,7 +25,7 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; -use crate::execution_plan::{CardinalityEffect, EmissionType}; +use crate::execution_plan::{CardinalityEffect, EmissionType, RequiredInputOrdering}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::get_field_metadata; use crate::windows::get_ordered_partition_by_indices; @@ -370,7 +370,7 @@ pub struct AggregateExec { pub input_schema: SchemaRef, /// Execution metrics metrics: ExecutionPlanMetricsSet, - required_input_ordering: Option, + required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, cache: PlanProperties, @@ -502,8 +502,8 @@ impl AggregateExec { let group_expr_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); + let required_input_ordering = (!new_requirement.is_empty()) + .then_some(RequiredInputOrdering::Soft(new_requirement)); let cache = Self::compute_properties( &input, @@ -635,8 +635,8 @@ impl AggregateExec { return false; } // ensure no ordering is required on the input - if self.required_input_ordering()[0].is_some() { - return false; + if let Some(requirement) = self.required_input_ordering()[0].clone() { + return matches!(requirement, RequiredInputOrdering::Hard(_)); } true } @@ -822,7 +822,7 @@ impl ExecutionPlan for AggregateExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.required_input_ordering.clone()] } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 0104c91eb5d5d..2c9dde65aebaa 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -48,7 +48,9 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{exec_err, Constraints, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{ + LexRequirement, PhysicalSortRequirement, +}; use futures::stream::{StreamExt, TryStreamExt}; use tokio::task::JoinSet; @@ -136,7 +138,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// NOTE that checking `!is_empty()` does **not** check for a /// required input ordering. Instead, the correct check is that at /// least one entry must be `Some` - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![None; self.children().len()] } @@ -1051,6 +1053,65 @@ pub enum CardinalityEffect { GreaterEqual, } +/// Represents the plan's input ordering requirement +#[derive(Debug, Clone, PartialEq)] +pub enum RequiredInputOrdering { + /// The operator can not work without this ordering + Hard(LexRequirement), + /// The operator can benefit from the ordering if provided + /// but if not provided it can also work + Soft(LexRequirement), +} + +impl Default for RequiredInputOrdering { + fn default() -> Self { + RequiredInputOrdering::Hard(LexRequirement::default()) + } +} + +impl From for RequiredInputOrdering { + fn from(ordering: LexOrdering) -> Self { + RequiredInputOrdering::Hard(LexRequirement::from(ordering)) + } +} + +impl RequiredInputOrdering { + pub fn lex_requirement(&self) -> &LexRequirement { + match self { + RequiredInputOrdering::Hard(lex) => lex, + RequiredInputOrdering::Soft(lex) => lex, + } + } + + pub fn is_empty(&self) -> bool { + self.lex_requirement().is_empty() + } + + pub fn to_vec(&self) -> Vec { + self.lex_requirement().to_vec() + } + + pub fn with_updated_requirements( + &self, + requirement: Vec, + ) -> Self { + match self { + RequiredInputOrdering::Hard(_) => { + RequiredInputOrdering::Hard(LexRequirement::new(requirement)) + } + RequiredInputOrdering::Soft(_) => { + RequiredInputOrdering::Soft(LexRequirement::new(requirement)) + } + } + } + + pub fn push(&self, requirement: PhysicalSortRequirement) -> Self { + let mut requirements = self.lex_requirement().clone(); + requirements.push(requirement); + self.with_updated_requirements(requirements.to_vec()) + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 0b8c1eede36c6..151a4ef7a02ae 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -26,6 +26,7 @@ use super::{ execute_input_stream, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; +use crate::execution_plan::RequiredInputOrdering; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; use crate::ExecutionPlanProperties; @@ -38,7 +39,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{Distribution, EquivalenceProperties}; use async_trait::async_trait; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::StreamExt; /// `DataSink` implements writing streams of [`RecordBatch`]es to @@ -90,7 +90,7 @@ pub struct DataSinkExec { /// Schema describing the structure of the output data. count_schema: SchemaRef, /// Optional required sort order for output data. - sort_order: Option, + sort_order: Option, cache: PlanProperties, } @@ -105,7 +105,7 @@ impl DataSinkExec { pub fn new( input: Arc, sink: Arc, - sort_order: Option, + sort_order: Option, ) -> Self { let count_schema = make_count_schema(); let cache = Self::create_schema(&input, count_schema); @@ -129,7 +129,7 @@ impl DataSinkExec { } /// Optional sort order for output data - pub fn sort_order(&self) -> &Option { + pub fn sort_order(&self) -> &Option { &self.sort_order } @@ -184,10 +184,10 @@ impl ExecutionPlan for DataSinkExec { vec![Distribution::SinglePartition; self.children().len()] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { // The required input ordering is set externally (e.g. by a `ListingTable`). // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). - vec![self.sort_order.as_ref().cloned()] + vec![self.sort_order.clone()] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index a3e835c64131c..2f4b7948199cf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,7 +34,9 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::execution_plan::{boundedness_from_children, EmissionType}; +use crate::execution_plan::{ + boundedness_from_children, EmissionType, RequiredInputOrdering, +}; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, @@ -70,7 +72,7 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::{Stream, StreamExt}; @@ -398,10 +400,10 @@ impl ExecutionPlan for SortMergeJoinExec { ] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![ - Some(LexRequirement::from(self.left_sort_exprs.clone())), - Some(LexRequirement::from(self.right_sort_exprs.clone())), + Some(RequiredInputOrdering::from(self.left_sort_exprs.clone())), + Some(RequiredInputOrdering::from(self.right_sort_exprs.clone())), ] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 47af4ab9a7652..4e46d0d889a17 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -33,7 +33,9 @@ use std::task::{Context, Poll}; use std::vec; use crate::common::SharedMemoryReservation; -use crate::execution_plan::{boundedness_from_children, emission_type_from_children}; +use crate::execution_plan::{ + boundedness_from_children, emission_type_from_children, RequiredInputOrdering, +}; use crate::joins::hash_join::{equal_rows_arr, update_hash}; use crate::joins::stream_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, @@ -416,16 +418,18 @@ impl ExecutionPlan for SymmetricHashJoinExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![ self.left_sort_exprs .as_ref() .cloned() - .map(LexRequirement::from), + .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard), self.right_sort_exprs .as_ref() .cloned() - .map(LexRequirement::from), + .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard), ] } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index c2cd9d7db3c65..03a40bace5fd2 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -21,6 +21,7 @@ use std::any::Any; use std::sync::Arc; use crate::common::spawn_buffered; +use crate::execution_plan::RequiredInputOrdering; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{make_with_child, update_expr, ProjectionExec}; @@ -34,7 +35,7 @@ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use log::{debug, trace}; @@ -227,8 +228,8 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![false] } - fn required_input_ordering(&self) -> Vec> { - vec![Some(LexRequirement::from(self.expr.clone()))] + fn required_input_ordering(&self) -> Vec> { + vec![Some(RequiredInputOrdering::from(self.expr.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index a734feae5533c..29efa7bf194ba 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -28,6 +28,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; +use crate::execution_plan::RequiredInputOrdering; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -38,7 +39,7 @@ use crate::{ ExecutionPlanProperties, InputOrderMode, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; -use ahash::RandomState; + use arrow::compute::take_record_batch; use arrow::{ array::{Array, ArrayRef, RecordBatchOptions, UInt32Builder}, @@ -61,8 +62,9 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use ahash::RandomState; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::hash_table::HashTable; @@ -275,14 +277,14 @@ impl ExecutionPlan for BoundedWindowAggExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements(partition_bys, order_keys.iter(), true)] } fn required_input_distribution(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index cdab1fa5929d5..bec7b3dd8005b 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -20,6 +20,7 @@ use std::borrow::Borrow; use std::sync::Arc; +use crate::execution_plan::RequiredInputOrdering; use crate::{ expressions::PhysicalSortExpr, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, @@ -275,7 +276,8 @@ pub(crate) fn calc_requirements< >( partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, -) -> Option { + is_soft_requirements: bool, +) -> Option { let mut sort_reqs = LexRequirement::new( partition_by_exprs .into_iter() @@ -293,8 +295,12 @@ pub(crate) fn calc_requirements< )); } } - // Convert empty result to None. Otherwise wrap result inside Some() - (!sort_reqs.is_empty()).then_some(sort_reqs) + + if is_soft_requirements { + (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Soft(sort_reqs)) + } else { + (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Hard(sort_reqs)) + } } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -624,21 +630,35 @@ mod tests { orderbys.push(PhysicalSortExpr { expr, options }); } - let mut expected: Option = None; - for (col_name, reqs) in expected_params { - let options = reqs.map(|(descending, nulls_first)| SortOptions { - descending, - nulls_first, - }); - let expr = col(col_name, &schema)?; - let res = PhysicalSortRequirement::new(expr, options); - if let Some(expected) = &mut expected { - expected.push(res); - } else { - expected = Some(LexRequirement::new(vec![res])); + for is_soft in [true, false] { + let mut expected: Option = None; + for (col_name, reqs) in expected_params.clone() { + let options = reqs.map(|(descending, nulls_first)| SortOptions { + descending, + nulls_first, + }); + let expr = col(col_name, &schema)?; + let res = PhysicalSortRequirement::new(expr, options); + if let Some(expected) = &mut expected { + expected.push(res); + } else { + expected = Some(LexRequirement::new(vec![res])); + } } + let expected_result = if let Some(expected) = expected { + if is_soft { + Some(RequiredInputOrdering::Soft(expected)) + } else { + Some(RequiredInputOrdering::Hard(expected)) + } + } else { + None + }; + assert_eq!( + calc_requirements(partitionbys.clone(), orderbys.clone(), is_soft), + expected_result + ); } - assert_eq!(calc_requirements(partitionbys, orderbys), expected); } Ok(()) } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index d31fd66ca1f14..29c99fd341ce0 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::execution_plan::EmissionType; +use crate::execution_plan::{EmissionType, RequiredInputOrdering}; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -44,7 +44,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::{ready, Stream, StreamExt}; @@ -208,17 +208,17 @@ impl ExecutionPlan for WindowAggExec { vec![true] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements(partition_bys, order_keys.iter(), false)] } else { let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements(partition_bys, order_keys.iter(), false)] } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 84b9529659582..06e581c0e6adb 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -42,6 +42,7 @@ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::empty::EmptyExec; +use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::filter::FilterExec; @@ -1071,6 +1072,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { extension_codec, ) .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1100,6 +1102,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { extension_codec, ) .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1136,6 +1139,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { extension_codec, ) .map(LexRequirement::from) + .map(RequiredInputOrdering::Hard) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1992,6 +1996,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let sort_order = match exec.sort_order() { Some(requirements) => { let expr = requirements + .lex_requirement() .iter() .map(|requirement| { let expr: PhysicalSortExpr = requirement.to_owned().into(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 7418184fcac15..38a2d0ada9e72 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -65,6 +65,7 @@ use datafusion::physical_plan::aggregates::{ }; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; +use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; @@ -1309,7 +1310,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(sort_order), + Some(RequiredInputOrdering::Hard(sort_order)), ))) } @@ -1334,13 +1335,15 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( - Arc::new(Column::new("plan_type", 0)), - Some(SortOptions { - descending: true, - nulls_first: false, - }), - )]); + let sort_order = RequiredInputOrdering::Hard(LexRequirement::new(vec![ + PhysicalSortRequirement::new( + Arc::new(Column::new("plan_type", 0)), + Some(SortOptions { + descending: true, + nulls_first: false, + }), + ), + ])); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1400,7 +1403,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(sort_order), + Some(RequiredInputOrdering::Hard(sort_order)), ))) } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 5d311bc43293f..ec512bc261dc2 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3320,22 +3320,19 @@ logical_plan 10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST] -02)--SortExec: expr=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST], preserve_partitioning=[true] -03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -09)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -12)--------CoalesceBatchesExec: target_batch_size=2 -13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -16)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -17)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +09)----CoalesceBatchesExec: target_batch_size=2 +10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +13)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +14)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; From 7c9f6d34d667994bc89f5204b45798e0574a1edc Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 15:34:31 +0300 Subject: [PATCH 002/167] add documentation to replace_with_partial_sort simplify indentation --- .../src/enforce_sorting/mod.rs | 59 +++++++++++-------- 1 file changed, 33 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index dbeac7a4d6e07..bb7641806f73c 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -208,38 +208,45 @@ impl PhysicalOptimizerRule for EnforceSorting { } } +/// Only interested with `SortExec`s and their unbounded children. +/// If the plan is not a `SortExec` or its child is not unbounded, returns the original plan. +/// Otherwise, by checking the requirement satisfaction searches for a replacement chance. +/// If there's one replaces the `SortExec` plan with a PartialSortExec fn replace_with_partial_sort( plan: Arc, ) -> Result> { let plan_any = plan.as_any(); - if let Some(sort_plan) = plan_any.downcast_ref::() { - let child = Arc::clone(sort_plan.children()[0]); - if !child.boundedness().is_unbounded() { - return Ok(plan); - } + let Some(sort_plan) = plan_any.downcast_ref::() else { + return Ok(plan); + }; - // here we're trying to find the common prefix for sorted columns that is required for the - // sort and already satisfied by the given ordering - let child_eq_properties = child.equivalence_properties(); - let sort_req = LexRequirement::from(sort_plan.expr().clone()); + // It's safe to get first child of the SortExec + let child = Arc::clone(sort_plan.children()[0]); + if !child.boundedness().is_unbounded() { + return Ok(plan); + } - let mut common_prefix_length = 0; - while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { - inner: sort_req[0..common_prefix_length + 1].to_vec(), - }) { - common_prefix_length += 1; - } - if common_prefix_length > 0 { - return Ok(Arc::new( - PartialSortExec::new( - LexOrdering::new(sort_plan.expr().to_vec()), - Arc::clone(sort_plan.input()), - common_prefix_length, - ) - .with_preserve_partitioning(sort_plan.preserve_partitioning()) - .with_fetch(sort_plan.fetch()), - )); - } + // Here we're trying to find the common prefix for sorted columns that is required for the + // sort and already satisfied by the given ordering + let child_eq_properties = child.equivalence_properties(); + let sort_req = LexRequirement::from(sort_plan.expr().clone()); + + let mut common_prefix_length = 0; + while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { + inner: sort_req[0..common_prefix_length + 1].to_vec(), + }) { + common_prefix_length += 1; + } + if common_prefix_length > 0 { + return Ok(Arc::new( + PartialSortExec::new( + LexOrdering::new(sort_plan.expr().to_vec()), + Arc::clone(sort_plan.input()), + common_prefix_length, + ) + .with_preserve_partitioning(sort_plan.preserve_partitioning()) + .with_fetch(sort_plan.fetch()), + )); } Ok(plan) } From 3c76bdc31cf513325943a17f5942d34d93bbd789 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 15:43:36 +0300 Subject: [PATCH 003/167] add documentation to analyze_immediate_sort_removal simplify indentation --- .../src/enforce_sorting/mod.rs | 94 ++++++++++--------- 1 file changed, 49 insertions(+), 45 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index bb7641806f73c..ad549af37326d 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -391,60 +391,64 @@ pub fn ensure_sorting( update_sort_ctx_children(requirements, false).map(Transformed::yes) } -/// Analyzes a given [`SortExec`] (`plan`) to determine whether its input -/// already has a finer ordering than it enforces. +/// Analyzes if there are any immediate sort removals by checking the `SortExec`s +/// and their ordering requirement satisfactions with children +/// If the sort is unnecessary, either replaces it with `SortPreservingMergeExec`/`LimitExec` +/// or removes the `SortExec`. +/// Otherwise returns the original plan fn analyze_immediate_sort_removal( mut node: PlanWithCorrespondingSort, ) -> Transformed { - if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { - let sort_input = sort_exec.input(); - // If this sort is unnecessary, we should remove it: - if sort_input.equivalence_properties().ordering_satisfy( - sort_exec + let Some(sort_exec) = node.plan.as_any().downcast_ref::() else { + return Transformed::no(node) + }; + let sort_input = sort_exec.input(); + // Check if the sort is unnecessary + if !sort_input.equivalence_properties().ordering_satisfy( + sort_exec + .properties() + .output_ordering() + .unwrap_or(LexOrdering::empty()), + ) { + return Transformed::no(node) + }; + node.plan = if !sort_exec.preserve_partitioning() + && sort_input.output_partitioning().partition_count() > 1 + { + // Replace the sort with a sort-preserving merge: + let expr = LexOrdering::new(sort_exec.expr().to_vec()); + Arc::new( + SortPreservingMergeExec::new(expr, Arc::clone(sort_input)) + .with_fetch(sort_exec.fetch()), + ) as _ + } else { + // Remove the sort: + node.children = node.children.swap_remove(0).children; + if let Some(fetch) = sort_exec.fetch() { + // If the sort has a fetch, we need to add a limit: + if sort_exec .properties() - .output_ordering() - .unwrap_or(LexOrdering::empty()), - ) { - node.plan = if !sort_exec.preserve_partitioning() - && sort_input.output_partitioning().partition_count() > 1 + .output_partitioning() + .partition_count() + == 1 { - // Replace the sort with a sort-preserving merge: - let expr = LexOrdering::new(sort_exec.expr().to_vec()); - Arc::new( - SortPreservingMergeExec::new(expr, Arc::clone(sort_input)) - .with_fetch(sort_exec.fetch()), - ) as _ + Arc::new(GlobalLimitExec::new( + Arc::clone(sort_input), + 0, + Some(fetch), + )) } else { - // Remove the sort: - node.children = node.children.swap_remove(0).children; - if let Some(fetch) = sort_exec.fetch() { - // If the sort has a fetch, we need to add a limit: - if sort_exec - .properties() - .output_partitioning() - .partition_count() - == 1 - { - Arc::new(GlobalLimitExec::new( - Arc::clone(sort_input), - 0, - Some(fetch), - )) - } else { - Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) - } - } else { - Arc::clone(sort_input) - } - }; - for child in node.children.iter_mut() { - child.data = false; + Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) } - node.data = false; - return Transformed::yes(node); + } else { + Arc::clone(sort_input) } + }; + for child in node.children.iter_mut() { + child.data = false; } - Transformed::no(node) + node.data = false; + Transformed::yes(node) } /// Adjusts a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine From 265e00ee5375e5ad3e6de5b5486ee79bfab7139f Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 16:16:38 +0300 Subject: [PATCH 004/167] remove prefer_existing_sort effects remove prefer_existing_sort based test cases --- .../enforce_distribution.rs | 1 - .../physical_optimizer/enforce_sorting.rs | 2 - .../replace_with_order_preserving_variants.rs | 184 ++---------------- .../src/enforce_distribution.rs | 6 +- .../src/enforce_sorting/mod.rs | 11 +- .../replace_with_order_preserving_variants.rs | 8 +- datafusion/sqllogictest/test_files/joins.slt | 13 -- datafusion/sqllogictest/test_files/window.slt | 3 - 8 files changed, 28 insertions(+), 200 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 4ed5d1471c2b7..0fdb09b0d0792 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -367,7 +367,6 @@ macro_rules! plans_matches_expected { /// * `FIRST_ENFORCE_DIST` - /// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) /// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) -/// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted /// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to /// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans /// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 473346684b940..920d1ecd3ee54 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -152,7 +152,6 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - &config, ) }) .data() @@ -1476,7 +1475,6 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - &config, ) }) .data() diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index d9b569dfa6115..770e099267758 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -45,46 +45,12 @@ use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; -use datafusion_common::config::ConfigOptions; use object_store::memory::InMemory; use object_store::ObjectStore; use rstest::rstest; use url::Url; -/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts -/// the plan against the original and expected plans. -/// -/// # Parameters -/// -/// * `$EXPECTED_PLAN_LINES`: Expected input plan. -/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag -/// `prefer_existing_sort` is `false`. -/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when -/// the flag `prefer_existing_sort` is `true`. -/// * `$PLAN`: The plan to optimize. -macro_rules! assert_optimized_prefer_sort_on_off { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { - if $PREFER_EXISTING_SORT { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } else { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } - }; -} - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans for both bounded and /// unbounded cases. @@ -93,32 +59,24 @@ macro_rules! assert_optimized_prefer_sort_on_off { /// /// * `EXPECTED_UNBOUNDED_PLAN_LINES`: Expected input unbounded plan. /// * `EXPECTED_BOUNDED_PLAN_LINES`: Expected input bounded plan. -/// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan, which is -/// the same regardless of the value of the `prefer_existing_sort` flag. -/// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag -/// `prefer_existing_sort` is `false` for bounded cases. -/// * `EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan -/// when the flag `prefer_existing_sort` is `true` for bounded cases. +/// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan for unbounded +/// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan for bounded /// * `$PLAN`: The plan to optimize. /// * `$SOURCE_UNBOUNDED`: Whether the given plan contains an unbounded source. macro_rules! assert_optimized_in_all_boundedness_situations { - ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr, $PREFER_EXISTING_SORT: expr) => { + ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr) => { if $SOURCE_UNBOUNDED { - assert_optimized_prefer_sort_on_off!( + assert_optimized!( $EXPECTED_UNBOUNDED_PLAN_LINES, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, - $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, $PLAN, - $PREFER_EXISTING_SORT, $SOURCE_UNBOUNDED ); } else { - assert_optimized_prefer_sort_on_off!( + assert_optimized!( $EXPECTED_BOUNDED_PLAN_LINES, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES, - $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, $PLAN, - $PREFER_EXISTING_SORT, $SOURCE_UNBOUNDED ); } @@ -133,10 +91,9 @@ macro_rules! assert_optimized_in_all_boundedness_situations { /// * `$EXPECTED_PLAN_LINES`: Expected input plan. /// * `$EXPECTED_OPTIMIZED_PLAN_LINES`: Expected optimized plan. /// * `$PLAN`: The plan to optimize. -/// * `$PREFER_EXISTING_SORT`: Value of the `prefer_existing_sort` flag. #[macro_export] macro_rules! assert_optimized { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr) => { let physical_plan = $PLAN; let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -152,10 +109,8 @@ macro_rules! assert_optimized { let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES.iter().map(|s| *s).collect(); // Run the rule top-down - let mut config = ConfigOptions::new(); - config.optimizer.prefer_existing_sort=$PREFER_EXISTING_SORT; let plan_with_pipeline_fixer = OrderPreservationContext::new_default(physical_plan); - let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, &config)).data().and_then(check_integrity)?; + let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false)).data().and_then(check_integrity)?; let optimized_physical_plan = parallel.plan; // Get string representation of the plan @@ -185,7 +140,6 @@ macro_rules! assert_optimized { // Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected async fn test_replace_multiple_input_repartition_1( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -231,21 +185,13 @@ async fn test_replace_multiple_input_repartition_1( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -254,7 +200,6 @@ async fn test_replace_multiple_input_repartition_1( #[tokio::test] async fn test_with_inter_children_change_only( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_default("a", &schema)]; @@ -330,25 +275,13 @@ async fn test_with_inter_children_change_only( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortPreservingMergeExec: [a@0 ASC]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -357,7 +290,6 @@ async fn test_with_inter_children_change_only( #[tokio::test] async fn test_replace_multiple_input_repartition_2( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -409,22 +341,13 @@ async fn test_replace_multiple_input_repartition_2( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -433,7 +356,6 @@ async fn test_replace_multiple_input_repartition_2( #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -490,23 +412,13 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -515,7 +427,6 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -577,24 +488,13 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -603,7 +503,6 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[tokio::test] async fn test_not_replacing_when_no_need_to_preserve_sorting( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -657,17 +556,14 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -676,7 +572,6 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( #[tokio::test] async fn test_with_multiple_replacable_repartitions( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -738,24 +633,13 @@ async fn test_with_multiple_replacable_repartitions( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -764,7 +648,6 @@ async fn test_with_multiple_replacable_repartitions( #[tokio::test] async fn test_not_replace_with_different_orderings( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -817,17 +700,14 @@ async fn test_not_replace_with_different_orderings( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -836,7 +716,6 @@ async fn test_not_replace_with_different_orderings( #[tokio::test] async fn test_with_lost_ordering( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -883,21 +762,13 @@ async fn test_with_lost_ordering( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -906,7 +777,6 @@ async fn test_with_lost_ordering( #[tokio::test] async fn test_with_lost_and_kept_ordering( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -983,26 +853,14 @@ async fn test_with_lost_and_kept_ordering( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [c@1 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } @@ -1011,7 +869,6 @@ async fn test_with_lost_and_kept_ordering( #[tokio::test] async fn test_with_multiple_child_trees( #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; @@ -1106,17 +963,14 @@ async fn test_with_multiple_child_trees( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded, - prefer_existing_sort + source_unbounded ); Ok(()) } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index c3cd86fb1498b..ef828592f8a70 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -853,8 +853,7 @@ fn add_roundrobin_on_top( // during repartition. This will be un-done in the future // If any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements - // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.prefer_existing_sort`) + // - Required input ordering is a hard requirement let partitioning = Partitioning::RoundRobinBatch(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? @@ -912,8 +911,7 @@ fn add_hash_on_top( // following conditions is true: // - Preserving ordering is not helpful in terms of satisfying ordering // requirements. - // - Usage of order preserving variants is not desirable (per the flag - // `config.optimizer.prefer_existing_sort`). + // - Required input ordering is a hard requirement let partitioning = dist.create_partitioning(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index ad549af37326d..c9f15995f4ffa 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -184,7 +184,6 @@ impl PhysicalOptimizerRule for EnforceSorting { plan_with_pipeline_fixer, false, true, - config, ) }) .data()?; @@ -400,7 +399,7 @@ fn analyze_immediate_sort_removal( mut node: PlanWithCorrespondingSort, ) -> Transformed { let Some(sort_exec) = node.plan.as_any().downcast_ref::() else { - return Transformed::no(node) + return Transformed::no(node); }; let sort_input = sort_exec.input(); // Check if the sort is unnecessary @@ -410,7 +409,7 @@ fn analyze_immediate_sort_removal( .output_ordering() .unwrap_or(LexOrdering::empty()), ) { - return Transformed::no(node) + return Transformed::no(node); }; node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 @@ -432,11 +431,7 @@ fn analyze_immediate_sort_removal( .partition_count() == 1 { - Arc::new(GlobalLimitExec::new( - Arc::clone(sort_input), - 0, - Some(fetch), - )) + Arc::new(GlobalLimitExec::new(Arc::clone(sort_input), 0, Some(fetch))) } else { Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 92012dfa12889..1ddb279e0dd7c 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -25,12 +25,11 @@ use crate::utils::{ is_coalesce_partitions, is_repartition, is_sort, is_sort_preserving_merge, }; -use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::execution_plan::{EmissionType, RequiredInputOrdering}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; @@ -238,16 +237,17 @@ pub fn replace_with_order_preserving_variants( // `SortExec` from the plan. If this flag is `false`, this replacement // should only be made to fix the pipeline (streaming). is_spm_better: bool, - config: &ConfigOptions, ) -> Result> { update_children(&mut requirements); if !(is_sort(&requirements.plan) && requirements.children[0].data) { return Ok(Transformed::no(requirements)); } + let requirement = requirements.plan.required_input_ordering()[0].clone(); // For unbounded cases, we replace with the order-preserving variant in any // case, as doing so helps fix the pipeline. Also replace if config allows. - let use_order_preserving_variant = config.optimizer.prefer_existing_sort + let use_order_preserving_variant = (requirement.is_some() + && matches!(requirement.unwrap(), RequiredInputOrdering::Hard(_))) || (requirements.plan.boundedness().is_unbounded() && requirements.plan.pipeline_behavior() == EmissionType::Final); diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ec512bc261dc2..7e903b70b1e60 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3220,9 +3220,6 @@ WITH ORDER (a ASC NULLS FIRST, b ASC, c ASC) LOCATION '../core/tests/data/window_2.csv' OPTIONS ('format.has_header' 'true'); -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - # sort merge join should propagate ordering equivalence of the left side # for inner join. Hence final requirement rn1 ASC is already satisfied at # the end of SortMergeJoinExec. @@ -3291,9 +3288,6 @@ physical_plan 11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 12)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -statement ok -set datafusion.optimizer.prefer_existing_sort = false; - # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering # below query shouldn't add any SortExec for order by clause. @@ -3475,10 +3469,6 @@ physical_plan statement ok set datafusion.execution.target_partitions = 2; -# use bounded variants -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 FROM annotated_data as l @@ -3564,9 +3554,6 @@ set datafusion.optimizer.prefer_hash_join = true; statement ok set datafusion.execution.target_partitions = 2; -statement ok -set datafusion.optimizer.prefer_existing_sort = false; - statement ok drop table annotated_data; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index ca4713e7d5160..0e967eae0ef06 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5476,9 +5476,6 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' WITH ORDER (c1) OPTIONS ('format.has_header' 'true'); -statement ok -set datafusion.optimizer.prefer_existing_sort = true; - query TT EXPLAIN SELECT c1, SUM(c9) OVER(PARTITION BY c1) as sum_c9 FROM aggregate_test_100_ordered ORDER BY c1, sum_c9; ---- From 23e0bf667f7d95aa5bfad1a599d44171008edc39 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 16:23:05 +0300 Subject: [PATCH 005/167] remove prefer_existing_sort configuration --- datafusion/common/src/config.rs | 8 -------- datafusion/execution/src/config.rs | 17 ----------------- .../test_files/information_schema.slt | 2 -- docs/source/user-guide/configs.md | 1 - 4 files changed, 28 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index c9900204b97f2..15d8c04371f81 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -635,14 +635,6 @@ config_namespace! { /// ``` pub repartition_sorts: bool, default = true - /// When true, DataFusion will opportunistically remove sorts when the data is already sorted, - /// (i.e. setting `preserve_order` to true on `RepartitionExec` and - /// using `SortPreservingMergeExec`) - /// - /// When false, DataFusion will maximize plan parallelism using - /// `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. - pub prefer_existing_sort: bool, default = false - /// When set to true, the logical plan optimizer will produce warning /// messages if any optimization rules produce errors and then proceed to the next /// rule. When set to false, any rules that produce errors will cause the query to fail diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 53646dc5b468e..28da568470af1 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -243,14 +243,6 @@ impl SessionConfig { self.options.optimizer.repartition_sorts } - /// Prefer existing sort (true) or maximize parallelism (false). See - /// [prefer_existing_sort] for more details - /// - /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort - pub fn prefer_existing_sort(&self) -> bool { - self.options.optimizer.prefer_existing_sort - } - /// Are statistics collected during execution? pub fn collect_statistics(&self) -> bool { self.options.execution.collect_statistics @@ -321,15 +313,6 @@ impl SessionConfig { self } - /// Prefer existing sort (true) or maximize parallelism (false). See - /// [prefer_existing_sort] for more details - /// - /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort - pub fn with_prefer_existing_sort(mut self, enabled: bool) -> Self { - self.options.optimizer.prefer_existing_sort = enabled; - self - } - /// Prefer existing union (true). See [prefer_existing_union] for more details /// /// [prefer_existing_union]: datafusion_common::config::OptimizerOptions::prefer_existing_union diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 5a1caad46732c..4002ed475dbab 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -246,7 +246,6 @@ datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 datafusion.optimizer.max_passes 3 -datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true datafusion.optimizer.repartition_aggregations true @@ -341,7 +340,6 @@ datafusion.optimizer.filter_null_join_keys false When set to true, the optimizer datafusion.optimizer.hash_join_single_partition_threshold 1048576 The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition datafusion.optimizer.max_passes 3 Number of times that the optimizer will attempt to optimize the plan -datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 999735f4c0592..0faeae0b2591b 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -106,7 +106,6 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | | datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | From ee626d1cf1ed862e46fdac917a868a4264b4ae13 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 17:25:07 +0300 Subject: [PATCH 006/167] remove prefer_existing_sort configuration --- .../src/enforce_sorting/mod.rs | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index c9f15995f4ffa..85e880b88be33 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -264,6 +264,16 @@ fn replace_with_partial_sort( /// ``` /// by following connections from [`CoalescePartitionsExec`]s to [`SortExec`]s. /// By performing sorting in parallel, we can increase performance in some scenarios. +/// +/// **Steps** +/// 1. Checks if the plan is either `SortExec`/`SortPreservingMergeExec`/`CoalescePartitionsExec` otherwise does nothing +/// 2. If the plan is a `SortExec` or a final `SortPreservingMergeExec` (output partitioning is 1) +/// 2.1. Check for `CoalescePartitionsExec` in children, when found check if it can be removed, if so remove. (see `remove_bottleneck_in_subplanæ) +/// 2.2. Remove the current plan +/// 2.3. If the plan is satisfying the ordering requirements, add a `SortExec` +/// 2.4. Add a SPM above the plan and return +/// 3. If the plan is a `CoalescePartitionsExec` +/// 3.1. pub fn parallelize_sorts( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result> { @@ -326,6 +336,20 @@ pub fn parallelize_sorts( /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. +/// +/// **Steps** +/// 1. Analyze if there are any immediate removals of `SortExec`s if so, removes them (see `analyze_immediate_sort_removal`) +/// 2. For each child of the plan, if the plan requires input ordering +/// 2.1. Checks if ordering is satisfied with the child, if it's not satisfied +/// 2.1.1. If the child has output ordering, removes the unnecessary `SortExec` +/// 2.1.2. Adds sort above the child plan +/// 2.2 (Plan not requires input ordering) +/// 2.2.1 Checks if the `SortExec` is neutralized in the plan, if so removes it. +/// 3. Check and modify window operator +/// 3.1. Checks if the plan is a window operator, and connected with a sort. +/// If so, either tries to update the window definition or removes unnecessary `SortExec`s (see `adjust_window_sort_removal`) +/// 4. Check and remove possibly unnecessary SPM +/// 4.1. Checks if the plan is SPM and child 1 output partitions, if so decides this SPM is unnecessary and removes it from the plan. pub fn ensure_sorting( mut requirements: PlanWithCorrespondingSort, ) -> Result> { From 0b9838237ce126a208cb635ea1b027a20fe72c9b Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 12 Feb 2025 17:33:14 +0300 Subject: [PATCH 007/167] add documentation --- .../physical-optimizer/src/enforce_sorting/mod.rs | 11 ++++++----- .../src/enforce_sorting/sort_pushdown.rs | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 85e880b88be33..1f9c7041a294a 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -268,12 +268,13 @@ fn replace_with_partial_sort( /// **Steps** /// 1. Checks if the plan is either `SortExec`/`SortPreservingMergeExec`/`CoalescePartitionsExec` otherwise does nothing /// 2. If the plan is a `SortExec` or a final `SortPreservingMergeExec` (output partitioning is 1) -/// 2.1. Check for `CoalescePartitionsExec` in children, when found check if it can be removed, if so remove. (see `remove_bottleneck_in_subplanæ) -/// 2.2. Remove the current plan -/// 2.3. If the plan is satisfying the ordering requirements, add a `SortExec` -/// 2.4. Add a SPM above the plan and return +/// 2.1. Check for `CoalescePartitionsExec` in children, when found check if it can be removed (with possible `RepartitionExec`s) +/// if so remove. (see `remove_bottleneck_in_subplan`) +/// 2.2. If the plan is satisfying the ordering requirements, add a `SortExec` +/// 2.3. Add an SPM above the plan and return /// 3. If the plan is a `CoalescePartitionsExec` -/// 3.1. +/// 3.1. Check if it can be removed (with possible `RepartitionExec`s) +/// if so remove (see `remove_bottleneck_in_subplan`) pub fn parallelize_sorts( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result> { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 6af1e8f326c07..a08c2fb0bdd00 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -72,6 +72,7 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } +/// Tried to push down the sort requirements as far as possible, if decides a `SortExec` is unnecessary removes it. pub fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { let mut new_node = pushdown_sorts_helper(sort_pushdown)?; while new_node.tnr == TreeNodeRecursion::Stop { From a41dfec3f6c1b67182dd2b164078e4c3cb3c3d8d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 13 Feb 2025 11:41:42 +0300 Subject: [PATCH 008/167] add documentation --- .../physical-optimizer/src/enforce_sorting/mod.rs | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 1f9c7041a294a..8bfc154d086fa 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -156,6 +156,18 @@ fn update_coalesce_ctx_children( /// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades /// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to /// perform sorting in parallel. +/// +/// Optimizer consists of 5 main parts which work sequentially +/// 1. `ensure_sorting` Responsible for removing unnecessary [`SortExec`]s, [`SortPreservingMergeExec`]s +/// adjusting window operators, etc. +/// 2. `parallelize_sorts` (Depends on the repartition_sorts configuration) Responsible to identify +/// and remove unnecessary partition unifier operators such as [`SortPreservingMergeExec`], [`CoalescePartitionsExec`] +/// follows [`SortExec`]s does possible simplifications. +/// 3. `replace_with_order_preserving_variants` Replaces operators with order preserving variants, for example can merge +/// a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`] or [`SortExec`] + [`RepartitionExec`] +/// into an order preserving [`RepartitionExec`], etc. +/// 4. `sort_pushdown` Responsible to push down sort operators as deep as possible in the plan. +/// 5. `replace_with_partial_sort` Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, From f6dae9d3d0d8fda023b328ce5fdeb6fe5657df79 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 13 Feb 2025 11:45:58 +0300 Subject: [PATCH 009/167] add documentation --- .../src/enforce_sorting/mod.rs | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 8bfc154d086fa..3a73e5f91c2ca 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -219,10 +219,10 @@ impl PhysicalOptimizerRule for EnforceSorting { } } -/// Only interested with `SortExec`s and their unbounded children. -/// If the plan is not a `SortExec` or its child is not unbounded, returns the original plan. +/// Only interested with [`SortExec`]s and their unbounded children. +/// If the plan is not a [`SortExec`] or its child is not unbounded, returns the original plan. /// Otherwise, by checking the requirement satisfaction searches for a replacement chance. -/// If there's one replaces the `SortExec` plan with a PartialSortExec +/// If there's one replaces the [`SortExec`] plan with a [`PartialSortExec`] fn replace_with_partial_sort( plan: Arc, ) -> Result> { @@ -278,14 +278,14 @@ fn replace_with_partial_sort( /// By performing sorting in parallel, we can increase performance in some scenarios. /// /// **Steps** -/// 1. Checks if the plan is either `SortExec`/`SortPreservingMergeExec`/`CoalescePartitionsExec` otherwise does nothing -/// 2. If the plan is a `SortExec` or a final `SortPreservingMergeExec` (output partitioning is 1) -/// 2.1. Check for `CoalescePartitionsExec` in children, when found check if it can be removed (with possible `RepartitionExec`s) +/// 1. Checks if the plan is either [`SortExec`]/[`SortPreservingMergeExec`]/[`CoalescePartitionsExec`] otherwise does nothing +/// 2. If the plan is a [`SortExec`] or a final `[SortPreservingMergeExec` (output partitioning is 1) +/// 2.1. Check for [`CoalescePartitionsExec`] in children, when found check if it can be removed (with possible [`RepartitionExec`]s) /// if so remove. (see `remove_bottleneck_in_subplan`) /// 2.2. If the plan is satisfying the ordering requirements, add a `SortExec` /// 2.3. Add an SPM above the plan and return -/// 3. If the plan is a `CoalescePartitionsExec` -/// 3.1. Check if it can be removed (with possible `RepartitionExec`s) +/// 3. If the plan is a [`CoalescePartitionsExec`] +/// 3.1. Check if it can be removed (with possible [`RepartitionExec`]s) /// if so remove (see `remove_bottleneck_in_subplan`) pub fn parallelize_sorts( mut requirements: PlanWithCorrespondingCoalescePartitions, @@ -351,7 +351,7 @@ pub fn parallelize_sorts( /// violating these requirements whenever possible. /// /// **Steps** -/// 1. Analyze if there are any immediate removals of `SortExec`s if so, removes them (see `analyze_immediate_sort_removal`) +/// 1. Analyze if there are any immediate removals of [`SortExec`]s if so, removes them (see `analyze_immediate_sort_removal`) /// 2. For each child of the plan, if the plan requires input ordering /// 2.1. Checks if ordering is satisfied with the child, if it's not satisfied /// 2.1.1. If the child has output ordering, removes the unnecessary `SortExec` @@ -360,7 +360,7 @@ pub fn parallelize_sorts( /// 2.2.1 Checks if the `SortExec` is neutralized in the plan, if so removes it. /// 3. Check and modify window operator /// 3.1. Checks if the plan is a window operator, and connected with a sort. -/// If so, either tries to update the window definition or removes unnecessary `SortExec`s (see `adjust_window_sort_removal`) +/// If so, either tries to update the window definition or removes unnecessary [`SortExec`]s (see `adjust_window_sort_removal`) /// 4. Check and remove possibly unnecessary SPM /// 4.1. Checks if the plan is SPM and child 1 output partitions, if so decides this SPM is unnecessary and removes it from the plan. pub fn ensure_sorting( @@ -429,9 +429,9 @@ pub fn ensure_sorting( /// Analyzes if there are any immediate sort removals by checking the `SortExec`s /// and their ordering requirement satisfactions with children -/// If the sort is unnecessary, either replaces it with `SortPreservingMergeExec`/`LimitExec` -/// or removes the `SortExec`. -/// Otherwise returns the original plan +/// If the sort is unnecessary, either replaces it with [`SortPreservingMergeExec`]/`[LimitExec`] +/// or removes the [`SortExec`]. +/// Otherwise, returns the original plan fn analyze_immediate_sort_removal( mut node: PlanWithCorrespondingSort, ) -> Transformed { From d551623f401cd781bbcd82cd33e78ee8b88c8ae5 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 10 Mar 2025 11:40:22 +0300 Subject: [PATCH 010/167] fix imports and test cases --- .../examples/custom_file_format.rs | 7 +- .../enforce_distribution.rs | 76 +++---------------- datafusion/datasource-csv/src/file_format.rs | 4 +- datafusion/datasource-json/src/file_format.rs | 4 +- .../datasource-parquet/src/file_format.rs | 4 +- datafusion/datasource/src/file_format.rs | 5 +- .../src/enforce_sorting/sort_pushdown.rs | 26 ++++--- 7 files changed, 35 insertions(+), 91 deletions(-) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index ca3ca774aa706..30033cba4d2d8 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -21,25 +21,20 @@ use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use datafusion::datasource::data_source::FileSource; use datafusion::execution::session_state::SessionStateBuilder; -use datafusion::physical_expr::LexRequirement; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::{ catalog::Session, common::{GetExt, Statistics}, }; -use datafusion::{ - datasource::physical_plan::FileSource, execution::session_state::SessionStateBuilder, -}; use datafusion::{ datasource::{ file_format::{ csv::CsvFormatFactory, file_compression_type::FileCompressionType, FileFormat, FileFormatFactory, }, - physical_plan::{FileScanConfig, FileSinkConfig}, + physical_plan::{FileScanConfig, FileSinkConfig, FileSource}, MemTable, }, error::Result, diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 39e6a32ce874f..47366f9bab2bb 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -372,9 +372,6 @@ macro_rules! plans_matches_expected { fn test_suite_default_config_options() -> ConfigOptions { let mut config = ConfigOptions::new(); - // By default, will not repartition / resort data if it is already sorted. - config.optimizer.prefer_existing_sort = false; - // By default, will attempt to convert Union to Interleave. config.optimizer.prefer_existing_union = false; @@ -1784,17 +1781,6 @@ fn merge_does_not_need_sort() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, &TestConfig::new(DoFirst::Distribution)); - - // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: PREFER_EXISTING_SORT) - // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with - // SortExec at the top. - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=4096", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; assert_optimized!(expected, exec, &TestConfig::new(DoFirst::Sorting)); Ok(()) @@ -2133,12 +2119,6 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) @@ -2168,14 +2148,6 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) @@ -2208,11 +2180,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - assert_optimized!( - expected, - plan, - &TestConfig::new(DoFirst::Sorting) - ); + assert_optimized!(expected, plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -2522,8 +2490,8 @@ fn parallelization_multiple_files() -> Result<()> { let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); - let test_config = TestConfig::new(DoFirst::Distribution) - .with_prefer_repartition_file_scans(1); + let test_config = + TestConfig::new(DoFirst::Distribution).with_prefer_repartition_file_scans(1); // The groups must have only contiguous ranges of rows from the same file // if any group has rows from multiple files, the data is no longer sorted destroyed @@ -3106,11 +3074,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { physical_plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - assert_optimized!( - expected, - physical_plan, - &TestConfig::new(DoFirst::Sorting) - ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -3136,11 +3100,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { physical_plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - assert_optimized!( - expected, - physical_plan, - &TestConfig::new(DoFirst::Sorting) - ); + assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) } @@ -3157,10 +3117,9 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!( @@ -3168,14 +3127,6 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { physical_plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) @@ -3196,7 +3147,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { "CoalescePartitionsExec", // Since after this stage c is constant. c@2 ASC ordering is already satisfied. " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!( @@ -3237,15 +3188,6 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { physical_plan.clone(), &TestConfig::new(DoFirst::Distribution) ); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); Ok(()) diff --git a/datafusion/datasource-csv/src/file_format.rs b/datafusion/datasource-csv/src/file_format.rs index cab561d163b36..8fcfd891966a0 100644 --- a/datafusion/datasource-csv/src/file_format.rs +++ b/datafusion/datasource-csv/src/file_format.rs @@ -49,7 +49,7 @@ use datafusion_datasource::write::BatchSerializer; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use async_trait::async_trait; use bytes::{Buf, Bytes}; @@ -435,7 +435,7 @@ impl FileFormat for CsvFormat { input: Arc, state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for CSV"); diff --git a/datafusion/datasource-json/src/file_format.rs b/datafusion/datasource-json/src/file_format.rs index 2df49b5351286..f4070eb037395 100644 --- a/datafusion/datasource-json/src/file_format.rs +++ b/datafusion/datasource-json/src/file_format.rs @@ -51,12 +51,12 @@ use datafusion_datasource::write::BatchSerializer; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use async_trait::async_trait; use bytes::{Buf, Bytes}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; use crate::source::JsonSource; @@ -259,7 +259,7 @@ impl FileFormat for JsonFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Json"); diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 48761d85e7082..32b83d7278c51 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -55,7 +55,7 @@ use datafusion_expr::dml::InsertOp; use datafusion_expr::Expr; use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::Accumulator; use async_trait::async_trait; @@ -428,7 +428,7 @@ impl FileFormat for ParquetFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Parquet"); diff --git a/datafusion/datasource/src/file_format.rs b/datafusion/datasource/src/file_format.rs index aa0338fab71dd..29709aca81f76 100644 --- a/datafusion/datasource/src/file_format.rs +++ b/datafusion/datasource/src/file_format.rs @@ -34,7 +34,8 @@ use datafusion_catalog::Session; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt, Result, Statistics}; use datafusion_expr::Expr; -use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::ExecutionPlan; use object_store::{ObjectMeta, ObjectStore}; @@ -104,7 +105,7 @@ pub trait FileFormat: Send + Sync + fmt::Debug { _input: Arc, _state: &dyn Session, _conf: FileSinkConfig, - _order_requirements: Option, + _order_requirements: Option, ) -> Result> { not_impl_err!("Writer not implemented for this format") } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index fd2a29b734602..2502aa6f275b6 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -109,12 +109,14 @@ fn pushdown_sorts_helper( .map(LexRequirement::from) .map(RequiredInputOrdering::Hard) .unwrap_or_default(); - let parent_is_stricter = plan - .equivalence_properties() - .requirements_compatible(&parent_reqs, ¤t_plan_reqs); - let current_is_stricter = plan - .equivalence_properties() - .requirements_compatible(¤t_plan_reqs, &parent_reqs); + let parent_is_stricter = plan.equivalence_properties().requirements_compatible( + parent_reqs.lex_requirement(), + current_plan_reqs.lex_requirement(), + ); + let current_is_stricter = plan.equivalence_properties().requirements_compatible( + current_plan_reqs.lex_requirement(), + parent_reqs.lex_requirement(), + ); if !satisfy_parent && !parent_is_stricter { // This new sort has different requirements than the ordering being pushed down. @@ -127,8 +129,11 @@ fn pushdown_sorts_helper( sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan // add back sort exec matching parent - sort_push_down = - add_sort_above(sort_push_down, parent_reqs.lex_requirement().clone(), parent_req_fetch); + sort_push_down = add_sort_above( + sort_push_down, + parent_reqs.lex_requirement().clone(), + parent_req_fetch, + ); // make pushdown requirements be the new ones. sort_push_down.children[0].data = ParentRequirements { @@ -188,7 +193,8 @@ fn pushdown_sorts_helper( .clone() .unwrap_or_default(); let fetch = sort_push_down.data.fetch; - sort_push_down = add_sort_above(sort_push_down, sort_reqs.lex_requirement().clone(), fetch); + sort_push_down = + add_sort_above(sort_push_down, sort_reqs.lex_requirement().clone(), fetch); assign_initial_requirements(&mut sort_push_down); } @@ -224,7 +230,7 @@ fn pushdown_requirement_to_children( // that's the case, we block the pushdown of sort operation. if !plan .equivalence_properties() - .ordering_satisfy_requirement(parent_required) + .ordering_satisfy_requirement(parent_required.lex_requirement()) { return Ok(None); } From 9a1b5b5d698c29ea4c8aa89c7d75adc928652598 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 10 Mar 2025 11:42:00 +0300 Subject: [PATCH 011/167] fix imports and test cases --- .../enforce_distribution.rs | 31 ++----------------- 1 file changed, 2 insertions(+), 29 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 47366f9bab2bb..ff952f13d324f 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3105,33 +3105,6 @@ fn preserve_ordering_through_repartition() -> Result<()> { Ok(()) } -#[test] -fn do_not_preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - - assert_optimized!( - expected, - physical_plan.clone(), - &TestConfig::new(DoFirst::Distribution) - ); - assert_optimized!(expected, physical_plan, &TestConfig::new(DoFirst::Sorting)); - - Ok(()) -} - #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); @@ -3161,7 +3134,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { } #[test] -fn do_not_preserve_ordering_through_repartition2() -> Result<()> { +fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), @@ -3194,7 +3167,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { } #[test] -fn do_not_preserve_ordering_through_repartition3() -> Result<()> { +fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), From 77e97fdd85c0ad6a3679340cb7e90291507fab47 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 19 Mar 2025 13:31:43 +0300 Subject: [PATCH 012/167] implement RequiredInputOrdering as vectors --- .../core/src/datasource/listing/table.rs | 4 +-- datafusion/core/src/datasource/stream.rs | 7 ++-- .../physical_optimizer/projection_pushdown.rs | 34 ++++++++++--------- .../src/enforce_sorting/sort_pushdown.rs | 19 +++++++---- .../src/output_requirements.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/execution_plan.rs | 25 +++++++------- .../src/joins/symmetric_hash_join.rs | 8 ++--- datafusion/physical-plan/src/windows/mod.rs | 8 ++--- datafusion/proto/src/physical_plan/mod.rs | 11 +++--- .../tests/cases/roundtrip_physical_plan.rs | 8 ++--- 11 files changed, 65 insertions(+), 63 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 5ff3f9390bdcc..71155ad192818 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1058,13 +1058,13 @@ impl TableProvider for ListingTable { ); }; // Converts Vec> into type required by execution plan to specify its required input ordering - Some(RequiredInputOrdering::Hard(LexRequirement::new( + Some(RequiredInputOrdering::Hard(vec![LexRequirement::new( ordering .into_iter() .cloned() .map(PhysicalSortRequirement::from) .collect::>(), - ))) + )])) } else { None }; diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 4ca26f6213cbd..bd19e0f334c70 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -358,9 +358,10 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, std::slice::from_ref(x))?; let ordering = orders.into_iter().next().unwrap(); - Some(RequiredInputOrdering::Hard( - ordering.into_iter().map(Into::into).collect(), - )) + Some(RequiredInputOrdering::Hard(vec![ordering + .into_iter() + .map(Into::into) + .collect()])) } None => None, }; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index aead1e908eff4..a6fe97bc1ba3f 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -642,20 +642,22 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv.clone(), - Some(RequiredInputOrdering::Hard(LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ]))), + Some(RequiredInputOrdering::Hard(vec![LexRequirement::new( + vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 1)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: Some(SortOptions::default()), + }, + ], + )])), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -688,7 +690,7 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = RequiredInputOrdering::Hard(LexRequirement::new(vec![ + let expected_reqs = RequiredInputOrdering::Hard(vec![LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 2)), options: Some(SortOptions::default()), @@ -701,7 +703,7 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ])); + ])]); assert_eq!( after_optimize .as_any() diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 2502aa6f275b6..e1db530b39b7b 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -106,8 +106,7 @@ fn pushdown_sorts_helper( let current_plan_reqs = plan .output_ordering() .cloned() - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard) + .map(RequiredInputOrdering::from) .unwrap_or_default(); let parent_is_stricter = plan.equivalence_properties().requirements_compatible( parent_reqs.lex_requirement(), @@ -217,9 +216,9 @@ fn pushdown_requirement_to_children( { RequirementsCompatibility::Satisfy => { let req = (!request_child.is_empty()).then(|| { - RequiredInputOrdering::Hard(LexRequirement::new( + RequiredInputOrdering::Hard(vec![LexRequirement::new( request_child.to_vec(), - )) + )]) }); Ok(Some(vec![req])) } @@ -282,7 +281,9 @@ fn pushdown_requirement_to_children( .requirements_compatible(parent_required.lex_requirement(), &output_req) { let req = (!parent_required.is_empty()).then(|| { - RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + RequiredInputOrdering::Hard(vec![LexRequirement::new( + parent_required.to_vec(), + )]) }); Ok(Some(vec![req])) } else { @@ -355,7 +356,9 @@ fn pushdown_requirement_to_children( // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. let req = (!parent_required.is_empty()).then(|| { - RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + RequiredInputOrdering::Hard(vec![LexRequirement::new( + parent_required.to_vec(), + )]) }); Ok(Some(vec![req])) } @@ -414,7 +417,9 @@ fn determine_children_requirement( // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: let adjusted = (!parent_required.lex_requirement().is_empty()).then(|| { - RequiredInputOrdering::Hard(LexRequirement::new(parent_required.to_vec())) + RequiredInputOrdering::Hard(vec![LexRequirement::new( + parent_required.to_vec(), + )]) }); RequirementsCompatibility::Compatible(adjusted) } else { diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 89ab59810e599..4a3519ba71716 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -248,7 +248,7 @@ impl ExecutionPlan for OutputRequirementExec { OutputRequirementExec::new( input, (!updated_sort_reqs.is_empty()) - .then_some(RequiredInputOrdering::Hard(updated_sort_reqs)), + .then_some(RequiredInputOrdering::Hard(vec![updated_sort_reqs])), dist_req, ) }) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 407a1a4c70907..b880c7f3eecc7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -522,7 +522,7 @@ impl AggregateExec { ProjectionMapping::try_new(&group_by.expr, &input.schema())?; let required_input_ordering = (!new_requirement.is_empty()) - .then_some(RequiredInputOrdering::Soft(new_requirement)); + .then_some(RequiredInputOrdering::Soft(vec![new_requirement])); let cache = Self::compute_properties( &input, diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 81d39e56e7131..f762d44e4effd 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -1053,33 +1053,34 @@ pub enum CardinalityEffect { GreaterEqual, } -/// Represents the plan's input ordering requirement +/// Represents the plan's input ordering requirements, +/// the elements of the vectors' represent alternative requirements #[derive(Debug, Clone, PartialEq)] pub enum RequiredInputOrdering { - /// The operator can not work without this ordering - Hard(LexRequirement), - /// The operator can benefit from the ordering if provided - /// but if not provided it can also work - Soft(LexRequirement), + /// The operator is not able to work without one of these requirements + Hard(Vec), + /// The operator can benefit from the ordering alternatives if provided + /// but if not provided it can also work. + Soft(Vec), } impl Default for RequiredInputOrdering { fn default() -> Self { - RequiredInputOrdering::Hard(LexRequirement::default()) + RequiredInputOrdering::Hard(vec![LexRequirement::default()]) } } impl From for RequiredInputOrdering { fn from(ordering: LexOrdering) -> Self { - RequiredInputOrdering::Hard(LexRequirement::from(ordering)) + RequiredInputOrdering::Hard(vec![LexRequirement::from(ordering)]) } } impl RequiredInputOrdering { pub fn lex_requirement(&self) -> &LexRequirement { match self { - RequiredInputOrdering::Hard(lex) => lex, - RequiredInputOrdering::Soft(lex) => lex, + RequiredInputOrdering::Hard(lex) => &lex[0], + RequiredInputOrdering::Soft(lex) => &lex[0], } } @@ -1097,10 +1098,10 @@ impl RequiredInputOrdering { ) -> Self { match self { RequiredInputOrdering::Hard(_) => { - RequiredInputOrdering::Hard(LexRequirement::new(requirement)) + RequiredInputOrdering::Hard(vec![LexRequirement::new(requirement)]) } RequiredInputOrdering::Soft(_) => { - RequiredInputOrdering::Soft(LexRequirement::new(requirement)) + RequiredInputOrdering::Soft(vec![LexRequirement::new(requirement)]) } } } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index e55462cfa94d5..c3e283415798a 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -426,14 +426,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { vec![ self.left_sort_exprs .as_ref() - .cloned() - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard), + .map(|e| RequiredInputOrdering::from(e.clone())), self.right_sort_exprs .as_ref() - .cloned() - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard), + .map(|e| RequiredInputOrdering::from(e.clone())), ] } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 3edee33ec4631..517a1786d2d8c 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -301,9 +301,9 @@ pub(crate) fn calc_requirements< } if is_soft_requirements { - (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Soft(sort_reqs)) + (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Soft(vec![sort_reqs])) } else { - (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Hard(sort_reqs)) + (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Hard(vec![sort_reqs])) } } @@ -779,9 +779,9 @@ mod tests { } let expected_result = if let Some(expected) = expected { if is_soft { - Some(RequiredInputOrdering::Soft(expected)) + Some(RequiredInputOrdering::Soft(vec![expected])) } else { - Some(RequiredInputOrdering::Hard(expected)) + Some(RequiredInputOrdering::Hard(vec![expected])) } } else { None diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 4f262d71bdf2e..75c39748f749a 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -38,7 +38,7 @@ use datafusion::datasource::source::DataSourceExec; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; -use datafusion::physical_expr::{LexOrdering, LexRequirement, PhysicalExprRef}; +use datafusion::physical_expr::{LexOrdering, PhysicalExprRef}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -1079,8 +1079,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard) + .map(RequiredInputOrdering::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1109,8 +1108,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard) + .map(RequiredInputOrdering::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1146,8 +1144,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) - .map(RequiredInputOrdering::Hard) + .map(RequiredInputOrdering::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 5a274b95cd681..7871fd17e0199 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -1306,7 +1306,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::Hard(sort_order)), + Some(RequiredInputOrdering::Hard(vec![sort_order])), ))) } @@ -1331,7 +1331,7 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = RequiredInputOrdering::Hard(LexRequirement::new(vec![ + let sort_order = RequiredInputOrdering::Hard(vec![LexRequirement::new(vec![ PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { @@ -1339,7 +1339,7 @@ fn roundtrip_csv_sink() -> Result<()> { nulls_first: false, }), ), - ])); + ])]); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1399,7 +1399,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::Hard(sort_order)), + Some(RequiredInputOrdering::Hard(vec![sort_order])), ))) } From 1d90a71285dd4d2f14ba6277006f3936793c99e1 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 19 Mar 2025 14:28:59 +0300 Subject: [PATCH 013/167] implement RequiredInputOrdering as vectors return alternative on BoundedWindowAggExec --- .../src/joins/symmetric_hash_join.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 99 ++++++++++++------- .../src/windows/window_agg_exec.rs | 4 +- 4 files changed, 67 insertions(+), 40 deletions(-) diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index c3e283415798a..2e170f2a2c3ba 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -76,7 +76,7 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use ahash::RandomState; use futures::{ready, Stream, StreamExt}; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index ce9445a7d07b7..ff9d01b3a5ee9 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -288,7 +288,7 @@ impl ExecutionPlan for BoundedWindowAggExec { .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter(), true)] + vec![calc_requirements(partition_bys, order_keys.iter())] } fn required_input_distribution(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 517a1786d2d8c..1210de40df218 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -280,9 +280,8 @@ pub(crate) fn calc_requirements< >( partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, - is_soft_requirements: bool, ) -> Option { - let mut sort_reqs = LexRequirement::new( + let mut sort_reqs_with_partition = LexRequirement::new( partition_by_exprs .into_iter() .map(|partition_by| { @@ -290,21 +289,25 @@ pub(crate) fn calc_requirements< }) .collect::>(), ); + let mut sort_reqs = LexRequirement::new(vec![]); for element in orderby_sort_exprs.into_iter() { let PhysicalSortExpr { expr, options } = element.borrow(); + let sort_req = PhysicalSortRequirement::new( + Arc::clone(expr), + Some(*options), + ); + if !sort_reqs_with_partition.iter().any(|e| e.expr.eq(expr)) { + sort_reqs_with_partition.push(sort_req.clone()); + } if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { - sort_reqs.push(PhysicalSortRequirement::new( - Arc::clone(expr), - Some(*options), - )); + sort_reqs.push(sort_req); } } - if is_soft_requirements { - (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Soft(vec![sort_reqs])) - } else { - (!sort_reqs.is_empty()).then_some(RequiredInputOrdering::Hard(vec![sort_reqs])) - } + (!sort_reqs_with_partition.is_empty()).then_some(RequiredInputOrdering::Hard(vec![ + sort_reqs_with_partition, + sort_reqs, + ])) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -725,25 +728,54 @@ mod tests { ( vec!["a"], vec![("b", true, true)], - vec![("a", None), ("b", Some((true, true)))], + vec![ + vec![ + ("a", None), + ("b", Some((true, true))) + ], + vec![ + ("b", Some((true, true))) + ], + ], ), // PARTITION BY a, ORDER BY a ASC NULLS FIRST - (vec!["a"], vec![("a", true, true)], vec![("a", None)]), + (vec!["a"], vec![("a", true, true)], vec![ + vec![ + ("a", None) + ], + vec![ + ("a", Some((true, true)))] + ]), // PARTITION BY a, ORDER BY b ASC NULLS FIRST, c DESC NULLS LAST ( vec!["a"], vec![("b", true, true), ("c", false, false)], vec![ - ("a", None), - ("b", Some((true, true))), - ("c", Some((false, false))), + vec![ + ("a", None), + ("b", Some((true, true))), + ("c", Some((false, false))), + ], + vec![ + ("b", Some((true, true))), + ("c", Some((false, false))), + ] + ], ), // PARTITION BY a, c, ORDER BY b ASC NULLS FIRST, c DESC NULLS LAST ( vec!["a", "c"], vec![("b", true, true), ("c", false, false)], - vec![("a", None), ("c", None), ("b", Some((true, true)))], + vec![vec![ + ("a", None), ("c", None), + ("b", Some((true, true))) + ], + vec![ + ("b", Some((true, true))), + ("c", Some((false, false))) + ] + ], ), ]; for (pb_params, ob_params, expected_params) in test_data { @@ -762,35 +794,30 @@ mod tests { orderbys.push(PhysicalSortExpr { expr, options }); } - for is_soft in [true, false] { - let mut expected: Option = None; - for (col_name, reqs) in expected_params.clone() { + let mut expected: Option = None; + for expected_param in expected_params.clone() { + let mut lex_requirement = LexRequirement::new(vec![]); + for (col_name, reqs) in expected_param { let options = reqs.map(|(descending, nulls_first)| SortOptions { descending, nulls_first, }); let expr = col(col_name, &schema)?; let res = PhysicalSortRequirement::new(expr, options); - if let Some(expected) = &mut expected { - expected.push(res); - } else { - expected = Some(LexRequirement::new(vec![res])); - } + lex_requirement.push(res); } - let expected_result = if let Some(expected) = expected { - if is_soft { - Some(RequiredInputOrdering::Soft(vec![expected])) + if !lex_requirement.is_empty() { + if let Some(expect) = expected { + expected = Some(RequiredInputOrdering::Hard(vec![expect.lex_requirement().clone(), lex_requirement])) } else { - Some(RequiredInputOrdering::Hard(vec![expected])) + expected = Some(RequiredInputOrdering::Hard(vec![lex_requirement])) } - } else { - None - }; - assert_eq!( - calc_requirements(partitionbys.clone(), orderbys.clone(), is_soft), - expected_result - ); + } } + assert_eq!( + calc_requirements(partitionbys.clone(), orderbys.clone()), + expected + ); } Ok(()) } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 1171fb7ba5828..28355e6901875 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -216,13 +216,13 @@ impl ExecutionPlan for WindowAggExec { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { - vec![calc_requirements(partition_bys, order_keys.iter(), false)] + vec![calc_requirements(partition_bys, order_keys.iter())] } else { let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter(), false)] + vec![calc_requirements(partition_bys, order_keys.iter())] } } From 55ef9aa5e591e69927eca1acb745841233c5e218 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 19 Mar 2025 14:55:34 +0300 Subject: [PATCH 014/167] fix test cases --- datafusion/catalog/src/stream.rs | 2 +- .../enforce_distribution.rs | 150 ++---------------- .../src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 51 +++--- 4 files changed, 37 insertions(+), 168 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index 0ac8a16b083b9..ba00783603ff4 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -33,8 +33,8 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_expr::create_ordering; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 2debb4833b92e..5e678f00cd2d4 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1325,7 +1325,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { #[test] fn multi_smj_joins() -> Result<()> { - let test_config = TestConfig::default().with_prefer_existing_sort(); + let test_config = TestConfig::default(); let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -1652,7 +1652,7 @@ fn smj_join_key_ordering() -> Result<()> { let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default().with_prefer_existing_sort(); + let test_config = TestConfig::default(); // Test: run EnforceDistribution, then EnforceSort. // Only two RepartitionExecs added @@ -1736,20 +1736,7 @@ fn merge_does_not_need_sort() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, exec.clone(), &DISTRIB_DISTRIB_SORT)?; - - // Test: result IS DIFFERENT, if EnforceSorting is run first: - // - // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: PREFER_EXISTING_SORT) - // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with - // SortExec at the top. - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=4096", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, exec, &SORT_DISTRIB_DISTRIB)?; + test_config.run(expected, exec, &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -2059,14 +2046,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - - // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -2093,16 +2073,7 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - - // test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -2121,8 +2092,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { sort_key, ); - // TestConfig: Prefer existing sort. - let test_config = TestConfig::default().with_prefer_existing_sort(); + let test_config = TestConfig::default(); // during repartitioning ordering is preserved let expected = &[ @@ -2428,8 +2398,7 @@ fn parallelization_multiple_files() -> Result<()> { let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); - let test_config = TestConfig::default() - .with_prefer_repartition_file_scans(1); + let test_config = TestConfig::default().with_prefer_repartition_file_scans(1); // The groups must have only contiguous ranges of rows from the same file // if any group has rows from multiple files, the data is no longer sorted destroyed @@ -2874,20 +2843,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { plan_parquet.clone(), &DISTRIB_DISTRIB_SORT, )?; - let expected_parquet_first_sort_enforcement = &[ - // no SPM - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // has coalesce - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_parquet_first_sort_enforcement, - plan_parquet, - &SORT_DISTRIB_DISTRIB, - )?; + test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; // Test: with csv let expected_csv = &[ @@ -2897,20 +2853,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - let expected_csv_first_sort_enforcement = &[ - // no SPM - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // has coalesce - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - test_config.run( - expected_csv_first_sort_enforcement, - plan_csv.clone(), - &SORT_DISTRIB_DISTRIB, - )?; + test_config.run(expected_csv, plan_csv.clone(), &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -3085,7 +3028,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default().with_prefer_existing_sort(); + let test_config = TestConfig::default(); // Expected Outcome: // Original plan expects its output to be ordered by c@2 ASC. @@ -3114,7 +3057,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default().with_prefer_existing_sort(); + let test_config = TestConfig::default(); let expected = &[ "SortPreservingMergeExec: [d@3 ASC]", @@ -3128,45 +3071,6 @@ fn preserve_ordering_through_repartition() -> Result<()> { Ok(()) } -#[test] -fn do_not_preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let test_config = TestConfig::default(); - - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - - // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run( - expected_first_sort_enforcement, - physical_plan, - &SORT_DISTRIB_DISTRIB, - )?; - - Ok(()) -} - #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); @@ -3196,43 +3100,23 @@ fn no_need_for_sort_after_filter() -> Result<()> { fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key]); - - let sort_req = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]); - let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); let test_config = TestConfig::default(); // Test: run EnforceDistribution, then EnforceSort. let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - - // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_first_sort_enforcement, - physical_plan, - &SORT_DISTRIB_DISTRIB, - )?; + test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; Ok(()) } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 7cd90ec073699..a6de0090a28b4 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -76,8 +76,8 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_expr_common::physical_expr::fmt_sql; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use ahash::RandomState; use futures::{ready, Stream, StreamExt}; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1210de40df218..ab3d7d8ca24d0 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -292,10 +292,7 @@ pub(crate) fn calc_requirements< let mut sort_reqs = LexRequirement::new(vec![]); for element in orderby_sort_exprs.into_iter() { let PhysicalSortExpr { expr, options } = element.borrow(); - let sort_req = PhysicalSortRequirement::new( - Arc::clone(expr), - Some(*options), - ); + let sort_req = PhysicalSortRequirement::new(Arc::clone(expr), Some(*options)); if !sort_reqs_with_partition.iter().any(|e| e.expr.eq(expr)) { sort_reqs_with_partition.push(sort_req.clone()); } @@ -729,23 +726,16 @@ mod tests { vec!["a"], vec![("b", true, true)], vec![ - vec![ - ("a", None), - ("b", Some((true, true))) - ], - vec![ - ("b", Some((true, true))) - ], + vec![("a", None), ("b", Some((true, true)))], + vec![("b", Some((true, true)))], ], ), // PARTITION BY a, ORDER BY a ASC NULLS FIRST - (vec!["a"], vec![("a", true, true)], vec![ - vec![ - ("a", None) - ], - vec![ - ("a", Some((true, true)))] - ]), + ( + vec!["a"], + vec![("a", true, true)], + vec![vec![("a", None)], vec![("a", Some((true, true)))]], + ), // PARTITION BY a, ORDER BY b ASC NULLS FIRST, c DESC NULLS LAST ( vec!["a"], @@ -756,25 +746,16 @@ mod tests { ("b", Some((true, true))), ("c", Some((false, false))), ], - vec![ - ("b", Some((true, true))), - ("c", Some((false, false))), - ] - + vec![("b", Some((true, true))), ("c", Some((false, false)))], ], ), // PARTITION BY a, c, ORDER BY b ASC NULLS FIRST, c DESC NULLS LAST ( vec!["a", "c"], vec![("b", true, true), ("c", false, false)], - vec![vec![ - ("a", None), ("c", None), - ("b", Some((true, true))) - ], - vec![ - ("b", Some((true, true))), - ("c", Some((false, false))) - ] + vec![ + vec![("a", None), ("c", None), ("b", Some((true, true)))], + vec![("b", Some((true, true))), ("c", Some((false, false)))], ], ), ]; @@ -808,9 +789,13 @@ mod tests { } if !lex_requirement.is_empty() { if let Some(expect) = expected { - expected = Some(RequiredInputOrdering::Hard(vec![expect.lex_requirement().clone(), lex_requirement])) + expected = Some(RequiredInputOrdering::Hard(vec![ + expect.lex_requirement().clone(), + lex_requirement, + ])) } else { - expected = Some(RequiredInputOrdering::Hard(vec![lex_requirement])) + expected = + Some(RequiredInputOrdering::Hard(vec![lex_requirement])) } } } From ea2583b61fc9201d78428de3185bed0366712905 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 19 Mar 2025 15:40:00 +0300 Subject: [PATCH 015/167] change doc --- .../src/enforce_sorting/mod.rs | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index af4e1adec2f12..1a1a2d24f1832 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -311,20 +311,21 @@ fn replace_with_partial_sort( /// The [`CoalescePartitionsExec`] + [`SortExec`] cascades /// combine the partitions first, and then sort: /// ```text -/// ┌ ─ ─ ─ ─ ─ ┐ -/// ┌─┬─┬─┐ -/// ││B│A│D│... ├──┐ -/// └─┴─┴─┘ │ -/// └ ─ ─ ─ ─ ─ ┘ │ ┌────────────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ┐ -/// Partition 1 │ │ Coalesce │ ┌─┬─┬─┬─┬─┐ │ │ ┌─┬─┬─┬─┬─┐ -/// ├──▶(no ordering guarantees)│──▶││B│E│A│D│C│...───▶ Sort ├───▶││A│B│C│D│E│... │ -/// │ │ │ └─┴─┴─┴─┴─┘ │ │ └─┴─┴─┴─┴─┘ -/// ┌ ─ ─ ─ ─ ─ ┐ │ └────────────────────────┘ └ ─ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ─ ─ ┘ -/// ┌─┬─┐ │ Partition Partition -/// ││E│C│ ... ├──┘ -/// └─┴─┘ -/// └ ─ ─ ─ ─ ─ ┘ -/// Partition 2 +/// ┌ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ┐ +/// ┌─┬─┬─┐ │ │ ┌─┬─┬─┐ +/// ││B│A│D│... │──▶│ Sort │──▶││A│B│D│... │──┐ +/// └─┴─┴─┘ │ │ └─┴─┴─┘ │ +/// └ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ┘ │ ┌─────────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ┐ +/// Partition 1 Partition 1 │ │ │ ┌─┬─┬─┬─┬─┐ +/// ├──▶ SortPreservingMerge ├───▶││A│B│C│D│E│... │ +/// │ │ │ └─┴─┴─┴─┴─┘ +/// │ │ │ └─┴─┴─┴─┴─┘ +/// ┌ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ┐ │ └─────────────────────┘ └ ─ ─ ─ ─ ─ ─ ─ ┘ +/// ┌─┬─┐ │ │ ┌─┬─┐ │ Partition +/// ││E│C│ ... │──▶│ Sort ├──▶││C│E│ ... │──┘ +/// └─┴─┘ │ │ └─┴─┘ +/// └ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ┘ +/// Partition 2 Partition 2 /// ``` /// /// From c83862f902ebc3a1b6b82243d2363502396767a1 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 20 Mar 2025 11:06:30 +0300 Subject: [PATCH 016/167] revert prefer_existing_sort flag --- datafusion/catalog/src/stream.rs | 8 +- datafusion/common/src/config.rs | 8 + .../core/src/datasource/listing/table.rs | 17 +- .../enforce_distribution.rs | 164 ++++++++++++++-- .../physical_optimizer/enforce_sorting.rs | 2 + .../physical_optimizer/projection_pushdown.rs | 41 ++-- .../replace_with_order_preserving_variants.rs | 183 ++++++++++++++++-- datafusion/execution/src/config.rs | 17 ++ .../src/enforce_distribution.rs | 21 +- .../src/enforce_sorting/mod.rs | 30 +-- .../replace_with_order_preserving_variants.rs | 12 +- .../src/enforce_sorting/sort_pushdown.rs | 43 ++-- .../src/output_requirements.rs | 3 +- .../physical-plan/src/aggregates/mod.rs | 4 +- .../physical-plan/src/execution_plan.rs | 32 +-- datafusion/physical-plan/src/windows/mod.rs | 15 +- .../tests/cases/roundtrip_physical_plan.rs | 15 +- .../test_files/information_schema.slt | 2 + datafusion/sqllogictest/test_files/joins.slt | 42 ++-- datafusion/sqllogictest/test_files/window.slt | 3 + docs/source/user-guide/configs.md | 1 + 21 files changed, 496 insertions(+), 167 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index ba00783603ff4..75ab12225839f 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -356,10 +356,10 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, std::slice::from_ref(x))?; let ordering = orders.into_iter().next().unwrap(); - Some(RequiredInputOrdering::Hard(vec![ordering - .into_iter() - .map(Into::into) - .collect()])) + RequiredInputOrdering::new( + vec![ordering.into_iter().map(Into::into).collect()], + false, + ) } None => None, }; diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index cf0e1d7c50948..b0f17630c910c 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -647,6 +647,14 @@ config_namespace! { /// ``` pub repartition_sorts: bool, default = true + /// When true, DataFusion will opportunistically remove sorts when the data is already sorted, + /// (i.e. setting `preserve_order` to true on `RepartitionExec` and + /// using `SortPreservingMergeExec`) + /// + /// When false, DataFusion will maximize plan parallelism using + /// `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. + pub prefer_existing_sort: bool, default = false + /// When set to true, the logical plan optimizer will produce warning /// messages if any optimization rules produce errors and then proceed to the next /// rule. When set to false, any rules that produce errors will cause the query to fail diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e11f28e4c5127..1b85b4d040cac 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1059,13 +1059,16 @@ impl TableProvider for ListingTable { ); }; // Converts Vec> into type required by execution plan to specify its required input ordering - Some(RequiredInputOrdering::Hard(vec![LexRequirement::new( - ordering - .into_iter() - .cloned() - .map(PhysicalSortRequirement::from) - .collect::>(), - )])) + RequiredInputOrdering::new( + vec![LexRequirement::new( + ordering + .into_iter() + .cloned() + .map(PhysicalSortRequirement::from) + .collect::>(), + )], + false, + ) } else { None }; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5e678f00cd2d4..10c561ca3609d 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -343,6 +343,7 @@ fn sort_required_exec_with_req( fn ensure_distribution_helper( plan: Arc, target_partitions: usize, + prefer_existing_sort: bool, ) -> Result> { let distribution_context = DistributionContext::new_default(plan); let mut config = ConfigOptions::new(); @@ -350,6 +351,7 @@ fn ensure_distribution_helper( config.optimizer.enable_round_robin_repartition = true; config.optimizer.repartition_file_scans = false; config.optimizer.repartition_file_min_size = 1024; + config.optimizer.prefer_existing_sort = prefer_existing_sort; ensure_distribution(distribution_context, &config).map(|item| item.data.plan) } @@ -372,6 +374,9 @@ macro_rules! plans_matches_expected { fn test_suite_default_config_options() -> ConfigOptions { let mut config = ConfigOptions::new(); + // By default, will not repartition / resort data if it is already sorted. + config.optimizer.prefer_existing_sort = false; + // By default, will attempt to convert Union to Interleave. config.optimizer.prefer_existing_union = false; @@ -414,6 +419,12 @@ impl Default for TestConfig { } impl TestConfig { + /// If preferred, will not repartition / resort data if it is already sorted. + fn with_prefer_existing_sort(mut self) -> Self { + self.config.optimizer.prefer_existing_sort = true; + self + } + /// If preferred, will not attempt to convert Union to Interleave. fn with_prefer_existing_union(mut self) -> Self { self.config.optimizer.prefer_existing_union = true; @@ -1083,6 +1094,7 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let bottom_left_join = ensure_distribution_helper( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, + true, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1113,6 +1125,7 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let bottom_right_join = ensure_distribution_helper( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, + true, )?; // Join on (B == b1 and C == c and AA = a1) @@ -1215,6 +1228,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let bottom_left_join = ensure_distribution_helper( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, + true, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1245,6 +1259,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let bottom_right_join = ensure_distribution_helper( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, + true, )?; // Join on (B == b1 and C == c and AA = a1) @@ -1323,9 +1338,10 @@ fn reorder_join_keys_to_right_input() -> Result<()> { Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn multi_smj_joins() -> Result<()> { - let test_config = TestConfig::default(); + let test_config = TestConfig::default().with_prefer_existing_sort(); let left = parquet_exec(); let alias_pairs: Vec<(String, String)> = vec![ @@ -1605,6 +1621,7 @@ fn multi_smj_joins() -> Result<()> { Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn smj_join_key_ordering() -> Result<()> { // group by (a as a1, b as b1) @@ -1652,7 +1669,7 @@ fn smj_join_key_ordering() -> Result<()> { let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default(); + let test_config = TestConfig::default().with_prefer_existing_sort(); // Test: run EnforceDistribution, then EnforceSort. // Only two RepartitionExecs added @@ -1736,7 +1753,19 @@ fn merge_does_not_need_sort() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, exec.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, exec, &SORT_DISTRIB_DISTRIB)?; + // Test: result IS DIFFERENT, if EnforceSorting is run first: + // + // In this case preserving ordering through order preserving operators is not desirable + // (according to flag: PREFER_EXISTING_SORT) + // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with + // SortExec at the top. + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=4096", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; + test_config.run(expected_first_sort_enforcement, exec, &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -2046,7 +2075,14 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + + // Test: result IS DIFFERENT, if EnforceSorting is run first: + let expected_first_sort_enforcement = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + ]; + test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; Ok(()) } @@ -2073,11 +2109,21 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + + // test: result IS DIFFERENT, if EnforceSorting is run first: + let expected_first_sort_enforcement = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + ]; + test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; Ok(()) } +/// These test cases use [`TestConfig::with_prefer_existing_sort`]. #[test] fn repartition_does_not_destroy_sort() -> Result<()> { // SortRequired @@ -2092,7 +2138,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { sort_key, ); - let test_config = TestConfig::default(); + let test_config = TestConfig::default().with_prefer_existing_sort(); // during repartitioning ordering is preserved let expected = &[ @@ -2398,7 +2444,9 @@ fn parallelization_multiple_files() -> Result<()> { let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); - let test_config = TestConfig::default().with_prefer_repartition_file_scans(1); + let test_config = TestConfig::default() + .with_prefer_existing_sort() + .with_prefer_repartition_file_scans(1); // The groups must have only contiguous ranges of rows from the same file // if any group has rows from multiple files, the data is no longer sorted destroyed @@ -2843,7 +2891,20 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { plan_parquet.clone(), &DISTRIB_DISTRIB_SORT, )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let expected_parquet_first_sort_enforcement = &[ + // no SPM + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // has coalesce + " CoalescePartitionsExec", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + ]; + test_config.run( + expected_parquet_first_sort_enforcement, + plan_parquet, + &SORT_DISTRIB_DISTRIB, + )?; // Test: with csv let expected_csv = &[ @@ -2853,7 +2914,20 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv.clone(), &SORT_DISTRIB_DISTRIB)?; + let expected_csv_first_sort_enforcement = &[ + // no SPM + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // has coalesce + " CoalescePartitionsExec", + " UnionExec", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + ]; + test_config.run( + expected_csv_first_sort_enforcement, + plan_csv.clone(), + &SORT_DISTRIB_DISTRIB, + )?; Ok(()) } @@ -3017,6 +3091,7 @@ fn remove_redundant_roundrobins() -> Result<()> { Ok(()) } +/// This test case uses [`TestConfig::with_prefer_existing_sort`]. #[test] fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); @@ -3028,7 +3103,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default(); + let test_config = TestConfig::default().with_prefer_existing_sort(); // Expected Outcome: // Original plan expects its output to be ordered by c@2 ASC. @@ -3046,6 +3121,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { Ok(()) } +/// This test case uses [`TestConfig::with_prefer_existing_sort`]. #[test] fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); @@ -3057,7 +3133,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); // TestConfig: Prefer existing sort. - let test_config = TestConfig::default(); + let test_config = TestConfig::default().with_prefer_existing_sort(); let expected = &[ "SortPreservingMergeExec: [d@3 ASC]", @@ -3071,6 +3147,45 @@ fn preserve_ordering_through_repartition() -> Result<()> { Ok(()) } +#[test] +fn do_not_preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let test_config = TestConfig::default(); + + // Test: run EnforceDistribution, then EnforceSort. + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; + test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + // Test: result IS DIFFERENT, if EnforceSorting is run first: + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; + test_config.run( + expected_first_sort_enforcement, + physical_plan, + &SORT_DISTRIB_DISTRIB, + )?; + + Ok(()) +} + #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); @@ -3086,7 +3201,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { "CoalescePartitionsExec", // Since after this stage c is constant. c@2 ASC ordering is already satisfied. " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let test_config = TestConfig::default(); @@ -3097,7 +3212,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { } #[test] -fn do_not_preserve_ordering_through_repartition() -> Result<()> { +fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), @@ -3111,18 +3226,31 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { // Test: run EnforceDistribution, then EnforceSort. let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=a@0 ASC", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + // Test: result IS DIFFERENT, if EnforceSorting is run first: + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; + test_config.run( + expected_first_sort_enforcement, + physical_plan, + &SORT_DISTRIB_DISTRIB, + )?; Ok(()) } #[test] -fn do_not_preserve_ordering_through_repartition2() -> Result<()> { +fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index bf3f9da015fd0..80bb590a3bfdd 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -155,6 +155,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, + &config ) }) .data() @@ -1277,6 +1278,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, + &config ) }) .data() diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 963033070d8ca..6d073de77e036 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -649,8 +649,8 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv.clone(), - Some(RequiredInputOrdering::Hard(vec![LexRequirement::new( - vec![ + RequiredInputOrdering::new( + vec![LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), options: Some(SortOptions::default()), @@ -663,8 +663,9 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ], - )])), + ])], + false, + ), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -697,20 +698,24 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = RequiredInputOrdering::Hard(vec![LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ])]); + let expected_reqs = RequiredInputOrdering::new( + vec![LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 2)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_a", 1)), + )), + options: Some(SortOptions::default()), + }, + ])], + false, + ) + .unwrap(); assert_eq!( after_optimize .as_any() diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 4669bc276116e..12bb0c5c53369 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -41,6 +41,7 @@ use datafusion_physical_plan::{ use datafusion::datasource::source::DataSourceExec; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::Result; +use datafusion_common::config::ConfigOptions; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; @@ -51,6 +52,39 @@ use object_store::ObjectStore; use rstest::rstest; use url::Url; +/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts +/// the plan against the original and expected plans. +/// +/// # Parameters +/// +/// * `$EXPECTED_PLAN_LINES`: Expected input plan. +/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag +/// `prefer_existing_sort` is `false`. +/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when +/// the flag `prefer_existing_sort` is `true`. +/// * `$PLAN`: The plan to optimize. +macro_rules! assert_optimized_prefer_sort_on_off { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { + if $PREFER_EXISTING_SORT { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } else { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } + }; +} + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans for both bounded and /// unbounded cases. @@ -59,24 +93,32 @@ use url::Url; /// /// * `EXPECTED_UNBOUNDED_PLAN_LINES`: Expected input unbounded plan. /// * `EXPECTED_BOUNDED_PLAN_LINES`: Expected input bounded plan. -/// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan for unbounded -/// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan for bounded +/// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan, which is +/// the same regardless of the value of the `prefer_existing_sort` flag. +/// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag +/// `prefer_existing_sort` is `false` for bounded cases. +/// * `EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan +/// when the flag `prefer_existing_sort` is `true` for bounded cases. /// * `$PLAN`: The plan to optimize. /// * `$SOURCE_UNBOUNDED`: Whether the given plan contains an unbounded source. macro_rules! assert_optimized_in_all_boundedness_situations { - ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr) => { + ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr, $PREFER_EXISTING_SORT: expr) => { if $SOURCE_UNBOUNDED { - assert_optimized!( + assert_optimized_prefer_sort_on_off!( $EXPECTED_UNBOUNDED_PLAN_LINES, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, $PLAN, + $PREFER_EXISTING_SORT, $SOURCE_UNBOUNDED ); } else { - assert_optimized!( + assert_optimized_prefer_sort_on_off!( $EXPECTED_BOUNDED_PLAN_LINES, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, $PLAN, + $PREFER_EXISTING_SORT, $SOURCE_UNBOUNDED ); } @@ -91,9 +133,10 @@ macro_rules! assert_optimized_in_all_boundedness_situations { /// * `$EXPECTED_PLAN_LINES`: Expected input plan. /// * `$EXPECTED_OPTIMIZED_PLAN_LINES`: Expected optimized plan. /// * `$PLAN`: The plan to optimize. +/// * `$PREFER_EXISTING_SORT`: Value of the `prefer_existing_sort` flag. #[macro_export] macro_rules! assert_optimized { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr) => { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { let physical_plan = $PLAN; let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -109,8 +152,10 @@ macro_rules! assert_optimized { let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES.iter().map(|s| *s).collect(); // Run the rule top-down + let mut config = ConfigOptions::new(); + config.optimizer.prefer_existing_sort=$PREFER_EXISTING_SORT; let plan_with_pipeline_fixer = OrderPreservationContext::new_default(physical_plan); - let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false)).data().and_then(check_integrity)?; + let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, &config)).data().and_then(check_integrity)?; let optimized_physical_plan = parallel.plan; // Get string representation of the plan @@ -140,6 +185,7 @@ macro_rules! assert_optimized { // Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected async fn test_replace_multiple_input_repartition_1( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -185,13 +231,21 @@ async fn test_replace_multiple_input_repartition_1( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -200,6 +254,7 @@ async fn test_replace_multiple_input_repartition_1( #[tokio::test] async fn test_with_inter_children_change_only( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr_default("a", &schema)]; @@ -275,13 +330,25 @@ async fn test_with_inter_children_change_only( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortPreservingMergeExec: [a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -290,6 +357,7 @@ async fn test_with_inter_children_change_only( #[tokio::test] async fn test_replace_multiple_input_repartition_2( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -341,13 +409,22 @@ async fn test_replace_multiple_input_repartition_2( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -356,6 +433,7 @@ async fn test_replace_multiple_input_repartition_2( #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -412,13 +490,23 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -427,6 +515,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( #[tokio::test] async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -488,13 +577,24 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -503,6 +603,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[tokio::test] async fn test_not_replacing_when_no_need_to_preserve_sorting( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -556,14 +657,17 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -572,6 +676,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( #[tokio::test] async fn test_with_multiple_replacable_repartitions( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -633,13 +738,24 @@ async fn test_with_multiple_replacable_repartitions( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -648,6 +764,7 @@ async fn test_with_multiple_replacable_repartitions( #[tokio::test] async fn test_not_replace_with_different_orderings( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -700,14 +817,17 @@ async fn test_not_replace_with_different_orderings( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -716,6 +836,7 @@ async fn test_not_replace_with_different_orderings( #[tokio::test] async fn test_with_lost_ordering( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -762,13 +883,21 @@ async fn test_with_lost_ordering( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -777,6 +906,7 @@ async fn test_with_lost_ordering( #[tokio::test] async fn test_with_lost_and_kept_ordering( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; @@ -853,14 +983,27 @@ async fn test_with_lost_and_kept_ordering( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } @@ -869,6 +1012,7 @@ async fn test_with_lost_and_kept_ordering( #[tokio::test] async fn test_with_multiple_child_trees( #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; @@ -963,14 +1107,17 @@ async fn test_with_multiple_child_trees( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, expected_optimized_unbounded, expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, physical_plan, - source_unbounded + source_unbounded, + prefer_existing_sort ); Ok(()) } diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 28da568470af1..53646dc5b468e 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -243,6 +243,14 @@ impl SessionConfig { self.options.optimizer.repartition_sorts } + /// Prefer existing sort (true) or maximize parallelism (false). See + /// [prefer_existing_sort] for more details + /// + /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort + pub fn prefer_existing_sort(&self) -> bool { + self.options.optimizer.prefer_existing_sort + } + /// Are statistics collected during execution? pub fn collect_statistics(&self) -> bool { self.options.execution.collect_statistics @@ -313,6 +321,15 @@ impl SessionConfig { self } + /// Prefer existing sort (true) or maximize parallelism (false). See + /// [prefer_existing_sort] for more details + /// + /// [prefer_existing_sort]: datafusion_common::config::OptimizerOptions::prefer_existing_sort + pub fn with_prefer_existing_sort(mut self, enabled: bool) -> Self { + self.options.optimizer.prefer_existing_sort = enabled; + self + } + /// Prefer existing union (true). See [prefer_existing_union] for more details /// /// [prefer_existing_union]: datafusion_common::config::OptimizerOptions::prefer_existing_union diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index ef828592f8a70..ee2012354a978 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -47,7 +47,7 @@ use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::{EmissionType, RequiredInputOrdering}; +use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, }; @@ -853,7 +853,8 @@ fn add_roundrobin_on_top( // during repartition. This will be un-done in the future // If any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements - // - Required input ordering is a hard requirement + // - Usage of order preserving variants is not desirable + // (determined by flag `config.optimizer.prefer_existing_sort`) let partitioning = Partitioning::RoundRobinBatch(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? @@ -911,7 +912,8 @@ fn add_hash_on_top( // following conditions is true: // - Preserving ordering is not helpful in terms of satisfying ordering // requirements. - // - Required input ordering is a hard requirement + // - Usage of order preserving variants is not desirable + // (determined by flag `config.optimizer.prefer_existing_sort`) let partitioning = dist.create_partitioning(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? @@ -1176,6 +1178,12 @@ pub fn ensure_distribution( dist_context.plan.pipeline_behavior(), EmissionType::Incremental | EmissionType::Both ); + // Use order preserving variants either of the conditions true + // - it is desired according to config + // - when plan is unbounded + // - when it is pipeline friendly (can incrementally produce results) + let order_preserving_variants_desirable = + unbounded_and_pipeline_friendly || config.optimizer.prefer_existing_sort; // Remove unnecessary repartition from the physical plan if any let DistributionContext { @@ -1282,13 +1290,6 @@ pub fn ensure_distribution( required_input_ordering.lex_requirement(), ); - // Use order preserving variants either of the conditions true - // - it is desired according to requirement - // - when plan is unbounded - // - when it is pipeline friendly (can incrementally produce results) - // - when the requirement is not optional - let order_preserving_variants_desirable = unbounded_and_pipeline_friendly - || matches!(required_input_ordering, RequiredInputOrdering::Hard(_)); if (!ordering_satisfied || !order_preserving_variants_desirable) && child.data { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 1a1a2d24f1832..0134bb7f54d88 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -239,6 +239,7 @@ impl PhysicalOptimizerRule for EnforceSorting { plan_with_pipeline_fixer, false, true, + config, ) }) .data()?; @@ -311,21 +312,20 @@ fn replace_with_partial_sort( /// The [`CoalescePartitionsExec`] + [`SortExec`] cascades /// combine the partitions first, and then sort: /// ```text -/// ┌ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ┐ -/// ┌─┬─┬─┐ │ │ ┌─┬─┬─┐ -/// ││B│A│D│... │──▶│ Sort │──▶││A│B│D│... │──┐ -/// └─┴─┴─┘ │ │ └─┴─┴─┘ │ -/// └ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ┘ │ ┌─────────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ┐ -/// Partition 1 Partition 1 │ │ │ ┌─┬─┬─┬─┬─┐ -/// ├──▶ SortPreservingMerge ├───▶││A│B│C│D│E│... │ -/// │ │ │ └─┴─┴─┴─┴─┘ -/// │ │ │ └─┴─┴─┴─┴─┘ -/// ┌ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ┐ │ └─────────────────────┘ └ ─ ─ ─ ─ ─ ─ ─ ┘ -/// ┌─┬─┐ │ │ ┌─┬─┐ │ Partition -/// ││E│C│ ... │──▶│ Sort ├──▶││C│E│ ... │──┘ -/// └─┴─┘ │ │ └─┴─┘ -/// └ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ┘ -/// Partition 2 Partition 2 +/// ┌ ─ ─ ─ ─ ─ ┐ +/// ┌─┬─┬─┐ +/// ││B│A│D│... ├──┐ +/// └─┴─┴─┘ │ +/// └ ─ ─ ─ ─ ─ ┘ │ ┌────────────────────────┐ ┌ ─ ─ ─ ─ ─ ─ ┐ ┌────────┐ ┌ ─ ─ ─ ─ ─ ─ ─ ┐ +/// Partition 1 │ │ Coalesce │ ┌─┬─┬─┬─┬─┐ │ │ ┌─┬─┬─┬─┬─┐ +/// ├──▶(no ordering guarantees)│──▶││B│E│A│D│C│...───▶ Sort ├───▶││A│B│C│D│E│... │ +/// │ │ │ └─┴─┴─┴─┴─┘ │ │ └─┴─┴─┴─┴─┘ +/// ┌ ─ ─ ─ ─ ─ ┐ │ └────────────────────────┘ └ ─ ─ ─ ─ ─ ─ ┘ └────────┘ └ ─ ─ ─ ─ ─ ─ ─ ┘ +/// ┌─┬─┐ │ Partition Partition +/// ││E│C│ ... ├──┘ +/// └─┴─┘ +/// └ ─ ─ ─ ─ ─ ┘ +/// Partition 2 /// ``` /// /// diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index b770a1b39afea..30bbea16f8da1 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -29,12 +29,13 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::{EmissionType, RequiredInputOrdering}; +use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_common::config::ConfigOptions; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its @@ -209,6 +210,10 @@ fn plan_with_order_breaking_variants( /// If this replacement is helpful for removing a `SortExec`, it updates the plan. /// Otherwise, it leaves the plan unchanged. /// +/// NOTE: This optimizer sub-rule will only produce sort-preserving `RepartitionExec`s +/// if the query is bounded or if the config option `prefer_existing_sort` is +/// set to `true`. +/// /// The algorithm flow is simply like this: /// 1. Visit nodes of the physical plan bottom-up and look for `SortExec` nodes. /// During the traversal, keep track of operators that maintain ordering (or @@ -237,17 +242,16 @@ pub fn replace_with_order_preserving_variants( // `SortExec` from the plan. If this flag is `false`, this replacement // should only be made to fix the pipeline (streaming). is_spm_better: bool, + config: &ConfigOptions, ) -> Result> { update_order_preservation_ctx_children_data(&mut requirements); if !(is_sort(&requirements.plan) && requirements.children[0].data) { return Ok(Transformed::no(requirements)); } - let requirement = requirements.plan.required_input_ordering()[0].clone(); // For unbounded cases, we replace with the order-preserving variant in any // case, as doing so helps fix the pipeline. Also replace if config allows. - let use_order_preserving_variant = (requirement.is_some() - && matches!(requirement.unwrap(), RequiredInputOrdering::Hard(_))) + let use_order_preserving_variant = config.optimizer.prefer_existing_sort || (requirements.plan.boundedness().is_unbounded() && requirements.plan.pipeline_behavior() == EmissionType::Final); diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index e1db530b39b7b..d39856fa524ac 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -215,11 +215,10 @@ fn pushdown_requirement_to_children( match determine_children_requirement(parent_required, &request_child, child_plan) { RequirementsCompatibility::Satisfy => { - let req = (!request_child.is_empty()).then(|| { - RequiredInputOrdering::Hard(vec![LexRequirement::new( - request_child.to_vec(), - )]) - }); + let req = RequiredInputOrdering::new( + vec![LexRequirement::new(request_child.to_vec())], + false, + ); Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => { @@ -280,11 +279,10 @@ fn pushdown_requirement_to_children( .eq_properties .requirements_compatible(parent_required.lex_requirement(), &output_req) { - let req = (!parent_required.is_empty()).then(|| { - RequiredInputOrdering::Hard(vec![LexRequirement::new( - parent_required.to_vec(), - )]) - }); + let req = RequiredInputOrdering::new( + vec![LexRequirement::new(parent_required.to_vec())], + false, + ); Ok(Some(vec![req])) } else { Ok(None) @@ -355,11 +353,10 @@ fn pushdown_requirement_to_children( } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - let req = (!parent_required.is_empty()).then(|| { - RequiredInputOrdering::Hard(vec![LexRequirement::new( - parent_required.to_vec(), - )]) - }); + let req = RequiredInputOrdering::new( + vec![LexRequirement::new(parent_required.to_vec())], + false, + ); Ok(Some(vec![req])) } } else if let Some(hash_join) = plan.as_any().downcast_ref::() { @@ -416,11 +413,10 @@ fn determine_children_requirement( ) { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: - let adjusted = (!parent_required.lex_requirement().is_empty()).then(|| { - RequiredInputOrdering::Hard(vec![LexRequirement::new( - parent_required.to_vec(), - )]) - }); + let adjusted = RequiredInputOrdering::new( + vec![LexRequirement::new(parent_required.to_vec())], + false, + ); RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -669,10 +665,7 @@ fn handle_custom_pushdown( .iter() .map(|&maintains_order| { if maintains_order { - Some( - parent_required - .with_updated_requirements(updated_parent_req.clone()), - ) + parent_required.with_updated_requirements(updated_parent_req.clone()) } else { None } @@ -750,7 +743,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - Some(parent_required.with_updated_requirements(updated_parent_req)), + parent_required.with_updated_requirements(updated_parent_req), ])) } else { Ok(None) diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 4a3519ba71716..c9a11411f697f 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -247,8 +247,7 @@ impl ExecutionPlan for OutputRequirementExec { .map(|input| { OutputRequirementExec::new( input, - (!updated_sort_reqs.is_empty()) - .then_some(RequiredInputOrdering::Hard(vec![updated_sort_reqs])), + RequiredInputOrdering::new(vec![updated_sort_reqs], false), dist_req, ) }) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 62216bbea3912..7a161b34ddc06 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -521,8 +521,8 @@ impl AggregateExec { let group_expr_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; - let required_input_ordering = (!new_requirement.is_empty()) - .then_some(RequiredInputOrdering::Soft(vec![new_requirement])); + let required_input_ordering = + RequiredInputOrdering::new(vec![new_requirement], true); let cache = Self::compute_properties( &input, diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index ae1575eeb9694..abd26bbe88d0f 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -1096,6 +1096,18 @@ impl From for RequiredInputOrdering { } impl RequiredInputOrdering { + pub fn new(lex_requirements: Vec, is_soft: bool) -> Option { + if lex_requirements.is_empty() || lex_requirements[0].is_empty() { + return None; + } + + Some(if is_soft { + RequiredInputOrdering::Soft(lex_requirements) + } else { + RequiredInputOrdering::Hard(lex_requirements) + }) + } + pub fn lex_requirement(&self) -> &LexRequirement { match self { RequiredInputOrdering::Hard(lex) => &lex[0], @@ -1114,21 +1126,11 @@ impl RequiredInputOrdering { pub fn with_updated_requirements( &self, requirement: Vec, - ) -> Self { - match self { - RequiredInputOrdering::Hard(_) => { - RequiredInputOrdering::Hard(vec![LexRequirement::new(requirement)]) - } - RequiredInputOrdering::Soft(_) => { - RequiredInputOrdering::Soft(vec![LexRequirement::new(requirement)]) - } - } - } - - pub fn push(&self, requirement: PhysicalSortRequirement) -> Self { - let mut requirements = self.lex_requirement().clone(); - requirements.push(requirement); - self.with_updated_requirements(requirements.to_vec()) + ) -> Option { + RequiredInputOrdering::new( + vec![LexRequirement::new(requirement)], + matches!(self, RequiredInputOrdering::Soft(_)), + ) } } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ab3d7d8ca24d0..26309717e36cb 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -301,10 +301,7 @@ pub(crate) fn calc_requirements< } } - (!sort_reqs_with_partition.is_empty()).then_some(RequiredInputOrdering::Hard(vec![ - sort_reqs_with_partition, - sort_reqs, - ])) + RequiredInputOrdering::new(vec![sort_reqs_with_partition, sort_reqs], false) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -789,13 +786,13 @@ mod tests { } if !lex_requirement.is_empty() { if let Some(expect) = expected { - expected = Some(RequiredInputOrdering::Hard(vec![ - expect.lex_requirement().clone(), - lex_requirement, - ])) + expected = RequiredInputOrdering::new( + vec![expect.lex_requirement().clone(), lex_requirement], + false, + ) } else { expected = - Some(RequiredInputOrdering::Hard(vec![lex_requirement])) + RequiredInputOrdering::new(vec![lex_requirement], false) } } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 2dfcb812c6fb4..1f235186892d4 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -1311,7 +1311,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::Hard(vec![sort_order])), + RequiredInputOrdering::new(vec![sort_order], false), ))) } @@ -1337,20 +1337,21 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = RequiredInputOrdering::Hard(vec![LexRequirement::new(vec![ - PhysicalSortRequirement::new( + let sort_order = RequiredInputOrdering::new( + vec![LexRequirement::new(vec![PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - ), - ])]); + )])], + false, + ); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; let roundtrip_plan = roundtrip_test_and_return( - Arc::new(DataSinkExec::new(input, data_sink, Some(sort_order))), + Arc::new(DataSinkExec::new(input, data_sink, sort_order)), &ctx, &codec, ) @@ -1406,7 +1407,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::Hard(vec![sort_order])), + RequiredInputOrdering::new(vec![sort_order], false), ))) } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ae23e1fb06451..496f24abf6ed7 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -248,6 +248,7 @@ datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 datafusion.optimizer.max_passes 3 +datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true datafusion.optimizer.repartition_aggregations true @@ -346,6 +347,7 @@ datafusion.optimizer.filter_null_join_keys false When set to true, the optimizer datafusion.optimizer.hash_join_single_partition_threshold 1048576 The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition datafusion.optimizer.max_passes 3 Number of times that the optimizer will attempt to optimize the plan +datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 5a4ec8b5b75b6..50af06dc40fce 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3220,6 +3220,9 @@ WITH ORDER (a ASC NULLS FIRST, b ASC, c ASC) LOCATION '../core/tests/data/window_2.csv' OPTIONS ('format.has_header' 'true'); +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + # sort merge join should propagate ordering equivalence of the left side # for inner join. Hence final requirement rn1 ASC is already satisfied at # the end of SortMergeJoinExec. @@ -3288,6 +3291,9 @@ physical_plan 11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 12)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +statement ok +set datafusion.optimizer.prefer_existing_sort = false; + # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering # below query shouldn't add any SortExec for order by clause. @@ -3314,19 +3320,22 @@ logical_plan 10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST] -02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true -09)----CoalesceBatchesExec: target_batch_size=2 -10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -13)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -14)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +02)--SortExec: expr=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST], preserve_partitioning=[true] +03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +09)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true +11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +12)--------CoalesceBatchesExec: target_batch_size=2 +13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +16)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +17)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3469,6 +3478,10 @@ physical_plan statement ok set datafusion.execution.target_partitions = 2; +# use bounded variants +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 FROM annotated_data as l @@ -3554,6 +3567,9 @@ set datafusion.optimizer.prefer_hash_join = true; statement ok set datafusion.execution.target_partitions = 2; +statement ok +set datafusion.optimizer.prefer_existing_sort = false; + statement ok drop table annotated_data; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c7136e03931b4..fd623b67fe9f3 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5474,6 +5474,9 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' WITH ORDER (c1) OPTIONS ('format.has_header' 'true'); +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + query TT EXPLAIN SELECT c1, SUM(c9) OVER(PARTITION BY c1) as sum_c9 FROM aggregate_test_100_ordered ORDER BY c1, sum_c9; ---- diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 323d46eb5e0fe..68e21183938b1 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -107,6 +107,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | | datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | From 87dd95e617d982afb82967747688c829a18c5f81 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 20 Mar 2025 11:27:33 +0300 Subject: [PATCH 017/167] fix changes --- .../enforce_distribution.rs | 21 ++++++++++++++----- .../physical_optimizer/enforce_sorting.rs | 4 ++-- .../src/enforce_distribution.rs | 4 ++-- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 10c561ca3609d..cb44309318dde 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1753,6 +1753,7 @@ fn merge_does_not_need_sort() -> Result<()> { ]; let test_config = TestConfig::default(); test_config.run(expected, exec.clone(), &DISTRIB_DISTRIB_SORT)?; + // Test: result IS DIFFERENT, if EnforceSorting is run first: // // In this case preserving ordering through order preserving operators is not desirable @@ -2138,6 +2139,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { sort_key, ); + // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); // during repartitioning ordering is preserved @@ -3215,11 +3217,16 @@ fn no_need_for_sort_after_filter() -> Result<()> { fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key]); + + let sort_req = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); let test_config = TestConfig::default(); @@ -3232,13 +3239,15 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + // Test: result IS DIFFERENT, if EnforceSorting is run first: let expected_first_sort_enforcement = &[ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; test_config.run( expected_first_sort_enforcement, @@ -3301,6 +3310,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, dist_plan); @@ -3337,6 +3347,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; assert_plan_txt!(expected, dist_plan); diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 80bb590a3bfdd..31cd528eaf391 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -155,7 +155,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - &config + &config, ) }) .data() @@ -1278,7 +1278,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - &config + &config, ) }) .data() diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index ee2012354a978..4855d01a59552 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -912,8 +912,8 @@ fn add_hash_on_top( // following conditions is true: // - Preserving ordering is not helpful in terms of satisfying ordering // requirements. - // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.prefer_existing_sort`) + // - Usage of order preserving variants is not desirable (per the flag + // `config.optimizer.prefer_existing_sort`). let partitioning = dist.create_partitioning(n_target); let repartition = RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? From 24dba1668cec11233140dae749a0ae1fcf666806 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 20 Mar 2025 11:34:34 +0300 Subject: [PATCH 018/167] fix test case --- .../core/tests/physical_optimizer/enforce_distribution.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index cb44309318dde..db3f8a81c02a7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3236,7 +3236,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; From 8635d5e0819a7662cf61755894d5c074e08c9c83 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 20 Mar 2025 13:34:48 +0300 Subject: [PATCH 019/167] make LexRequirement private --- .../replace_with_order_preserving_variants.rs | 116 +++++++++--------- .../physical-expr-common/src/sort_expr.rs | 10 +- .../src/equivalence/properties/mod.rs | 15 ++- .../src/enforce_sorting/mod.rs | 6 +- .../src/output_requirements.rs | 2 +- .../src/update_aggr_exprs.rs | 21 ++-- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 18 +-- 8 files changed, 102 insertions(+), 88 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 12bb0c5c53369..ebabad791a049 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -232,11 +232,11 @@ async fn test_replace_multiple_input_repartition_1( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -331,15 +331,15 @@ async fn test_with_inter_children_change_only( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortPreservingMergeExec: [a@0 ASC]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", - ]; + "SortPreservingMergeExec: [a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortPreservingMergeExec: [a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -410,12 +410,12 @@ async fn test_replace_multiple_input_repartition_2( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -491,13 +491,13 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -578,14 +578,14 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -739,14 +739,14 @@ async fn test_with_multiple_replacable_repartitions( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -884,11 +884,11 @@ async fn test_with_lost_ordering( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, @@ -984,16 +984,16 @@ async fn test_with_lost_and_kept_ordering( " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [c@1 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 3a54b5b403995..ca7769a6b2514 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -576,7 +576,7 @@ pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; /// represents a lexicographical ordering requirement. #[derive(Debug, Default, Clone, PartialEq)] pub struct LexRequirement { - pub inner: Vec, + inner: Vec, } impl LexRequirement { @@ -588,6 +588,10 @@ impl LexRequirement { self.inner.is_empty() } + pub fn to_vec(&self) -> Vec { + self.inner.clone() + } + pub fn iter(&self) -> impl Iterator { self.inner.iter() } @@ -596,6 +600,10 @@ impl LexRequirement { self.inner.push(physical_sort_requirement) } + pub fn extend(&mut self, requirements: Vec) { + self.inner.extend(requirements) + } + /// Create a new [`LexRequirement`] from a [`LexOrdering`] /// /// Returns [`LexRequirement`] that requires the exact diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index c7c33ba5b2ba5..dd266c6535e54 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -738,11 +738,10 @@ impl EquivalenceProperties { req1: &LexRequirement, req2: &LexRequirement, ) -> Option { - let mut lhs = self.normalize_sort_requirements(req1); - let mut rhs = self.normalize_sort_requirements(req2); - lhs.inner - .iter_mut() - .zip(rhs.inner.iter_mut()) + let mut lhs = self.normalize_sort_requirements(req1).to_vec(); + let mut rhs = self.normalize_sort_requirements(req2).to_vec(); + lhs.iter_mut() + .zip(rhs.iter_mut()) .all(|(lhs, rhs)| { lhs.expr.eq(&rhs.expr) && match (lhs.options, rhs.options) { @@ -758,7 +757,11 @@ impl EquivalenceProperties { (None, None) => true, } }) - .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) + .then_some(if lhs.len() >= rhs.len() { + LexRequirement::new(lhs) + } else { + LexRequirement::new(rhs) + }) } /// we substitute the ordering according to input expression type, this is a simplified version diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 0134bb7f54d88..54a455047dbe7 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -287,9 +287,9 @@ fn replace_with_partial_sort( let sort_req = LexRequirement::from(sort_plan.expr().clone()); let mut common_prefix_length = 0; - while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { - inner: sort_req[0..common_prefix_length + 1].to_vec(), - }) { + while child_eq_properties.ordering_satisfy_requirement(&LexRequirement::new( + sort_req[0..common_prefix_length + 1].to_vec(), + )) { common_prefix_length += 1; } if common_prefix_length > 0 { diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index c9a11411f697f..d3f42778fbbfe 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -216,7 +216,7 @@ impl ExecutionPlan for OutputRequirementExec { let mut updated_sort_reqs = LexRequirement::new(vec![]); // None or empty_vec can be treated in the same way. if let Some(reqs) = &self.required_input_ordering()[0] { - for req in reqs.lex_requirement().inner.clone() { + for req in reqs.lex_requirement().to_vec() { let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { return Ok(None); diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 6228ed10ec341..6ed0d6dd5e028 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -170,20 +170,21 @@ fn try_convert_aggregate_if_better( // Otherwise, leave it as is. if aggr_expr.order_sensitivity().is_beneficial() && !aggr_sort_reqs.is_empty() { - let reqs = LexRequirement { - inner: concat_slices(prefix_requirement, &aggr_sort_reqs), - }; - - let prefix_requirement = LexRequirement { - inner: prefix_requirement.to_vec(), - }; + let reqs = LexRequirement::new(concat_slices( + prefix_requirement, + &aggr_sort_reqs, + )); + let prefix_requirement = LexRequirement::new(prefix_requirement.to_vec()); if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) - } else if eq_properties.ordering_satisfy_requirement(&LexRequirement { - inner: concat_slices(&prefix_requirement, &reverse_aggr_req), - }) { + } else if eq_properties.ordering_satisfy_requirement( + &LexRequirement::new(concat_slices( + &prefix_requirement, + &reverse_aggr_req, + )), + ) { // Converting to reverse enables more efficient execution // given the existing ordering (if possible): aggr_expr diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7a161b34ddc06..aa6ec4e2b7070 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -493,7 +493,7 @@ impl AggregateExec { input_eq_properties, &mode, )?; - new_requirement.inner.extend(req); + new_requirement.extend(req.to_vec()); new_requirement = new_requirement.collapse(); // If our aggregation has grouping sets then our base grouping exprs will diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 26309717e36cb..ef8f90912c741 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -588,18 +588,20 @@ pub fn get_window_mode( input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); - let mut partition_by_reqs: LexRequirement = LexRequirement::new(vec![]); let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); vec![].extend(indices.iter().map(|&idx| PhysicalSortRequirement { expr: Arc::clone(&partitionby_exprs[idx]), options: None, })); - partition_by_reqs - .inner - .extend(indices.iter().map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&partitionby_exprs[idx]), - options: None, - })); + let partition_by_reqs = LexRequirement::new( + indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: Arc::clone(&partitionby_exprs[idx]), + options: None, + }) + .collect(), + ); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); @@ -609,7 +611,7 @@ pub fn get_window_mode( [(false, order_by_reqs), (true, reverse_order_by_reqs)] { let req = LexRequirement::new( - [partition_by_reqs.inner.clone(), order_by_reqs.inner].concat(), + [partition_by_reqs.to_vec(), order_by_reqs.to_vec()].concat(), ) .collapse(); if partition_by_eqs.ordering_satisfy_requirement(&req) { From ab28421cada25935ee4ba7c391e1ba2e8a727ff5 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 20 Mar 2025 17:33:29 +0300 Subject: [PATCH 020/167] ensure RequiredInputOrdering inner requirement can not be empty simplify sort_pushdown.rs --- .../enforce_distribution.rs | 6 +- .../tests/physical_optimizer/test_utils.rs | 2 +- .../src/equivalence/properties/mod.rs | 57 ++++++ .../src/enforce_distribution.rs | 2 +- .../src/enforce_sorting/mod.rs | 13 +- .../src/enforce_sorting/sort_pushdown.rs | 173 ++++++++++-------- .../src/output_requirements.rs | 4 +- datafusion/physical-optimizer/src/utils.rs | 7 +- .../physical-plan/src/execution_plan.rs | 20 +- .../src/joins/sort_merge_join.rs | 4 +- .../src/joins/symmetric_hash_join.rs | 4 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 18 +- 13 files changed, 187 insertions(+), 125 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index db3f8a81c02a7..54d8d98465994 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -138,11 +138,7 @@ impl ExecutionPlan for SortRequiredExec { // model that it requires the output ordering of its input fn required_input_ordering(&self) -> Vec> { - if self.expr.is_empty() { - vec![None] - } else { - vec![Some(RequiredInputOrdering::from(self.expr.clone()))] - } + vec![RequiredInputOrdering::from(self.expr.clone())] } fn with_new_children( diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 0fddd06061f29..5af836029177b 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -375,7 +375,7 @@ impl ExecutionPlan for RequirementsTestExec { fn required_input_ordering(&self) -> Vec> { let requirement = RequiredInputOrdering::from(self.required_input_ordering.clone()); - vec![Some(requirement)] + vec![requirement] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index dd266c6535e54..21ab84efc3090 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1643,4 +1643,61 @@ mod tests { assert!(is_constant_recurse(&constants, &expr)); Ok(()) } + #[test] + fn test_requirements_compatible() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), + ])); + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_d = col("d", &schema)?; + + let eq_properties = EquivalenceProperties::new(schema); + let default_lex = LexRequirement::default(); + let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { + expr: col_a.clone(), + options: None, + }]); + let lex_a_b = LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: col_a, + options: None, + }, + PhysicalSortRequirement { + expr: col_b, + options: None, + }, + ]); + let lex_d = LexRequirement::new(vec![PhysicalSortRequirement { + expr: col_d, + options: None, + }]); + + let res = eq_properties.requirements_compatible(&default_lex, &default_lex); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_a, &default_lex); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&default_lex, &lex_a); + assert_eq!(res, false); + + let res = eq_properties.requirements_compatible(&lex_a, &lex_a); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_a, &lex_a_b); + assert_eq!(res, false); + + let res = eq_properties.requirements_compatible(&lex_a_b, &lex_a); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_d, &lex_a); + assert_eq!(res, false); + + Ok(()) + } } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 4855d01a59552..465ed54556cc2 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1300,7 +1300,7 @@ pub fn ensure_distribution( // Make sure to satisfy ordering requirement: child = add_sort_above_with_check( child, - required_input_ordering.clone(), + required_input_ordering.lex_requirement().clone(), None, ); } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 54a455047dbe7..69aefabb40c59 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -59,7 +59,6 @@ use datafusion_common::Result; use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; @@ -407,7 +406,7 @@ pub fn parallelize_sorts( { // Take the initial sort expressions and requirements let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; - let sort_reqs = RequiredInputOrdering::from(sort_exprs.clone()); + let sort_reqs = LexRequirement::from(sort_exprs.clone()); let sort_exprs = sort_exprs.clone(); // If there is a connection between a `CoalescePartitionsExec` and a @@ -629,15 +628,13 @@ fn adjust_window_sort_removal( } else { // We were unable to change the window to accommodate the input, so we // will insert a sort. - let reqs = window_tree - .plan - .required_input_ordering() - .swap_remove(0) - .unwrap_or_default(); + let reqs = window_tree.plan.required_input_ordering().swap_remove(0); // Satisfy the ordering requirement so that the window can run: let mut child_node = window_tree.children.swap_remove(0); - child_node = add_sort_above(child_node, reqs.lex_requirement().clone(), None); + if let Some(reqs) = reqs { + child_node = add_sort_above(child_node, reqs.lex_requirement().clone(), None); + } let child_plan = Arc::clone(&child_node.plan); window_tree.children.push(child_node); diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index d39856fa524ac..dd451f1a7d709 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -89,33 +89,34 @@ fn min_fetch(f1: Option, f2: Option) -> Option { fn pushdown_sorts_helper( mut sort_push_down: SortPushDown, ) -> Result> { - let plan = &sort_push_down.plan; - let parent_reqs = sort_push_down - .data - .ordering_requirement - .clone() - .unwrap_or_default(); - let satisfy_parent = plan - .equivalence_properties() - .ordering_satisfy_requirement(parent_reqs.lex_requirement()); + let plan = &Arc::clone(&sort_push_down.plan); + let parent_reqs = sort_push_down.data.ordering_requirement.clone(); + let satisfy_parent = parent_reqs.clone().is_none_or(|reqs| { + plan.equivalence_properties() + .ordering_satisfy_requirement(reqs.lex_requirement()) + }); if is_sort(plan) { let current_sort_fetch = plan.fetch(); let parent_req_fetch = sort_push_down.data.fetch; - let current_plan_reqs = plan - .output_ordering() - .cloned() - .map(RequiredInputOrdering::from) - .unwrap_or_default(); - let parent_is_stricter = plan.equivalence_properties().requirements_compatible( - parent_reqs.lex_requirement(), - current_plan_reqs.lex_requirement(), - ); - let current_is_stricter = plan.equivalence_properties().requirements_compatible( - current_plan_reqs.lex_requirement(), - parent_reqs.lex_requirement(), - ); + let current_plan_ordering = plan.output_ordering().cloned().unwrap_or_default(); + // Since this is a SortExec it needs to have an output ordering + let current_plan_ordering_as_req = + RequiredInputOrdering::from(current_plan_ordering).unwrap(); + + let parent_is_stricter = parent_reqs.clone().is_some_and(|parent_req| { + plan.equivalence_properties().requirements_compatible( + parent_req.lex_requirement(), + current_plan_ordering_as_req.lex_requirement(), + ) + }); + let current_is_stricter = parent_reqs.clone().is_none_or(|parent_req| { + plan.equivalence_properties().requirements_compatible( + current_plan_ordering_as_req.lex_requirement(), + parent_req.lex_requirement(), + ) + }); if !satisfy_parent && !parent_is_stricter { // This new sort has different requirements than the ordering being pushed down. @@ -123,22 +124,25 @@ fn pushdown_sorts_helper( // 2. continue sort pushdown, but with the new ordering of the new sort. // remove current sort (which will be the new ordering to pushdown) - let new_reqs = current_plan_reqs; + let new_reqs = current_plan_ordering_as_req; sort_push_down = sort_push_down.children.swap_remove(0); sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan - // add back sort exec matching parent - sort_push_down = add_sort_above( - sort_push_down, - parent_reqs.lex_requirement().clone(), - parent_req_fetch, - ); + if let Some(parent_reqs) = parent_reqs.clone() { + // add back sort exec matching parent + sort_push_down = add_sort_above( + sort_push_down, + parent_reqs.lex_requirement().clone(), + parent_req_fetch, + ); + } // make pushdown requirements be the new ones. sort_push_down.children[0].data = ParentRequirements { ordering_requirement: Some(new_reqs), fetch: current_sort_fetch, }; + return Ok(Transformed::yes(sort_push_down)); } else { // Don't add a SortExec // Do update what sort requirements to keep pushing down @@ -152,32 +156,37 @@ fn pushdown_sorts_helper( // set the stricter ordering if current_is_stricter { - sort_push_down.data.ordering_requirement = Some(current_plan_reqs); + sort_push_down.data.ordering_requirement = + Some(current_plan_ordering_as_req); } else { - sort_push_down.data.ordering_requirement = Some(parent_reqs); + sort_push_down.data.ordering_requirement = parent_reqs; } // recursive call to helper, so it doesn't transform_down and miss the new node (previous child of sort) return pushdown_sorts_helper(sort_push_down); } - } else if parent_reqs.is_empty() { + } + + let Some(parent_requirement) = parent_reqs else { // note: this `satisfy_parent`, but we don't want to push down anything. // Nothing to do. return Ok(Transformed::no(sort_push_down)); - } else if satisfy_parent { + }; + + let parent_fetch = sort_push_down.data.fetch; + if satisfy_parent { // For non-sort operators which satisfy ordering: let reqs = plan.required_input_ordering(); - let parent_req_fetch = sort_push_down.data.fetch; for (child, order) in sort_push_down.children.iter_mut().zip(reqs) { child.data.ordering_requirement = order; - child.data.fetch = min_fetch(parent_req_fetch, child.data.fetch); + child.data.fetch = min_fetch(parent_fetch, child.data.fetch); } - } else if let Some(adjusted) = pushdown_requirement_to_children(plan, &parent_reqs)? { + } else if let Some(adjusted) = + pushdown_requirement_to_children(plan, &parent_requirement)? + { // For operators that can take a sort pushdown. - // Continue pushdown, with updated requirements: - let parent_fetch = sort_push_down.data.fetch; let current_fetch = plan.fetch(); for (child, order) in sort_push_down.children.iter_mut().zip(adjusted) { child.data.ordering_requirement = order; @@ -186,14 +195,14 @@ fn pushdown_sorts_helper( sort_push_down.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - let sort_reqs = sort_push_down - .data - .ordering_requirement - .clone() - .unwrap_or_default(); - let fetch = sort_push_down.data.fetch; - sort_push_down = - add_sort_above(sort_push_down, sort_reqs.lex_requirement().clone(), fetch); + let sort_reqs = sort_push_down.data.ordering_requirement.clone(); + if let Some(sort_reqs) = sort_reqs { + sort_push_down = add_sort_above( + sort_push_down, + sort_reqs.lex_requirement().clone(), + parent_fetch, + ); + } assign_initial_requirements(&mut sort_push_down); } @@ -209,14 +218,15 @@ fn pushdown_requirement_to_children( let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].clone().unwrap_or_default(); + let maybe_child_requirement = required_input_ordering[0].clone(); let child_plan = plan.children().swap_remove(0); - - match determine_children_requirement(parent_required, &request_child, child_plan) - { + let Some(child_req) = maybe_child_requirement else { + return Ok(None); + }; + match determine_children_requirement(parent_required, &child_req, child_plan) { RequirementsCompatibility::Satisfy => { let req = RequiredInputOrdering::new( - vec![LexRequirement::new(request_child.to_vec())], + vec![LexRequirement::new(child_req.to_vec())], false, ); Ok(Some(vec![req])) @@ -245,19 +255,17 @@ fn pushdown_requirement_to_children( .cloned() .unwrap_or(LexOrdering::default()), ); - if sort_exec - .properties() - .eq_properties - .requirements_compatible( - parent_required.lex_requirement(), - sort_req.lex_requirement(), - ) - { - debug_assert!(!parent_required.is_empty()); - Ok(Some(vec![Some(parent_required.clone())])) - } else { - Ok(None) - } + Ok(sort_req + .filter(|req| { + sort_exec + .properties() + .eq_properties + .requirements_compatible( + parent_required.lex_requirement(), + req.lex_requirement(), + ) + }) + .map(|_| vec![Some(parent_required.clone())])) } else if plan.fetch().is_some() && plan.supports_limit_pushdown() && plan @@ -398,18 +406,18 @@ fn pushdown_would_violate_requirements( /// - If they are not compatible, need to add a sort. fn determine_children_requirement( parent_required: &RequiredInputOrdering, - request_child: &RequiredInputOrdering, + child_requirement: &RequiredInputOrdering, child_plan: &Arc, ) -> RequirementsCompatibility { if child_plan.equivalence_properties().requirements_compatible( - request_child.lex_requirement(), + child_requirement.lex_requirement(), parent_required.lex_requirement(), ) { // Child requirements are more specific, no need to push down. RequirementsCompatibility::Satisfy } else if child_plan.equivalence_properties().requirements_compatible( parent_required.lex_requirement(), - request_child.lex_requirement(), + child_requirement.lex_requirement(), ) { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: @@ -441,26 +449,31 @@ fn try_pushdown_requirements_to_join( JoinSide::Left => { let left_eq_properties = left_eq_properties.clone().with_reorder(sort_expr.clone()); - if left_eq_properties.ordering_satisfy_requirement( - left_requirement.unwrap_or_default().lex_requirement(), - ) { - // After re-ordering requirement is still satisfied - (sort_expr, right_ordering) - } else { + let Some(left_requirement) = left_requirement else { + return Ok(None); + }; + if !left_eq_properties + .ordering_satisfy_requirement(left_requirement.lex_requirement()) + { return Ok(None); } + // After re-ordering requirement is still satisfied + (sort_expr, right_ordering) } JoinSide::Right => { let right_eq_properties = right_eq_properties.clone().with_reorder(sort_expr.clone()); - if right_eq_properties.ordering_satisfy_requirement( - right_requirement.unwrap_or_default().lex_requirement(), - ) { - // After re-ordering requirement is still satisfied - (left_ordering, sort_expr) - } else { + + let Some(right_requirement) = right_requirement else { + return Ok(None); + }; + if !right_eq_properties + .ordering_satisfy_requirement(right_requirement.lex_requirement()) + { return Ok(None); } + // After re-ordering requirement is still satisfied + (left_ordering, sort_expr) } JoinSide::None => return Ok(None), }; @@ -481,7 +494,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(RequiredInputOrdering::from(sort_expr.clone())); + let new_req = RequiredInputOrdering::from(sort_expr.clone()); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index d3f42778fbbfe..ec531fb35102b 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -320,7 +320,7 @@ fn require_top_ordering_helper( let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = RequiredInputOrdering::from(req_ordering.clone()); Ok(( - Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, + Arc::new(OutputRequirementExec::new(plan, reqs, req_dist)) as _, true, )) } else if let Some(spm) = plan.as_any().downcast_ref::() { @@ -328,7 +328,7 @@ fn require_top_ordering_helper( Ok(( Arc::new(OutputRequirementExec::new( plan, - Some(reqs), + reqs, Distribution::SinglePartition, )) as _, true, diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 4d31346c1dfd1..57a193315a5c3 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; @@ -60,15 +59,15 @@ pub fn add_sort_above( /// requirement is already satisfied no `SortExec` is added. pub fn add_sort_above_with_check( node: PlanContext, - sort_requirements: RequiredInputOrdering, + sort_requirements: LexRequirement, fetch: Option, ) -> PlanContext { if !node .plan .equivalence_properties() - .ordering_satisfy_requirement(sort_requirements.lex_requirement()) + .ordering_satisfy_requirement(&sort_requirements) { - add_sort_above(node, sort_requirements.lex_requirement().clone(), fetch) + add_sort_above(node, sort_requirements, fetch) } else { node } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index abd26bbe88d0f..b5ebc805daae5 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -1083,19 +1083,10 @@ pub enum RequiredInputOrdering { Soft(Vec), } -impl Default for RequiredInputOrdering { - fn default() -> Self { - RequiredInputOrdering::Hard(vec![LexRequirement::default()]) - } -} - -impl From for RequiredInputOrdering { - fn from(ordering: LexOrdering) -> Self { - RequiredInputOrdering::Hard(vec![LexRequirement::from(ordering)]) - } -} - impl RequiredInputOrdering { + /// Creates a new RequiredInputOrdering instance, + /// empty requirements are not allowed inside this type, + /// if given [`None`] will be returned pub fn new(lex_requirements: Vec, is_soft: bool) -> Option { if lex_requirements.is_empty() || lex_requirements[0].is_empty() { return None; @@ -1108,7 +1099,12 @@ impl RequiredInputOrdering { }) } + pub fn from(ordering: LexOrdering) -> Option { + RequiredInputOrdering::new(vec![LexRequirement::from(ordering)], false) + } + pub fn lex_requirement(&self) -> &LexRequirement { + // TODO This function is returning the first alternative temporarily, once alternatives handled correctly, this will be changed match self { RequiredInputOrdering::Hard(lex) => &lex[0], RequiredInputOrdering::Soft(lex) => &lex[0], diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 83e94c880a6bf..9882a57b2b2fb 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -418,8 +418,8 @@ impl ExecutionPlan for SortMergeJoinExec { fn required_input_ordering(&self) -> Vec> { vec![ - Some(RequiredInputOrdering::from(self.left_sort_exprs.clone())), - Some(RequiredInputOrdering::from(self.right_sort_exprs.clone())), + RequiredInputOrdering::from(self.left_sort_exprs.clone()), + RequiredInputOrdering::from(self.right_sort_exprs.clone()), ] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index a6de0090a28b4..47613b857eb18 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -439,10 +439,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { vec![ self.left_sort_exprs .as_ref() - .map(|e| RequiredInputOrdering::from(e.clone())), + .and_then(|e| RequiredInputOrdering::from(e.clone())), self.right_sort_exprs .as_ref() - .map(|e| RequiredInputOrdering::from(e.clone())), + .and_then(|e| RequiredInputOrdering::from(e.clone())), ] } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 26455a94b683f..524e3c37b6770 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -242,7 +242,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn required_input_ordering(&self) -> Vec> { - vec![Some(RequiredInputOrdering::from(self.expr.clone()))] + vec![RequiredInputOrdering::from(self.expr.clone())] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 75c39748f749a..71608be21cfb2 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -38,7 +38,7 @@ use datafusion::datasource::source::DataSourceExec; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; -use datafusion::physical_expr::{LexOrdering, PhysicalExprRef}; +use datafusion::physical_expr::{LexOrdering, LexRequirement, PhysicalExprRef}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -1079,13 +1079,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(RequiredInputOrdering::from) + .map(LexRequirement::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order, + sort_order + .and_then(|order| RequiredInputOrdering::new(vec![order], false)), ))) } PhysicalPlanType::CsvSink(sink) => { @@ -1108,13 +1109,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(RequiredInputOrdering::from) + .map(LexRequirement::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order, + sort_order + .and_then(|order| RequiredInputOrdering::new(vec![order], false)), ))) } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] @@ -1144,13 +1146,15 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(RequiredInputOrdering::from) + .map(LexRequirement::from) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order, + sort_order.and_then(|order| { + RequiredInputOrdering::new(vec![order], false) + }), ))) } #[cfg(not(feature = "parquet"))] From cb8f763a82dda756b15608c883e7276214672d47 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 11:11:16 +0300 Subject: [PATCH 021/167] add default test cases add requirements compatible test cases --- .../replace_with_order_preserving_variants.rs | 1 - .../tests/physical_optimizer/test_utils.rs | 6 +- .../src/equivalence/properties/dependency.rs | 82 +++++++++++++++++++ .../src/equivalence/properties/mod.rs | 57 ------------- 4 files changed, 85 insertions(+), 61 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index ebabad791a049..f69299d5701eb 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -994,7 +994,6 @@ async fn test_with_lost_and_kept_ordering( " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; - assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, expected_input_bounded, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 5af836029177b..71316520a422a 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -373,9 +373,9 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - let requirement = - RequiredInputOrdering::from(self.required_input_ordering.clone()); - vec![requirement] + vec![RequiredInputOrdering::from( + self.required_input_ordering.clone(), + )] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 9eba295e562e2..03df6ee0c9de2 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -444,6 +444,9 @@ mod tests { use datafusion_functions::string::concat; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_physical_expr_common::sort_expr::{ + LexRequirement, PhysicalSortRequirement, + }; #[test] fn project_equivalence_properties_test() -> Result<()> { @@ -1349,6 +1352,85 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_equivalence_with_empty_requirements() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); + + let col_a = col("a", &schema)?; + + let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); + assert_eq!( + eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), + true + ); + + eq_properties.add_new_ordering(LexOrdering::from(vec![ + PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), + ])); + assert_eq!( + eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), + true + ); + + Ok(()) + } + + #[test] + fn test_requirements_compatible() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ])); + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_c = col("c", &schema)?; + + let eq_properties = EquivalenceProperties::new(schema); + let default_lex = LexRequirement::default(); + let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { + expr: col_a.clone(), + options: None, + }]); + let lex_a_b = LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: col_a, + options: None, + }, + PhysicalSortRequirement { + expr: col_b, + options: None, + }, + ]); + let lex_c = LexRequirement::new(vec![PhysicalSortRequirement { + expr: col_c, + options: None, + }]); + + let res = eq_properties.requirements_compatible(&default_lex, &default_lex); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_a, &default_lex); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&default_lex, &lex_a); + assert_eq!(res, false); + + let res = eq_properties.requirements_compatible(&lex_a, &lex_a); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_a, &lex_a_b); + assert_eq!(res, false); + + let res = eq_properties.requirements_compatible(&lex_a_b, &lex_a); + assert_eq!(res, true); + + let res = eq_properties.requirements_compatible(&lex_c, &lex_a); + assert_eq!(res, false); + + Ok(()) + } + #[test] fn test_with_reorder_constant_filtering() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 21ab84efc3090..dd266c6535e54 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1643,61 +1643,4 @@ mod tests { assert!(is_constant_recurse(&constants, &expr)); Ok(()) } - #[test] - fn test_requirements_compatible() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), - ])); - let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; - let col_d = col("d", &schema)?; - - let eq_properties = EquivalenceProperties::new(schema); - let default_lex = LexRequirement::default(); - let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { - expr: col_a.clone(), - options: None, - }]); - let lex_a_b = LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: col_a, - options: None, - }, - PhysicalSortRequirement { - expr: col_b, - options: None, - }, - ]); - let lex_d = LexRequirement::new(vec![PhysicalSortRequirement { - expr: col_d, - options: None, - }]); - - let res = eq_properties.requirements_compatible(&default_lex, &default_lex); - assert_eq!(res, true); - - let res = eq_properties.requirements_compatible(&lex_a, &default_lex); - assert_eq!(res, true); - - let res = eq_properties.requirements_compatible(&default_lex, &lex_a); - assert_eq!(res, false); - - let res = eq_properties.requirements_compatible(&lex_a, &lex_a); - assert_eq!(res, true); - - let res = eq_properties.requirements_compatible(&lex_a, &lex_a_b); - assert_eq!(res, false); - - let res = eq_properties.requirements_compatible(&lex_a_b, &lex_a); - assert_eq!(res, true); - - let res = eq_properties.requirements_compatible(&lex_d, &lex_a); - assert_eq!(res, false); - - Ok(()) - } } From 1c1f116b5b0f7a8ee88769ded5a0a245938a86c7 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 11:29:45 +0300 Subject: [PATCH 022/167] doc fixes --- .../src/enforce_sorting/mod.rs | 33 ++++++++----------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 69aefabb40c59..e15da452e7b3d 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -191,25 +191,20 @@ fn update_coalesce_ctx_children( } /// Performs optimizations based upon a series of subrules. -/// /// Refer to each subrule for detailed descriptions of the optimizations performed: -/// [`ensure_sorting`], [`parallelize_sorts`], [`replace_with_order_preserving_variants()`], -/// and [`pushdown_sorts`]. -/// /// Subrule application is ordering dependent. /// -/// The subrule `parallelize_sorts` is only applied if `repartition_sorts` is enabled. /// Optimizer consists of 5 main parts which work sequentially -/// 1. `ensure_sorting` Responsible for removing unnecessary [`SortExec`]s, [`SortPreservingMergeExec`]s -/// adjusting window operators, etc. -/// 2. `parallelize_sorts` (Depends on the repartition_sorts configuration) Responsible to identify -/// and remove unnecessary partition unifier operators such as [`SortPreservingMergeExec`], [`CoalescePartitionsExec`] -/// follows [`SortExec`]s does possible simplifications. -/// 3. `replace_with_order_preserving_variants` Replaces operators with order preserving variants, for example can merge -/// a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`] or [`SortExec`] + [`RepartitionExec`] -/// into an order preserving [`RepartitionExec`], etc. -/// 4. `sort_pushdown` Responsible to push down sort operators as deep as possible in the plan. -/// 5. `replace_with_partial_sort` Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators +/// 1. [`ensure_sorting`] Works down-to-top to be able to remove unnecessary [`SortExec`]s, [`SortPreservingMergeExec`]s +/// add [`SortExec`]s if necessary by a requirement and adjusts window operators. +/// 2. [`parallelize_sorts`] (Optional, depends on the `repartition_sorts` configuration) +/// Responsible to identify and remove unnecessary partition unifier operators +/// such as [`SortPreservingMergeExec`], [`CoalescePartitionsExec`] follows [`SortExec`]s does possible simplifications. +/// 3. [`replace_with_order_preserving_variants`] Replaces with alternative operators, for example can merge +/// a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`] +/// or a [`SortExec`] + [`RepartitionExec`] combination into an order preserving [`RepartitionExec`] +/// 4. [`sort_pushdown`] Works top-down. Responsible to push down sort operators as deep as possible in the plan. +/// 5. [`replace_with_partial_sort`] Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, @@ -376,19 +371,17 @@ fn replace_with_partial_sort( /// " CoalescePartitionsExec", /// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", /// ``` -/// by following connections from [`CoalescePartitionsExec`]s to [`SortExec`]s. -/// By performing sorting in parallel, we can increase performance in some scenarios. /// /// **Steps** /// 1. Checks if the plan is either [`SortExec`]/[`SortPreservingMergeExec`]/[`CoalescePartitionsExec`] otherwise does nothing -/// 2. If the plan is a [`SortExec`] or a final `[SortPreservingMergeExec` (output partitioning is 1) +/// 2. If the plan is a [`SortExec`] or a final [`SortPreservingMergeExec`] (output partitioning is 1) /// 2.1. Check for [`CoalescePartitionsExec`] in children, when found check if it can be removed (with possible [`RepartitionExec`]s) -/// if so remove. (see `remove_bottleneck_in_subplan`) +/// if so remove. (see [`remove_bottleneck_in_subplan`]) /// 2.2. If the plan is satisfying the ordering requirements, add a `SortExec` /// 2.3. Add an SPM above the plan and return /// 3. If the plan is a [`CoalescePartitionsExec`] /// 3.1. Check if it can be removed (with possible [`RepartitionExec`]s) -/// if so remove (see `remove_bottleneck_in_subplan`) +/// if so remove (see [`remove_bottleneck_in_subplan`]) pub fn parallelize_sorts( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result> { From 7817b4f7661c7e2af5d652667b33b5c4dc9158c9 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 11:47:45 +0300 Subject: [PATCH 023/167] fix clippy and docs --- .../src/equivalence/properties/dependency.rs | 22 +++++++++---------- .../src/enforce_sorting/mod.rs | 2 +- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 03df6ee0c9de2..21892a8eed1c6 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1359,17 +1359,15 @@ mod tests { let col_a = col("a", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - assert_eq!( + assert!( eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), - true ); eq_properties.add_new_ordering(LexOrdering::from(vec![ PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), ])); - assert_eq!( + assert!( eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), - true ); Ok(()) @@ -1389,7 +1387,7 @@ mod tests { let eq_properties = EquivalenceProperties::new(schema); let default_lex = LexRequirement::default(); let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { - expr: col_a.clone(), + expr: Arc::clone(&col_a), options: None, }]); let lex_a_b = LexRequirement::new(vec![ @@ -1408,25 +1406,25 @@ mod tests { }]); let res = eq_properties.requirements_compatible(&default_lex, &default_lex); - assert_eq!(res, true); + assert!(res); let res = eq_properties.requirements_compatible(&lex_a, &default_lex); - assert_eq!(res, true); + assert!(res); let res = eq_properties.requirements_compatible(&default_lex, &lex_a); - assert_eq!(res, false); + assert!(!res); let res = eq_properties.requirements_compatible(&lex_a, &lex_a); - assert_eq!(res, true); + assert!(res); let res = eq_properties.requirements_compatible(&lex_a, &lex_a_b); - assert_eq!(res, false); + assert!(!res); let res = eq_properties.requirements_compatible(&lex_a_b, &lex_a); - assert_eq!(res, true); + assert!(res); let res = eq_properties.requirements_compatible(&lex_c, &lex_a); - assert_eq!(res, false); + assert!(!res); Ok(()) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index e15da452e7b3d..f01b41ae7ad92 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -200,7 +200,7 @@ fn update_coalesce_ctx_children( /// 2. [`parallelize_sorts`] (Optional, depends on the `repartition_sorts` configuration) /// Responsible to identify and remove unnecessary partition unifier operators /// such as [`SortPreservingMergeExec`], [`CoalescePartitionsExec`] follows [`SortExec`]s does possible simplifications. -/// 3. [`replace_with_order_preserving_variants`] Replaces with alternative operators, for example can merge +/// 3. [`replace_with_order_preserving_variants()`] Replaces with alternative operators, for example can merge /// a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`] /// or a [`SortExec`] + [`RepartitionExec`] combination into an order preserving [`RepartitionExec`] /// 4. [`sort_pushdown`] Works top-down. Responsible to push down sort operators as deep as possible in the plan. From 6b4398297fb9ec1eb8e23773a59364f52eec599b Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 11:49:20 +0300 Subject: [PATCH 024/167] format code --- .../src/equivalence/properties/dependency.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 21892a8eed1c6..8dcbc037e1198 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1359,16 +1359,12 @@ mod tests { let col_a = col("a", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - assert!( - eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), - ); + assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default()),); eq_properties.add_new_ordering(LexOrdering::from(vec![ PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), ])); - assert!( - eq_properties.ordering_satisfy_requirement(&LexRequirement::default()), - ); + assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default()),); Ok(()) } From 55b9cf1f7f71887ccaa06ca489a5857f1bcc9b82 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 11:49:41 +0300 Subject: [PATCH 025/167] format code --- .../physical-expr/src/equivalence/properties/dependency.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 8dcbc037e1198..055230889cba8 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1359,12 +1359,12 @@ mod tests { let col_a = col("a", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default()),); + assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default())); eq_properties.add_new_ordering(LexOrdering::from(vec![ PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), ])); - assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default()),); + assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default())); Ok(()) } From 41e42921f11950c5ec2d067cd6af8052afd5f01a Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 13:50:12 +0300 Subject: [PATCH 026/167] doc fix --- datafusion/physical-optimizer/src/enforce_sorting/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index f01b41ae7ad92..4081e731b468f 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -204,7 +204,7 @@ fn update_coalesce_ctx_children( /// a [`SortExec`] and a [`CoalescePartitionsExec`] into one [`SortPreservingMergeExec`] /// or a [`SortExec`] + [`RepartitionExec`] combination into an order preserving [`RepartitionExec`] /// 4. [`sort_pushdown`] Works top-down. Responsible to push down sort operators as deep as possible in the plan. -/// 5. [`replace_with_partial_sort`] Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators +/// 5. `replace_with_partial_sort` Checks if it's possible to replace [`SortExec`]s with [`PartialSortExec`] operators impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, @@ -376,12 +376,12 @@ fn replace_with_partial_sort( /// 1. Checks if the plan is either [`SortExec`]/[`SortPreservingMergeExec`]/[`CoalescePartitionsExec`] otherwise does nothing /// 2. If the plan is a [`SortExec`] or a final [`SortPreservingMergeExec`] (output partitioning is 1) /// 2.1. Check for [`CoalescePartitionsExec`] in children, when found check if it can be removed (with possible [`RepartitionExec`]s) -/// if so remove. (see [`remove_bottleneck_in_subplan`]) +/// if so remove. (see `remove_bottleneck_in_subplan`) /// 2.2. If the plan is satisfying the ordering requirements, add a `SortExec` /// 2.3. Add an SPM above the plan and return /// 3. If the plan is a [`CoalescePartitionsExec`] /// 3.1. Check if it can be removed (with possible [`RepartitionExec`]s) -/// if so remove (see [`remove_bottleneck_in_subplan`]) +/// if so remove (see `remove_bottleneck_in_subplan`) pub fn parallelize_sorts( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result> { From f80f0931da63a3d4983e12b7fbcab65af7a2d42c Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 21 Mar 2025 14:49:40 +0300 Subject: [PATCH 027/167] add TODO test cases with test_soft_hard_requirements prefix --- .../physical_optimizer/enforce_sorting.rs | 436 +++++++++++++++++- .../tests/physical_optimizer/test_utils.rs | 19 +- 2 files changed, 444 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 31cd528eaf391..e5f0e60abfb49 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -18,13 +18,14 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, - coalesce_partitions_exec, create_test_schema, create_test_schema2, - create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, - local_limit_exec, memory_exec, parquet_exec, repartition_exec, sort_exec, - sort_exec_with_fetch, sort_expr, sort_expr_options, sort_merge_join_exec, - sort_preserving_merge_exec, sort_preserving_merge_exec_with_fetch, - spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, + aggregate_exec, bounded_window_exec, bounded_window_exec_with_partition, + check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, + create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, + hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_exec, + projection_exec, repartition_exec, sort_exec, sort_exec_with_fetch, sort_expr, + sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, + sort_preserving_merge_exec_with_fetch, spr_repartition_exec, stream_exec_ordered, + union_exec, RequirementsTestExec, }; use arrow::compute::SortOptions; @@ -35,9 +36,9 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{JoinType, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr::expressions::{col, Column, NotExpr}; -use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement, PhysicalSortExpr}; +use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; +use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -55,6 +56,9 @@ use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; +use datafusion_expr_common::operator::Operator; +use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use rstest::rstest; @@ -629,6 +633,418 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_soft_hard_requirements_remove_soft_requirement() -> Result<()> { + let schema = create_test_schema()?; + let source = parquet_exec_sorted(&schema, vec![]); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), Arc::clone(&source)); + let partition_bys = &[col("nullable_col", &schema)?]; + let bounded_window = + bounded_window_exec_with_partition("nullable_col", vec![], partition_bys, sort); + + let physical_plan = bounded_window; + + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ] + ; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( +) -> Result<()> { + let schema = create_test_schema()?; + let source = parquet_exec_sorted(&schema, vec![]); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source.clone()); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("nullable_col", &schema).unwrap(), + Operator::Plus, + col("non_nullable_col", &schema).unwrap(), + )) as Arc, + "count".to_string(), + )]; + let partition_bys = &[col("nullable_col", &schema)?]; + let bounded_window = + bounded_window_exec_with_partition("nullable_col", vec![], partition_bys, sort); + let projection = projection_exec(proj_exprs, bounded_window)?; + let physical_plan = projection; + + let expected_input = [ + "ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as count]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as count]", + // " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as count]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source.clone()); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("nullable_col", &schema).unwrap(), + Operator::Plus, + col("non_nullable_col", &schema).unwrap(), + )) as Arc, + "nullable_col".to_string(), + )]; + let partition_bys = &[col("nullable_col", &schema)?]; + let projection = projection_exec(proj_exprs, sort)?; + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + vec![], + partition_bys, + projection, + ); + let physical_plan = bounded_window; + + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> { + let schema = create_test_schema()?; + let source = parquet_exec_sorted(&schema, vec![]); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source.clone()); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("nullable_col", &schema).unwrap(), + Operator::Plus, + col("non_nullable_col", &schema).unwrap(), + )) as Arc, + "nullable_col".to_string(), + )]; + let partition_bys = &[col("nullable_col", &schema)?]; + let projection = projection_exec(proj_exprs, sort)?; + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + vec![], + partition_bys, + projection, + ); + let bounded_window2 = bounded_window_exec_with_partition( + "count", + vec![], + partition_bys, + bounded_window, + ); + let physical_plan = bounded_window2; + + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source.clone()); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("nullable_col", &schema).unwrap(), + Operator::Plus, + col("non_nullable_col", &schema).unwrap(), + )) as Arc, + "nullable_col".to_string(), + )]; + let partition_bys = &[col("nullable_col", &schema)?]; + let projection = projection_exec(proj_exprs, sort)?; + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + vec![], + partition_bys, + projection, + ); + + let sort_exprs2 = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort2 = sort_exec(sort_exprs2.clone(), bounded_window.clone()); + let sort3 = sort_exec(sort_exprs2.clone(), sort2); + let bounded_window2 = + bounded_window_exec_with_partition("count", vec![], partition_bys, sort3); + let physical_plan = bounded_window2; + + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ] + ; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} +#[tokio::test] +async fn test_soft_hard_requirements_multiple_sorts() -> Result<()> { + let schema = create_test_schema()?; + let source = parquet_exec_sorted(&schema, vec![]); + + let sort_exprs = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source.clone()); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("nullable_col", &schema).unwrap(), + Operator::Plus, + col("non_nullable_col", &schema).unwrap(), + )) as Arc, + "nullable_col".to_string(), + )]; + let partition_bys = &[col("nullable_col", &schema)?]; + let projection = projection_exec(proj_exprs, sort)?; + let bounded_window = bounded_window_exec_with_partition( + "nullable_col", + vec![], + partition_bys, + projection, + ); + + let sort_exprs2 = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort2 = sort_exec(sort_exprs2.clone(), bounded_window.clone()); + let sort3 = sort_exec(sort_exprs2.clone(), sort2); + let physical_plan = sort3; + + let expected_input = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_requirement( +) -> Result<()> { + let schema = create_test_schema()?; + let source = parquet_exec_sorted(&schema, vec![]); + + let sort_exprs1 = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs1.clone(), source.clone()); + let partition_bys1 = &[col("nullable_col", &schema)?]; + let bounded_window = + bounded_window_exec_with_partition("nullable_col", vec![], partition_bys1, sort); + + let sort_exprs2 = vec![sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: false, + nulls_first: true, + }, + )]; + let partition_bys2 = &[col("non_nullable_col", &schema)?]; + let bounded_window2 = bounded_window_exec_with_partition( + "non_nullable_col", + vec![], + partition_bys2, + bounded_window, + ); + let output_requirements: Arc = + Arc::new(OutputRequirementExec::new( + bounded_window2, + RequiredInputOrdering::new( + vec![LexRequirement::from_lex_ordering(LexOrdering::new( + sort_exprs2, + ))], + false, + ), + Distribution::SinglePartition, + )); + let physical_plan = output_requirements; + + let expected_input = [ + "OutputRequirementExec", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + // TODO When sort pushdown respects to the alternatives, and removes soft SortExecs this should be changed + // let expected_optimized = [ + // "OutputRequirementExec", + // " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + // " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", + // " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Linear]", + // " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + // ]; + let expected_optimized = [ + "OutputRequirementExec", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + #[tokio::test] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 71316520a422a..d28b881f19388 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -54,6 +54,7 @@ use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -194,6 +195,15 @@ pub fn bounded_window_exec( col_name: &str, sort_exprs: impl IntoIterator, input: Arc, +) -> Arc { + bounded_window_exec_with_partition(col_name, sort_exprs, &[], input) +} + +pub fn bounded_window_exec_with_partition( + col_name: &str, + sort_exprs: impl IntoIterator, + partition_by: &[Arc], + input: Arc, ) -> Arc { let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); @@ -201,7 +211,7 @@ pub fn bounded_window_exec( &WindowFunctionDefinition::AggregateUDF(count_udaf()), "count".to_owned(), &[col(col_name, &schema).unwrap()], - &[], + partition_by, sort_exprs.as_ref(), Arc::new(WindowFrame::new(Some(false))), schema.as_ref(), @@ -307,6 +317,13 @@ pub fn sort_exec_with_fetch( Arc::new(SortExec::new(sort_exprs, input).with_fetch(fetch)) } +pub fn projection_exec( + expr: Vec<(Arc, String)>, + input: Arc, +) -> Result> { + Ok(Arc::new(ProjectionExec::try_new(expr, input)?)) +} + /// A test [`ExecutionPlan`] whose requirements can be configured. #[derive(Debug)] pub struct RequirementsTestExec { From b41fa2f2c209d49ced52d921b0e0f4b6a83d28a9 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 23 Mar 2025 22:45:01 +0300 Subject: [PATCH 028/167] Review Part 1 --- datafusion/catalog/src/stream.rs | 21 ++--- .../core/src/datasource/listing/table.rs | 66 +++++----------- .../physical_optimizer/enforce_sorting.rs | 9 +-- .../physical_optimizer/projection_pushdown.rs | 64 +++++++-------- .../physical-expr-common/src/sort_expr.rs | 42 +++------- .../src/enforce_sorting/sort_pushdown.rs | 78 +++++++------------ .../src/output_requirements.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/execution_plan.rs | 59 ++++++-------- datafusion/physical-plan/src/windows/mod.rs | 12 +-- datafusion/proto/src/physical_plan/mod.rs | 10 +-- .../tests/cases/roundtrip_physical_plan.rs | 13 ++-- 12 files changed, 142 insertions(+), 236 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index 75ab12225839f..8bec5268d1f40 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -351,23 +351,18 @@ impl TableProvider for StreamTable { input: Arc, _insert_op: InsertOp, ) -> Result> { - let ordering = match self.0.order.first() { - Some(x) => { - let schema = self.0.source.schema(); - let orders = create_ordering(schema, std::slice::from_ref(x))?; - let ordering = orders.into_iter().next().unwrap(); - RequiredInputOrdering::new( - vec![ordering.into_iter().map(Into::into).collect()], - false, - ) - } - None => None, - }; + let schema = self.0.source.schema(); + let orders = create_ordering(schema, &self.0.order)?; + // It is sufficient to pass only one of the equivalent orderings: + let order_requirements = orders.first().and_then(|ordering| { + let reqs = ordering.iter().cloned().map(Into::into).collect(); + RequiredInputOrdering::new(reqs) + }); Ok(Arc::new(DataSinkExec::new( input, Arc::new(StreamWrite(Arc::clone(&self.0))), - ordering, + order_requirements, ))) } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 1b85b4d040cac..0e3c175127e2b 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -22,40 +22,31 @@ use std::{any::Any, str::FromStr, sync::Arc}; use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; use super::{ListingTableUrl, PartitionedFile}; - -use crate::datasource::{ - create_ordering, - file_format::{ - file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport, - }, - get_statistics_with_limit, - physical_plan::FileSinkConfig, +use crate::datasource::file_format::{ + file_compression_type::FileCompressionType, FileFormat, FilePushdownSupport, }; +use crate::datasource::physical_plan::FileSinkConfig; +use crate::datasource::{create_ordering, get_statistics_with_limit}; use crate::execution::context::SessionState; -use datafusion_catalog::TableProvider; -use datafusion_common::{config_err, DataFusionError, Result}; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_expr::dml::InsertOp; -use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; -use datafusion_expr::{SortExpr, TableType}; -use datafusion_physical_plan::empty::EmptyExec; -use datafusion_physical_plan::{ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; -use datafusion_catalog::Session; +use datafusion_catalog::{Session, TableProvider}; use datafusion_common::{ - config_datafusion_err, internal_err, plan_err, project_schema, Constraints, - SchemaExt, ToDFSchema, + config_datafusion_err, config_err, internal_err, plan_err, project_schema, + Constraints, DataFusionError, Result, SchemaExt, ToDFSchema, }; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::cache::{ cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache, }; -use datafusion_physical_expr::{ - create_physical_expr, LexOrdering, PhysicalSortRequirement, -}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_expr::dml::InsertOp; +use datafusion_expr::utils::conjunction; +use datafusion_expr::{Expr, SortExpr, TableProviderFilterPushDown, TableType}; +use datafusion_physical_expr::{create_physical_expr, LexOrdering}; +use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_plan::{ExecutionPlan, Statistics}; use async_trait::async_trait; use futures::{future, stream, StreamExt, TryStreamExt}; @@ -278,6 +269,7 @@ pub struct ListingOptions { /// parquet metadata. /// /// See + /// /// NOTE: This attribute stores all equivalent orderings (the outer `Vec`) /// where each ordering consists of an individual lexicographic /// ordering (encapsulated by a `Vec`). If there aren't @@ -1050,28 +1042,12 @@ impl TableProvider for ListingTable { file_extension: self.options().format.get_ext(), }; - let order_requirements = if !self.options().file_sort_order.is_empty() { - // Multiple sort orders in outer vec are equivalent, so we pass only the first one - let orderings = self.try_create_output_ordering()?; - let Some(ordering) = orderings.first() else { - return internal_err!( - "Expected ListingTable to have a sort order, but none found!" - ); - }; - // Converts Vec> into type required by execution plan to specify its required input ordering - RequiredInputOrdering::new( - vec![LexRequirement::new( - ordering - .into_iter() - .cloned() - .map(PhysicalSortRequirement::from) - .collect::>(), - )], - false, - ) - } else { - None - }; + let orderings = self.try_create_output_ordering()?; + // It is sufficient to pass only one of the equivalent orderings: + let order_requirements = orderings.first().and_then(|ordering| { + let reqs = ordering.iter().cloned().map(Into::into).collect(); + RequiredInputOrdering::new(reqs) + }); self.options() .format diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index e5f0e60abfb49..3f8312668a43a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -1008,12 +1008,9 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ let output_requirements: Arc = Arc::new(OutputRequirementExec::new( bounded_window2, - RequiredInputOrdering::new( - vec![LexRequirement::from_lex_ordering(LexOrdering::new( - sort_exprs2, - ))], - false, - ), + RequiredInputOrdering::new(LexRequirement::from(LexOrdering::from( + sort_exprs2, + ))), Distribution::SinglePartition, )); let physical_plan = output_requirements; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 6d073de77e036..4d445f2793f0c 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -648,24 +648,21 @@ fn test_projection_after_projection() -> Result<()> { fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( - csv.clone(), - RequiredInputOrdering::new( - vec![LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ])], - false, - ), + csv, + RequiredInputOrdering::new(LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 1)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: Some(SortOptions::default()), + }, + ])), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -698,23 +695,20 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = RequiredInputOrdering::new( - vec![LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ])], - false, - ) + let expected_reqs = RequiredInputOrdering::new(LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 2)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_a", 1)), + )), + options: Some(SortOptions::default()), + }, + ])) .unwrap(); assert_eq!( after_optimize diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index ca7769a6b2514..a1d19866bae1e 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -236,12 +236,12 @@ impl From for PhysicalSortExpr { impl From for PhysicalSortRequirement { fn from(value: PhysicalSortExpr) -> Self { - PhysicalSortRequirement::new(value.expr, Some(value.options)) + Self::new(value.expr, Some(value.options)) } } impl PartialEq for PhysicalSortRequirement { - fn eq(&self, other: &PhysicalSortRequirement) -> bool { + fn eq(&self, other: &Self) -> bool { self.options == other.options && self.expr.eq(&other.expr) } } @@ -304,21 +304,6 @@ impl PhysicalSortRequirement { .options .is_none_or(|other_opts| self.options == Some(other_opts)) } - - #[deprecated(since = "43.0.0", note = "use LexRequirement::from_lex_ordering")] - pub fn from_sort_exprs<'a>( - ordering: impl IntoIterator, - ) -> LexRequirement { - let ordering = ordering.into_iter().cloned().collect(); - LexRequirement::from_lex_ordering(ordering) - } - #[deprecated(since = "43.0.0", note = "use LexOrdering::from_lex_requirement")] - pub fn to_sort_exprs( - requirements: impl IntoIterator, - ) -> LexOrdering { - let requirements = requirements.into_iter().collect(); - LexOrdering::from_lex_requirement(requirements) - } } /// Returns the SQL string representation of the given [SortOptions] object. @@ -480,7 +465,7 @@ impl From> for LexOrdering { impl From for LexOrdering { fn from(value: LexRequirement) -> Self { - Self::from_lex_requirement(value) + value.into_iter().map(Into::into).collect() } } @@ -604,19 +589,6 @@ impl LexRequirement { self.inner.extend(requirements) } - /// Create a new [`LexRequirement`] from a [`LexOrdering`] - /// - /// Returns [`LexRequirement`] that requires the exact - /// sort of the [`PhysicalSortExpr`]s in `ordering` - pub fn from_lex_ordering(ordering: LexOrdering) -> Self { - Self::new( - ordering - .into_iter() - .map(PhysicalSortRequirement::from) - .collect(), - ) - } - /// Constructs a duplicate-free `LexOrderingReq` by filtering out /// duplicate entries that have same physical expression inside. /// @@ -633,9 +605,15 @@ impl LexRequirement { } } +impl From> for LexRequirement { + fn from(value: Vec) -> Self { + Self::new(value) + } +} + impl From for LexRequirement { fn from(value: LexOrdering) -> Self { - Self::from_lex_ordering(value) + Self::new(value.into_iter().map(Into::into).collect()) } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index dd451f1a7d709..5f301b9f58262 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -91,7 +91,7 @@ fn pushdown_sorts_helper( ) -> Result> { let plan = &Arc::clone(&sort_push_down.plan); let parent_reqs = sort_push_down.data.ordering_requirement.clone(); - let satisfy_parent = parent_reqs.clone().is_none_or(|reqs| { + let satisfy_parent = parent_reqs.as_ref().is_none_or(|reqs| { plan.equivalence_properties() .ordering_satisfy_requirement(reqs.lex_requirement()) }); @@ -105,13 +105,13 @@ fn pushdown_sorts_helper( let current_plan_ordering_as_req = RequiredInputOrdering::from(current_plan_ordering).unwrap(); - let parent_is_stricter = parent_reqs.clone().is_some_and(|parent_req| { + let parent_is_stricter = parent_reqs.as_ref().is_some_and(|parent_req| { plan.equivalence_properties().requirements_compatible( parent_req.lex_requirement(), current_plan_ordering_as_req.lex_requirement(), ) }); - let current_is_stricter = parent_reqs.clone().is_none_or(|parent_req| { + let current_is_stricter = parent_reqs.as_ref().is_none_or(|parent_req| { plan.equivalence_properties().requirements_compatible( current_plan_ordering_as_req.lex_requirement(), parent_req.lex_requirement(), @@ -128,7 +128,7 @@ fn pushdown_sorts_helper( sort_push_down = sort_push_down.children.swap_remove(0); sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan - if let Some(parent_reqs) = parent_reqs.clone() { + if let Some(parent_reqs) = &parent_reqs { // add back sort exec matching parent sort_push_down = add_sort_above( sort_push_down, @@ -155,12 +155,11 @@ fn pushdown_sorts_helper( sort_push_down.data.fetch = min_fetch(current_sort_fetch, parent_req_fetch); // set the stricter ordering - if current_is_stricter { - sort_push_down.data.ordering_requirement = - Some(current_plan_ordering_as_req); + sort_push_down.data.ordering_requirement = if current_is_stricter { + Some(current_plan_ordering_as_req) } else { - sort_push_down.data.ordering_requirement = parent_reqs; - } + parent_reqs + }; // recursive call to helper, so it doesn't transform_down and miss the new node (previous child of sort) return pushdown_sorts_helper(sort_push_down); @@ -195,13 +194,10 @@ fn pushdown_sorts_helper( sort_push_down.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - let sort_reqs = sort_push_down.data.ordering_requirement.clone(); - if let Some(sort_reqs) = sort_reqs { - sort_push_down = add_sort_above( - sort_push_down, - sort_reqs.lex_requirement().clone(), - parent_fetch, - ); + if let Some(sort_reqs) = &sort_push_down.data.ordering_requirement { + let sort_requirements = sort_reqs.lex_requirement().clone(); + sort_push_down = + add_sort_above(sort_push_down, sort_requirements, parent_fetch); } assign_initial_requirements(&mut sort_push_down); } @@ -217,20 +213,14 @@ fn pushdown_requirement_to_children( ) -> Result>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { - let required_input_ordering = plan.required_input_ordering(); - let maybe_child_requirement = required_input_ordering[0].clone(); + let mut required_input_ordering = plan.required_input_ordering(); + let maybe_child_requirement = required_input_ordering.swap_remove(0); let child_plan = plan.children().swap_remove(0); let Some(child_req) = maybe_child_requirement else { return Ok(None); }; match determine_children_requirement(parent_required, &child_req, child_plan) { - RequirementsCompatibility::Satisfy => { - let req = RequiredInputOrdering::new( - vec![LexRequirement::new(child_req.to_vec())], - false, - ); - Ok(Some(vec![req])) - } + RequirementsCompatibility::Satisfy => Ok(Some(vec![Some(child_req)])), RequirementsCompatibility::Compatible(adjusted) => { // If parent requirements are more specific than output ordering // of the window plan, then we can deduce that the parent expects @@ -287,11 +277,7 @@ fn pushdown_requirement_to_children( .eq_properties .requirements_compatible(parent_required.lex_requirement(), &output_req) { - let req = RequiredInputOrdering::new( - vec![LexRequirement::new(parent_required.to_vec())], - false, - ); - Ok(Some(vec![req])) + Ok(Some(vec![Some(parent_required.clone())])) } else { Ok(None) } @@ -353,19 +339,16 @@ fn pushdown_requirement_to_children( let mut spm_eqs = plan.equivalence_properties().clone(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); - // Do not push-down through SortPreservingMergeExec when - // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_eqs.ordering_satisfy(&plan.output_ordering().cloned().unwrap_or_default()) + if spm_eqs + .ordering_satisfy(plan.output_ordering().unwrap_or(&LexOrdering::default())) { - Ok(None) - } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - let req = RequiredInputOrdering::new( - vec![LexRequirement::new(parent_required.to_vec())], - false, - ); - Ok(Some(vec![req])) + Ok(Some(vec![Some(parent_required.clone())])) + } else { + // Do not push-down through SortPreservingMergeExec when + // ordering requirement invalidates requirement of sort preserving merge exec. + Ok(None) } } else if let Some(hash_join) = plan.as_any().downcast_ref::() { handle_hash_join(hash_join, parent_required) @@ -409,23 +392,20 @@ fn determine_children_requirement( child_requirement: &RequiredInputOrdering, child_plan: &Arc, ) -> RequirementsCompatibility { - if child_plan.equivalence_properties().requirements_compatible( + let eqp = child_plan.equivalence_properties(); + if eqp.requirements_compatible( child_requirement.lex_requirement(), parent_required.lex_requirement(), ) { // Child requirements are more specific, no need to push down. RequirementsCompatibility::Satisfy - } else if child_plan.equivalence_properties().requirements_compatible( + } else if eqp.requirements_compatible( parent_required.lex_requirement(), child_requirement.lex_requirement(), ) { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: - let adjusted = RequiredInputOrdering::new( - vec![LexRequirement::new(parent_required.to_vec())], - false, - ); - RequirementsCompatibility::Compatible(adjusted) + RequirementsCompatibility::Compatible(Some(parent_required.clone())) } else { RequirementsCompatibility::NonCompatible } @@ -678,7 +658,7 @@ fn handle_custom_pushdown( .iter() .map(|&maintains_order| { if maintains_order { - parent_required.with_updated_requirements(updated_parent_req.clone()) + RequiredInputOrdering::new(updated_parent_req.clone().into()) } else { None } @@ -756,7 +736,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - parent_required.with_updated_requirements(updated_parent_req), + RequiredInputOrdering::new(updated_parent_req.into()), ])) } else { Ok(None) diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index ec531fb35102b..99ef1966d0fad 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -247,7 +247,7 @@ impl ExecutionPlan for OutputRequirementExec { .map(|input| { OutputRequirementExec::new( input, - RequiredInputOrdering::new(vec![updated_sort_reqs], false), + RequiredInputOrdering::new(updated_sort_reqs), dist_req, ) }) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 309f8c5e84f41..7058f95ca6f84 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -523,7 +523,7 @@ impl AggregateExec { ProjectionMapping::try_new(&group_by.expr, &input.schema())?; let required_input_ordering = - RequiredInputOrdering::new(vec![new_requirement], true); + RequiredInputOrdering::new_with_alternatives(vec![new_requirement], true); let cache = Self::compute_properties( &input, diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index a9c0f884d2af7..932ac1ae004d5 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -49,9 +49,7 @@ use datafusion_common::{exec_err, Constraints, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_expr_common::sort_expr::{ - LexRequirement, PhysicalSortRequirement, -}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::{StreamExt, TryStreamExt}; @@ -1087,62 +1085,53 @@ impl RequiredInputOrdering { /// Creates a new RequiredInputOrdering instance, /// empty requirements are not allowed inside this type, /// if given [`None`] will be returned - pub fn new(lex_requirements: Vec, is_soft: bool) -> Option { - if lex_requirements.is_empty() || lex_requirements[0].is_empty() { - return None; - } - - Some(if is_soft { - RequiredInputOrdering::Soft(lex_requirements) - } else { - RequiredInputOrdering::Hard(lex_requirements) + pub fn new_with_alternatives( + lex_requirements: Vec, + soft: bool, + ) -> Option { + (!(lex_requirements.is_empty() || lex_requirements[0].is_empty())).then(|| { + if soft { + Self::Soft(lex_requirements) + } else { + Self::Hard(lex_requirements) + } }) } + pub fn new(requirement: LexRequirement) -> Option { + (!requirement.is_empty()).then(|| Self::Hard(vec![requirement])) + } + pub fn from(ordering: LexOrdering) -> Option { - RequiredInputOrdering::new(vec![LexRequirement::from(ordering)], false) + Self::new(LexRequirement::from(ordering)) } + /// Returns the first (i.e. most preferred) among alternative requirements. pub fn lex_requirement(&self) -> &LexRequirement { - // TODO This function is returning the first alternative temporarily, once alternatives handled correctly, this will be changed match self { - RequiredInputOrdering::Hard(lex) => &lex[0], - RequiredInputOrdering::Soft(lex) => &lex[0], + Self::Hard(lex) => &lex[0], + Self::Soft(lex) => &lex[0], } } pub fn is_empty(&self) -> bool { self.lex_requirement().is_empty() } - - pub fn to_vec(&self) -> Vec { - self.lex_requirement().to_vec() - } - - pub fn with_updated_requirements( - &self, - requirement: Vec, - ) -> Option { - RequiredInputOrdering::new( - vec![LexRequirement::new(requirement)], - matches!(self, RequiredInputOrdering::Soft(_)), - ) - } } #[cfg(test)] mod tests { - use super::*; - use arrow::array::{DictionaryArray, Int32Array, NullArray, RunArray}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use std::any::Any; use std::sync::Arc; + use super::*; + use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; + + use arrow::array::{DictionaryArray, Int32Array, NullArray, RunArray}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; - #[derive(Debug)] pub struct EmptyExec; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ef8f90912c741..d4d6ae656b209 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -301,7 +301,10 @@ pub(crate) fn calc_requirements< } } - RequiredInputOrdering::new(vec![sort_reqs_with_partition, sort_reqs], false) + RequiredInputOrdering::new_with_alternatives( + vec![sort_reqs_with_partition, sort_reqs], + false, + ) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -787,14 +790,13 @@ mod tests { lex_requirement.push(res); } if !lex_requirement.is_empty() { - if let Some(expect) = expected { - expected = RequiredInputOrdering::new( + expected = if let Some(expect) = expected { + RequiredInputOrdering::new_with_alternatives( vec![expect.lex_requirement().clone(), lex_requirement], false, ) } else { - expected = - RequiredInputOrdering::new(vec![lex_requirement], false) + RequiredInputOrdering::new(lex_requirement) } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 3b79ea3d45020..1fffc9d369dc0 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1094,8 +1094,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order - .and_then(|order| RequiredInputOrdering::new(vec![order], false)), + sort_order.and_then(RequiredInputOrdering::new), ))) } PhysicalPlanType::CsvSink(sink) => { @@ -1124,8 +1123,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order - .and_then(|order| RequiredInputOrdering::new(vec![order], false)), + sort_order.and_then(RequiredInputOrdering::new), ))) } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] @@ -1161,9 +1159,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.and_then(|order| { - RequiredInputOrdering::new(vec![order], false) - }), + sort_order.and_then(RequiredInputOrdering::new), ))) } #[cfg(not(feature = "parquet"))] diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index ef0e7dd943603..8e1fd119bf3e0 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -1320,7 +1320,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - RequiredInputOrdering::new(vec![sort_order], false), + RequiredInputOrdering::new(sort_order), ))) } @@ -1346,16 +1346,15 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = RequiredInputOrdering::new( - vec![LexRequirement::new(vec![PhysicalSortRequirement::new( + let sort_order = RequiredInputOrdering::new(LexRequirement::new(vec![ + PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )])], - false, - ); + ), + ])); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1416,7 +1415,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - RequiredInputOrdering::new(vec![sort_order], false), + RequiredInputOrdering::new(sort_order), ))) } From 117783f8845a816a8ed1846e8b18d1fc29c8eac7 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 23 Mar 2025 23:43:59 +0300 Subject: [PATCH 029/167] Review Part 2 --- .../physical-expr-common/src/sort_expr.rs | 28 ++++--------------- .../src/enforce_sorting/sort_pushdown.rs | 11 ++++---- .../physical-plan/src/aggregates/mod.rs | 2 +- 3 files changed, 12 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index a1d19866bae1e..560a8d19551de 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -501,12 +501,8 @@ impl Display for LexOrdering { impl FromIterator for LexOrdering { fn from_iter>(iter: T) -> Self { - let mut lex_ordering = LexOrdering::default(); - - for i in iter { - lex_ordering.push(i); - } - + let mut lex_ordering = Self::default(); + lex_ordering.extend(iter); lex_ordering } } @@ -552,11 +548,6 @@ impl IntoIterator for LexOrdering { } } -///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents -/// a reference to a lexicographical ordering. -#[deprecated(since = "43.0.0", note = "use &LexOrdering instead")] -pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; - ///`LexRequirement` is an struct containing a `Vec`, which /// represents a lexicographical ordering requirement. #[derive(Debug, Default, Clone, PartialEq)] @@ -585,7 +576,7 @@ impl LexRequirement { self.inner.push(physical_sort_requirement) } - pub fn extend(&mut self, requirements: Vec) { + pub fn extend(&mut self, requirements: impl IntoIterator) { self.inner.extend(requirements) } @@ -627,12 +618,8 @@ impl Deref for LexRequirement { impl FromIterator for LexRequirement { fn from_iter>(iter: T) -> Self { - let mut lex_requirement = LexRequirement::new(vec![]); - - for i in iter { - lex_requirement.inner.push(i); - } - + let mut lex_requirement = Self::default(); + lex_requirement.extend(iter); lex_requirement } } @@ -654,8 +641,3 @@ impl<'a> IntoIterator for &'a LexOrdering { self.inner.iter() } } - -///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which -/// represents a reference to a lexicographical ordering requirement. -/// #[deprecated(since = "43.0.0", note = "use &LexRequirement instead")] -pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 5f301b9f58262..7b6e86c7b4ce5 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -24,7 +24,7 @@ use crate::utils::{ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{plan_err, HashSet, JoinSide, Result}; +use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; @@ -100,10 +100,11 @@ fn pushdown_sorts_helper( let current_sort_fetch = plan.fetch(); let parent_req_fetch = sort_push_down.data.fetch; - let current_plan_ordering = plan.output_ordering().cloned().unwrap_or_default(); - // Since this is a SortExec it needs to have an output ordering + let Some(current_plan_ordering) = plan.output_ordering() else { + return internal_err!("SortExec should have output ordering"); + }; let current_plan_ordering_as_req = - RequiredInputOrdering::from(current_plan_ordering).unwrap(); + RequiredInputOrdering::from(current_plan_ordering.clone()).unwrap(); let parent_is_stricter = parent_reqs.as_ref().is_some_and(|parent_req| { plan.equivalence_properties().requirements_compatible( @@ -684,7 +685,7 @@ fn handle_hash_join( } // Collect all unique column indices used in the parent-required sorting expression - let all_indices: HashSet = parent_required + let all_indices: HashSet<_> = parent_required .lex_requirement() .iter() .flat_map(|order| { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7058f95ca6f84..2f2acb059cd07 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -494,7 +494,7 @@ impl AggregateExec { input_eq_properties, &mode, )?; - new_requirement.extend(req.to_vec()); + new_requirement.extend(req); new_requirement = new_requirement.collapse(); // If our aggregation has grouping sets then our base grouping exprs will From d402680ac80d50909d0ab19686ed28d7b2aad9d2 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 24 Mar 2025 00:14:16 +0300 Subject: [PATCH 030/167] Review Part 3 --- .../physical-expr-common/src/sort_expr.rs | 41 +++++++++++-------- .../src/equivalence/properties/mod.rs | 10 ++--- .../src/output_requirements.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 7 ++-- 4 files changed, 30 insertions(+), 30 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 560a8d19551de..c6ed157e0fee2 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -17,19 +17,20 @@ //! Sort expressions -use crate::physical_expr::{fmt_sql, PhysicalExpr}; -use std::fmt; -use std::fmt::{Display, Formatter}; +use std::fmt::{self, Display, Formatter}; use std::hash::{Hash, Hasher}; -use std::ops::{Deref, Index, Range, RangeFrom, RangeTo}; +use std::ops::{Deref, DerefMut, Index, Range, RangeFrom, RangeTo}; use std::sync::{Arc, LazyLock}; use std::vec::IntoIter; +use crate::physical_expr::{fmt_sql, PhysicalExpr}; + use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_expr_common::columnar_value::ColumnarValue; + use itertools::Itertools; /// Represents Sort operation for a column in a RecordBatch @@ -137,7 +138,7 @@ impl AsRef for PhysicalSortExpr { } impl PartialEq for PhysicalSortExpr { - fn eq(&self, other: &PhysicalSortExpr) -> bool { + fn eq(&self, other: &Self) -> bool { self.options == other.options && self.expr.eq(&other.expr) } } @@ -181,13 +182,12 @@ impl PhysicalSortExpr { ) -> bool { // If the column is not nullable, NULLS FIRST/LAST is not important. let nullable = self.expr.nullable(schema).unwrap_or(true); + let opts = &requirement.options; self.expr.eq(&requirement.expr) && if nullable { - requirement.options.is_none_or(|opts| self.options == opts) + opts.is_none_or(|opts| self.options == opts) } else { - requirement - .options - .is_none_or(|opts| self.options.descending == opts.descending) + opts.is_none_or(|opts| self.options.descending == opts.descending) } } } @@ -298,7 +298,7 @@ impl PhysicalSortRequirement { } /// Returns whether this requirement is equal or more specific than `other`. - pub fn compatible(&self, other: &PhysicalSortRequirement) -> bool { + pub fn compatible(&self, other: &Self) -> bool { self.expr.eq(&other.expr) && other .options @@ -411,7 +411,7 @@ impl LexOrdering { } /// Merge the contents of `other` into `self`, removing duplicates. - pub fn merge(mut self, other: LexOrdering) -> Self { + pub fn merge(mut self, other: Self) -> Self { self.inner = self.inner.into_iter().chain(other).unique().collect(); self } @@ -424,7 +424,7 @@ impl LexOrdering { /// If the required ordering is `None` for an entry in `requirement`, the /// default ordering `ASC, NULLS LAST` is used (see /// [`PhysicalSortExpr::from`]). - pub fn from_lex_requirement(requirement: LexRequirement) -> LexOrdering { + pub fn from_lex_requirement(requirement: LexRequirement) -> Self { requirement .into_iter() .map(PhysicalSortExpr::from) @@ -438,7 +438,7 @@ impl LexOrdering { /// /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. pub fn collapse(self) -> Self { - let mut output = LexOrdering::default(); + let mut output = Self::default(); for item in self { if !output.iter().any(|req| req.expr.eq(&item.expr)) { output.push(item); @@ -564,10 +564,6 @@ impl LexRequirement { self.inner.is_empty() } - pub fn to_vec(&self) -> Vec { - self.inner.clone() - } - pub fn iter(&self) -> impl Iterator { self.inner.iter() } @@ -576,7 +572,10 @@ impl LexRequirement { self.inner.push(physical_sort_requirement) } - pub fn extend(&mut self, requirements: impl IntoIterator) { + pub fn extend( + &mut self, + requirements: impl IntoIterator, + ) { self.inner.extend(requirements) } @@ -616,6 +615,12 @@ impl Deref for LexRequirement { } } +impl DerefMut for LexRequirement { + fn deref_mut(&mut self) -> &mut Self::Target { + self.inner.as_mut_slice() + } +} + impl FromIterator for LexRequirement { fn from_iter>(iter: T) -> Self { let mut lex_requirement = Self::default(); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index dd266c6535e54..edd4bb03e4130 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -738,8 +738,8 @@ impl EquivalenceProperties { req1: &LexRequirement, req2: &LexRequirement, ) -> Option { - let mut lhs = self.normalize_sort_requirements(req1).to_vec(); - let mut rhs = self.normalize_sort_requirements(req2).to_vec(); + let mut lhs = self.normalize_sort_requirements(req1); + let mut rhs = self.normalize_sort_requirements(req2); lhs.iter_mut() .zip(rhs.iter_mut()) .all(|(lhs, rhs)| { @@ -757,11 +757,7 @@ impl EquivalenceProperties { (None, None) => true, } }) - .then_some(if lhs.len() >= rhs.len() { - LexRequirement::new(lhs) - } else { - LexRequirement::new(rhs) - }) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) } /// we substitute the ordering according to input expression type, this is a simplified version diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 99ef1966d0fad..66eb45648f031 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -216,7 +216,7 @@ impl ExecutionPlan for OutputRequirementExec { let mut updated_sort_reqs = LexRequirement::new(vec![]); // None or empty_vec can be treated in the same way. if let Some(reqs) = &self.required_input_ordering()[0] { - for req in reqs.lex_requirement().to_vec() { + for req in reqs.lex_requirement().iter() { let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { return Ok(None); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d4d6ae656b209..0fead22facda4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -613,10 +613,9 @@ pub fn get_window_mode( for (should_swap, order_by_reqs) in [(false, order_by_reqs), (true, reverse_order_by_reqs)] { - let req = LexRequirement::new( - [partition_by_reqs.to_vec(), order_by_reqs.to_vec()].concat(), - ) - .collapse(); + let mut req = partition_by_reqs.clone(); + req.extend(order_by_reqs.clone()); + req = req.collapse(); if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { From 807c28ddef37b7806c558663219e3cfb3b43e9a0 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 24 Mar 2025 00:46:53 +0300 Subject: [PATCH 031/167] Review Part 4 --- .../src/enforce_sorting/mod.rs | 19 +++++++++++-------- .../replace_with_order_preserving_variants.rs | 19 +++++++------------ 2 files changed, 18 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 4081e731b468f..32dea9e18967b 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -529,7 +529,7 @@ pub fn ensure_sorting( /// Analyzes if there are any immediate sort removals by checking the `SortExec`s /// and their ordering requirement satisfactions with children -/// If the sort is unnecessary, either replaces it with [`SortPreservingMergeExec`]/`[LimitExec`] +/// If the sort is unnecessary, either replaces it with [`SortPreservingMergeExec`]/`LimitExec` /// or removes the [`SortExec`]. /// Otherwise, returns the original plan fn analyze_immediate_sort_removal( @@ -539,13 +539,16 @@ fn analyze_immediate_sort_removal( return Transformed::no(node); }; let sort_input = sort_exec.input(); - // Check if the sort is unnecessary - if !sort_input.equivalence_properties().ordering_satisfy( - sort_exec - .properties() - .output_ordering() - .unwrap_or(LexOrdering::empty()), - ) { + // Check if the sort is unnecessary: + if !sort_exec + .properties() + .output_ordering() + .is_none_or(|ordering| { + sort_input + .equivalence_properties() + .ordering_satisfy(ordering) + }) + { return Transformed::no(node); }; node.plan = if !sort_exec.preserve_partitioning() diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 30bbea16f8da1..02ba93711fe63 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -25,9 +25,9 @@ use crate::utils::{ is_coalesce_partitions, is_repartition, is_sort, is_sort_preserving_merge, }; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::repartition::RepartitionExec; @@ -35,7 +35,6 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_common::config::ConfigOptions; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its @@ -264,16 +263,12 @@ pub fn replace_with_order_preserving_variants( )?; // If the alternate plan makes this sort unnecessary, accept the alternate: - if alternate_plan - .plan - .equivalence_properties() - .ordering_satisfy( - requirements - .plan - .output_ordering() - .unwrap_or(LexOrdering::empty()), - ) - { + if requirements.plan.output_ordering().is_none_or(|ordering| { + alternate_plan + .plan + .equivalence_properties() + .ordering_satisfy(ordering) + }) { for child in alternate_plan.children.iter_mut() { child.data = false; } From b07b3771d8b96336e3ee8919df9c2bafe9f2e20a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 24 Mar 2025 01:10:22 +0300 Subject: [PATCH 032/167] Review Part 5 --- .../physical-expr-common/src/sort_expr.rs | 8 +--- .../src/update_aggr_exprs.rs | 19 ++++----- .../src/sorts/streaming_merge.rs | 41 ++++--------------- 3 files changed, 18 insertions(+), 50 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index c6ed157e0fee2..264fea0141fcd 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -20,7 +20,7 @@ use std::fmt::{self, Display, Formatter}; use std::hash::{Hash, Hasher}; use std::ops::{Deref, DerefMut, Index, Range, RangeFrom, RangeTo}; -use std::sync::{Arc, LazyLock}; +use std::sync::Arc; use std::vec::IntoIter; use crate::physical_expr::{fmt_sql, PhysicalExpr}; @@ -340,12 +340,6 @@ impl LexOrdering { Self { inner } } - /// Return an empty LexOrdering (no expressions) - pub fn empty() -> &'static LexOrdering { - static EMPTY_ORDER: LazyLock = LazyLock::new(LexOrdering::default); - &EMPTY_ORDER - } - /// Returns the number of elements that can be stored in the LexOrdering /// without reallocating. pub fn capacity(&self) -> usize { diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 6ed0d6dd5e028..90bf131371f54 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -24,10 +24,10 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr::{ reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; -use datafusion_physical_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; use datafusion_physical_plan::{ @@ -159,30 +159,27 @@ fn try_convert_aggregate_if_better( aggr_exprs .into_iter() .map(|aggr_expr| { - let aggr_sort_exprs = aggr_expr.order_bys().unwrap_or(LexOrdering::empty()); - let reverse_aggr_sort_exprs = reverse_order_bys(aggr_sort_exprs); - let aggr_sort_reqs = LexRequirement::from(aggr_sort_exprs.clone()); - let reverse_aggr_req = LexRequirement::from(reverse_aggr_sort_exprs); - + let order_bys = aggr_expr.order_bys(); // If the aggregate expression benefits from input ordering, and // there is an actual ordering enabling this, try to update the // aggregate expression to benefit from the existing ordering. // Otherwise, leave it as is. - if aggr_expr.order_sensitivity().is_beneficial() && !aggr_sort_reqs.is_empty() - { + if !aggr_expr.order_sensitivity().is_beneficial() { + Ok(aggr_expr) + } else if let Some(order_bys) = order_bys { + let aggr_sort_reqs = LexRequirement::from(order_bys.clone()); let reqs = LexRequirement::new(concat_slices( prefix_requirement, &aggr_sort_reqs, )); - let prefix_requirement = LexRequirement::new(prefix_requirement.to_vec()); if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) } else if eq_properties.ordering_satisfy_requirement( &LexRequirement::new(concat_slices( - &prefix_requirement, - &reverse_aggr_req, + &LexRequirement::from_iter(prefix_requirement.iter().cloned()), + &LexRequirement::from(reverse_order_bys(order_bys)), )), ) { // Converting to reverse enables more efficient execution diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index a541f79dc7174..510b9ce9f4e4e 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -52,10 +52,11 @@ macro_rules! merge_helper { }}; } +#[derive(Default)] pub struct StreamingMergeBuilder<'a> { streams: Vec, schema: Option, - expressions: &'a LexOrdering, + expressions: Option<&'a LexOrdering>, metrics: Option, batch_size: Option, fetch: Option, @@ -63,21 +64,6 @@ pub struct StreamingMergeBuilder<'a> { enable_round_robin_tie_breaker: bool, } -impl Default for StreamingMergeBuilder<'_> { - fn default() -> Self { - Self { - streams: vec![], - schema: None, - expressions: LexOrdering::empty(), - metrics: None, - batch_size: None, - fetch: None, - reservation: None, - enable_round_robin_tie_breaker: false, - } - } -} - impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { Self { @@ -97,7 +83,7 @@ impl<'a> StreamingMergeBuilder<'a> { } pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { - self.expressions = expressions; + self.expressions = Some(expressions); self } @@ -145,22 +131,13 @@ impl<'a> StreamingMergeBuilder<'a> { enable_round_robin_tie_breaker, } = self; - // Early return if streams or expressions are empty - let checks = [ - ( - streams.is_empty(), - "Streams cannot be empty for streaming merge", - ), - ( - expressions.is_empty(), - "Sort expressions cannot be empty for streaming merge", - ), - ]; - - if let Some((_, error_message)) = checks.iter().find(|(condition, _)| *condition) - { - return internal_err!("{}", error_message); + // Early return if streams or expressions are empty: + if streams.is_empty() { + return internal_err!("Streams cannot be empty for streaming merge"); } + let Some(expressions) = expressions else { + return internal_err!("Sort expressions cannot be empty for streaming merge"); + }; // Unwrapping mandatory fields let schema = schema.expect("Schema cannot be empty for streaming merge"); From 51f5cd90a86667ea5dd798fe1f3a970f0dce3063 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 24 Mar 2025 01:31:04 +0300 Subject: [PATCH 033/167] Review Part 6 --- datafusion/physical-plan/src/sorts/streaming_merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 510b9ce9f4e4e..3105eab4ae408 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -83,7 +83,7 @@ impl<'a> StreamingMergeBuilder<'a> { } pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { - self.expressions = Some(expressions); + self.expressions = (!expressions.is_empty()).then(|| expressions); self } From 86e69a15de550adfbcca88a0d8a4e962e21ef77e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 24 Mar 2025 21:58:07 +0300 Subject: [PATCH 034/167] Enforce non-degeneracy for LexRequirement --- datafusion/catalog/src/stream.rs | 2 +- .../core/src/datasource/listing/table.rs | 2 +- .../enforce_distribution.rs | 2 +- .../physical_optimizer/enforce_sorting.rs | 4 +- .../physical_optimizer/projection_pushdown.rs | 7 +- .../tests/physical_optimizer/test_utils.rs | 4 +- datafusion/datasource/src/statistics.rs | 5 +- .../physical-expr-common/src/sort_expr.rs | 79 +++--- .../physical-expr/src/equivalence/class.rs | 27 +- .../physical-expr/src/equivalence/mod.rs | 16 +- .../src/equivalence/properties/dependency.rs | 31 +-- .../src/equivalence/properties/mod.rs | 240 ++++++++++++------ .../src/enforce_sorting/sort_pushdown.rs | 42 ++- .../src/output_requirements.rs | 10 +- .../src/update_aggr_exprs.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 40 +-- .../physical-plan/src/execution_plan.rs | 26 +- .../src/joins/sort_merge_join.rs | 4 +- .../src/joins/symmetric_hash_join.rs | 4 +- .../src/sorts/sort_preserving_merge.rs | 2 +- .../src/sorts/streaming_merge.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 84 +++--- datafusion/proto/src/physical_plan/mod.rs | 6 +- .../tests/cases/roundtrip_physical_plan.rs | 6 +- 24 files changed, 352 insertions(+), 295 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index 8bec5268d1f40..a74b690d208c6 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -354,7 +354,7 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, &self.0.order)?; // It is sufficient to pass only one of the equivalent orderings: - let order_requirements = orders.first().and_then(|ordering| { + let order_requirements = orders.first().map(|ordering| { let reqs = ordering.iter().cloned().map(Into::into).collect(); RequiredInputOrdering::new(reqs) }); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 0e3c175127e2b..4678bc0fe4d91 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1044,7 +1044,7 @@ impl TableProvider for ListingTable { let orderings = self.try_create_output_ordering()?; // It is sufficient to pass only one of the equivalent orderings: - let order_requirements = orderings.first().and_then(|ordering| { + let order_requirements = orderings.first().map(|ordering| { let reqs = ordering.iter().cloned().map(Into::into).collect(); RequiredInputOrdering::new(reqs) }); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 54d8d98465994..8860e7ee64c37 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -138,7 +138,7 @@ impl ExecutionPlan for SortRequiredExec { // model that it requires the output ordering of its input fn required_input_ordering(&self) -> Vec> { - vec![RequiredInputOrdering::from(self.expr.clone())] + vec![Some(RequiredInputOrdering::from(self.expr.clone()))] } fn with_new_children( diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 3f8312668a43a..34306280f14a0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -1008,8 +1008,8 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ let output_requirements: Arc = Arc::new(OutputRequirementExec::new( bounded_window2, - RequiredInputOrdering::new(LexRequirement::from(LexOrdering::from( - sort_exprs2, + Some(RequiredInputOrdering::new(LexRequirement::from( + LexOrdering::from(sort_exprs2), ))), Distribution::SinglePartition, )); diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 4d445f2793f0c..32825cc845b2a 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -649,7 +649,7 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv, - RequiredInputOrdering::new(LexRequirement::new(vec![ + Some(RequiredInputOrdering::new(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), options: Some(SortOptions::default()), @@ -662,7 +662,7 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ])), + ]))), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -708,8 +708,7 @@ fn test_output_req_after_projection() -> Result<()> { )), options: Some(SortOptions::default()), }, - ])) - .unwrap(); + ])); assert_eq!( after_optimize .as_any() diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index d28b881f19388..00011c8d6de0a 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -390,9 +390,9 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - vec![RequiredInputOrdering::from( + vec![Some(RequiredInputOrdering::from( self.required_input_ordering.clone(), - )] + ))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 9df5aa993d437..1e5ad18c82d7f 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -115,7 +115,7 @@ impl MinMaxStatistics { projected_schema .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, ); - let min_max_sort_order = LexOrdering::from( + let min_max_sort_order = LexOrdering::from_iter( sort_columns .iter() .zip(projected_sort_order.iter()) @@ -123,8 +123,7 @@ impl MinMaxStatistics { .map(|(i, (col, sort))| PhysicalSortExpr { expr: Arc::new(Column::new(col.name(), i)), options: sort.options, - }) - .collect::>(), + }), ); let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 264fea0141fcd..3cebe48fd5bbd 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -190,6 +190,18 @@ impl PhysicalSortExpr { opts.is_none_or(|opts| self.options.descending == opts.descending) } } + + /// Checks whether this sort expression satisfies the given `sort_expr`. + pub fn satisfy_expr(&self, sort_expr: &PhysicalSortExpr, schema: &Schema) -> bool { + // If the column is not nullable, NULLS FIRST/LAST is not important. + let nullable = self.expr.nullable(schema).unwrap_or(true); + self.expr.eq(&sort_expr.expr) + && if nullable { + self.options == sort_expr.options + } else { + self.options.descending == sort_expr.options.descending + } + } } /// Represents sort requirement associated with a plan @@ -410,21 +422,6 @@ impl LexOrdering { self } - /// Converts a `LexRequirement` into a `LexOrdering`. - /// - /// This function converts [`PhysicalSortRequirement`] to [`PhysicalSortExpr`] - /// for each entry in the input. - /// - /// If the required ordering is `None` for an entry in `requirement`, the - /// default ordering `ASC, NULLS LAST` is used (see - /// [`PhysicalSortExpr::from`]). - pub fn from_lex_requirement(requirement: LexRequirement) -> Self { - requirement - .into_iter() - .map(PhysicalSortExpr::from) - .collect() - } - /// Collapse a `LexOrdering` into a new duplicate-free `LexOrdering` based on expression. /// /// This function filters duplicate entries that have same physical @@ -457,12 +454,6 @@ impl From> for LexOrdering { } } -impl From for LexOrdering { - fn from(value: LexRequirement) -> Self { - value.into_iter().map(Into::into).collect() - } -} - /// Convert a `LexOrdering` into a `Arc[]` for fast copies impl From for Arc<[PhysicalSortExpr]> { fn from(value: LexOrdering) -> Self { @@ -478,6 +469,12 @@ impl Deref for LexOrdering { } } +impl DerefMut for LexOrdering { + fn deref_mut(&mut self) -> &mut Self::Target { + self.inner.as_mut_slice() + } +} + impl Display for LexOrdering { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut first = true; @@ -495,9 +492,7 @@ impl Display for LexOrdering { impl FromIterator for LexOrdering { fn from_iter>(iter: T) -> Self { - let mut lex_ordering = Self::default(); - lex_ordering.extend(iter); - lex_ordering + Self::new(iter.into_iter().collect()) } } @@ -542,15 +537,25 @@ impl IntoIterator for LexOrdering { } } +impl<'a> IntoIterator for &'a LexOrdering { + type Item = &'a PhysicalSortExpr; + type IntoIter = std::slice::Iter<'a, PhysicalSortExpr>; + + fn into_iter(self) -> Self::IntoIter { + self.inner.iter() + } +} + ///`LexRequirement` is an struct containing a `Vec`, which /// represents a lexicographical ordering requirement. -#[derive(Debug, Default, Clone, PartialEq)] +#[derive(Debug, Clone, PartialEq)] pub struct LexRequirement { inner: Vec, } impl LexRequirement { pub fn new(inner: Vec) -> Self { + debug_assert!(!inner.is_empty()); Self { inner } } @@ -595,12 +600,6 @@ impl From> for LexRequirement { } } -impl From for LexRequirement { - fn from(value: LexOrdering) -> Self { - Self::new(value.into_iter().map(Into::into).collect()) - } -} - impl Deref for LexRequirement { type Target = [PhysicalSortRequirement]; @@ -617,9 +616,7 @@ impl DerefMut for LexRequirement { impl FromIterator for LexRequirement { fn from_iter>(iter: T) -> Self { - let mut lex_requirement = Self::default(); - lex_requirement.extend(iter); - lex_requirement + Self::new(iter.into_iter().collect()) } } @@ -632,11 +629,15 @@ impl IntoIterator for LexRequirement { } } -impl<'a> IntoIterator for &'a LexOrdering { - type Item = &'a PhysicalSortExpr; - type IntoIter = std::slice::Iter<'a, PhysicalSortExpr>; +// Cross-conversions utilities between `LexOrdering` and `LexRequirement` +impl From for LexRequirement { + fn from(value: LexOrdering) -> Self { + Self::new(value.into_iter().map(Into::into).collect()) + } +} - fn into_iter(self) -> Self::IntoIter { - self.inner.iter() +impl From for LexOrdering { + fn from(value: LexRequirement) -> Self { + value.into_iter().map(Into::into).collect() } } diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 13a3c79a47a2f..4f022bf0a7e93 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -513,12 +513,13 @@ impl EquivalenceGroup { /// expressions in `sort_exprs` and returns the corresponding normalized /// sort expressions. pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { - // Convert sort expressions to sort requirements: - let sort_reqs = LexRequirement::from(sort_exprs.clone()); - // Normalize the requirements: - let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); - // Convert sort requirements back to sort expressions: - LexOrdering::from(normalized_sort_reqs) + LexOrdering::new( + sort_exprs + .iter() + .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) + .collect(), + ) + .collapse() } /// This function applies the `normalize_sort_requirement` function for all @@ -527,14 +528,12 @@ impl EquivalenceGroup { pub fn normalize_sort_requirements( &self, sort_reqs: &LexRequirement, - ) -> LexRequirement { - LexRequirement::new( - sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect(), - ) - .collapse() + ) -> Option { + let reqs = sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect::>(); + (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) } /// Projects `expr` according to the given projection mapping. diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index e94d2bad57126..c568fab51bd3d 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use crate::expressions::Column; -use crate::{LexRequirement, PhysicalExpr}; +use crate::PhysicalExpr; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -34,18 +34,6 @@ pub use properties::{ calculate_union, join_equivalence_properties, EquivalenceProperties, }; -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out -/// duplicate entries that have same physical expression inside. For example, -/// `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a Some(ASC)]`. -/// -/// It will also filter out entries that are ordered if the next entry is; -/// for instance, `vec![floor(a) Some(ASC), a Some(ASC)]` will be collapsed to -/// `vec![a Some(ASC)]`. -#[deprecated(since = "45.0.0", note = "Use LexRequirement::collapse")] -pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { - input.collapse() -} - /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. pub fn add_offset_to_expr( @@ -70,7 +58,7 @@ mod tests { use super::*; use crate::expressions::col; - use crate::PhysicalSortExpr; + use crate::{LexRequirement, PhysicalSortExpr}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 055230889cba8..03f058a006876 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1043,7 +1043,7 @@ mod tests { let expected_normalized = convert_to_sort_reqs(&expected_normalized); assert_eq!( - eq_properties.normalize_sort_requirements(&req), + eq_properties.normalize_sort_requirements(&req).unwrap(), expected_normalized ); } @@ -1077,7 +1077,7 @@ mod tests { let reqs = convert_to_sort_reqs(&reqs); let expected = convert_to_sort_reqs(&expected); - let normalized = eq_properties.normalize_sort_requirements(&reqs); + let normalized = eq_properties.normalize_sort_requirements(&reqs).unwrap(); assert!( expected.eq(&normalized), "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" @@ -1352,23 +1352,6 @@ mod tests { Ok(()) } - #[test] - fn test_ordering_equivalence_with_empty_requirements() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); - - let col_a = col("a", &schema)?; - - let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default())); - - eq_properties.add_new_ordering(LexOrdering::from(vec![ - PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), - ])); - assert!(eq_properties.ordering_satisfy_requirement(&LexRequirement::default())); - - Ok(()) - } - #[test] fn test_requirements_compatible() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1381,7 +1364,6 @@ mod tests { let col_c = col("c", &schema)?; let eq_properties = EquivalenceProperties::new(schema); - let default_lex = LexRequirement::default(); let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { expr: Arc::clone(&col_a), options: None, @@ -1401,15 +1383,6 @@ mod tests { options: None, }]); - let res = eq_properties.requirements_compatible(&default_lex, &default_lex); - assert!(res); - - let res = eq_properties.requirements_compatible(&lex_a, &default_lex); - assert!(res); - - let res = eq_properties.requirements_compatible(&default_lex, &lex_a); - assert!(!res); - let res = eq_properties.requirements_compatible(&lex_a, &lex_a); assert!(res); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index edd4bb03e4130..a8c29ca232d19 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -499,12 +499,21 @@ impl EquivalenceProperties { /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { - // Convert sort expressions to sort requirements: - let sort_reqs = LexRequirement::from(sort_exprs.clone()); - // Normalize the requirements: - let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); - // Convert sort requirements back to sort expressions: - LexOrdering::from(normalized_sort_reqs) + let normalized_sort_exprs = self.eq_group.normalize_sort_exprs(sort_exprs); + let mut constant_exprs = vec![]; + constant_exprs.extend( + self.constants + .iter() + .map(|const_expr| Arc::clone(const_expr.expr())), + ); + let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); + // Prune redundant sections in the ordering: + normalized_sort_exprs + .iter() + .filter(|&order| !physical_exprs_contains(&constants_normalized, &order.expr)) + .cloned() + .collect::() + .collapse() } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the @@ -520,8 +529,11 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - fn normalize_sort_requirements(&self, sort_reqs: &LexRequirement) -> LexRequirement { - let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); + fn normalize_sort_requirements( + &self, + sort_reqs: &LexRequirement, + ) -> Option { + let normalized_reqs = self.eq_group.normalize_sort_requirements(sort_reqs)?; let mut constant_exprs = vec![]; constant_exprs.extend( self.constants @@ -530,37 +542,95 @@ impl EquivalenceProperties { ); let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); // Prune redundant sections in the requirement: - normalized_sort_reqs - .iter() - .filter(|&order| !physical_exprs_contains(&constants_normalized, &order.expr)) - .cloned() - .collect::() - .collapse() + let reqs = normalized_reqs + .into_iter() + .filter(|order| !physical_exprs_contains(&constants_normalized, &order.expr)) + .collect::>(); + (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) } /// Checks whether the given ordering is satisfied by any of the existing /// orderings. pub fn ordering_satisfy(&self, given: &LexOrdering) -> bool { - // Convert the given sort expressions to sort requirements: - let sort_requirements = LexRequirement::from(given.clone()); - self.ordering_satisfy_requirement(&sort_requirements) + // First, standardize the given ordering: + let normalized_ordering = self.normalize_sort_exprs(given); + // Check whether given ordering is satisfied by constraints first: + if self.satisfied_by_constraints_ordering(&normalized_ordering) { + return true; + } + let schema = self.schema(); + let mut eq_properties = self.clone(); + for element in normalized_ordering { + // Check whether given ordering is satisfied: + let ExprProperties { + sort_properties, .. + } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); + let satisfy = match sort_properties { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { + expr: Arc::clone(&element.expr), + options, + }; + sort_expr.satisfy_expr(&element, schema) + } + // Singleton expressions satisfies any ordering. + SortProperties::Singleton => true, + SortProperties::Unordered => false, + }; + if !satisfy { + return false; + } + // Treat satisfied keys as constants in subsequent iterations. We + // can do this because the "next" key only matters in a lexicographical + // ordering when the keys to its left have the same values. + // + // Note that these expressions are not properly "constants". This is just + // an implementation strategy confined to this function. + // + // For example, assume that the requirement is `[a ASC, (b + c) ASC]`, + // and existing equivalent orderings are `[a ASC, b ASC]` and `[c ASC]`. + // From the analysis above, we know that `[a ASC]` is satisfied. Then, + // we add column `a` as constant to the algorithm state. This enables us + // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. + eq_properties = eq_properties + .with_constants(std::iter::once(ConstExpr::from(element.expr))); + } + true } /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - pub fn ordering_satisfy_requirement(&self, reqs: &LexRequirement) -> bool { - let mut eq_properties = self.clone(); + pub fn ordering_satisfy_requirement(&self, given: &LexRequirement) -> bool { // First, standardize the given requirement: - let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); - - // Check whether given ordering is satisfied by constraints first + let Some(normalized_reqs) = self.normalize_sort_requirements(given) else { + // If the requirement vanishes after normalization, it is satisfied + // by any ordering. + return true; + }; + // Check whether given requirement is satisfied by constraints first: if self.satisfied_by_constraints(&normalized_reqs) { return true; } - - for normalized_req in normalized_reqs { - // Check whether given ordering is satisfied - if !eq_properties.ordering_satisfy_single(&normalized_req) { + let schema = self.schema(); + let mut eq_properties = self.clone(); + for element in normalized_reqs { + // Check whether given requirement is satisfied: + let ExprProperties { + sort_properties, .. + } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); + let satisfy = match sort_properties { + SortProperties::Ordered(options) => { + let sort_expr = PhysicalSortExpr { + expr: Arc::clone(&element.expr), + options, + }; + sort_expr.satisfy(&element, schema) + } + // Singleton expressions satisfies any requirement. + SortProperties::Singleton => true, + SortProperties::Unordered => false, + }; + if !satisfy { return false; } // Treat satisfied keys as constants in subsequent iterations. We @@ -576,11 +646,60 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. eq_properties = eq_properties - .with_constants(std::iter::once(ConstExpr::from(normalized_req.expr))); + .with_constants(std::iter::once(ConstExpr::from(element.expr))); } true } + fn satisfied_by_constraints_ordering( + &self, + normalized_exprs: &[PhysicalSortExpr], + ) -> bool { + self.constraints.iter().any(|constraint| match constraint { + Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { + let check_null = matches!(constraint, Constraint::Unique(_)); + indices.len() <= normalized_exprs.len() + && self.oeq_class.iter().any(|ordering| { + if indices.len() > ordering.len() { + return false; + } + // Build a map of column positions in the ordering: + let mut col_positions = HashMap::with_capacity(ordering.len()); + for (pos, req) in ordering.iter().enumerate() { + if let Some(col) = req.expr.as_any().downcast_ref::() + { + let nullable = col.nullable(&self.schema).unwrap_or(true); + col_positions.insert(col.index(), (pos, nullable)); + } + } + // Check if all constraint indices appear in valid positions: + if !indices.iter().all(|&idx| { + col_positions + .get(&idx) + .map(|&(pos, nullable)| { + // For unique constraints, verify column is not nullable if it's first/last: + !check_null + || !nullable + || (pos != 0 && pos != ordering.len() - 1) + }) + .unwrap_or(false) + }) { + return false; + } + // Check if this ordering matches the prefix: + let ordering_len = ordering.len(); + normalized_exprs.len() >= ordering_len + && normalized_exprs[..ordering_len].iter().zip(ordering).all( + |(req, existing)| { + req.expr.eq(&existing.expr) + && req.options == existing.options + }, + ) + }) + } + }) + } + /// Checks if the sort requirements are satisfied by any of the table constraints (primary key or unique). /// Returns true if any constraint fully satisfies the requirements. fn satisfied_by_constraints( @@ -617,7 +736,7 @@ impl EquivalenceProperties { return false; } - // Build a map of column positions in the ordering + // Build a map of column positions in the requirement let mut col_positions = HashMap::with_capacity(ordering.len()); for (pos, req) in ordering.iter().enumerate() { if let Some(col) = req.expr.as_any().downcast_ref::() { @@ -643,7 +762,7 @@ impl EquivalenceProperties { return false; } - // Check if this ordering matches requirements prefix + // Check if this ordering matches the prefix let ordering_len = ordering.len(); normalized_reqs.len() >= ordering_len && normalized_reqs[..ordering_len].iter().zip(ordering).all( @@ -657,36 +776,6 @@ impl EquivalenceProperties { }) } - /// Determines whether the ordering specified by the given sort requirement - /// is satisfied based on the orderings within, equivalence classes, and - /// constant expressions. - /// - /// # Parameters - /// - /// - `req`: A reference to a `PhysicalSortRequirement` for which the ordering - /// satisfaction check will be done. - /// - /// # Returns - /// - /// Returns `true` if the specified ordering is satisfied, `false` otherwise. - fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { - let ExprProperties { - sort_properties, .. - } = self.get_expr_properties(Arc::clone(&req.expr)); - match sort_properties { - SortProperties::Ordered(options) => { - let sort_expr = PhysicalSortExpr { - expr: Arc::clone(&req.expr), - options, - }; - sort_expr.satisfy(req, self.schema()) - } - // Singleton expressions satisfies any ordering. - SortProperties::Singleton => true, - SortProperties::Unordered => false, - } - } - /// Checks whether the `given` sort requirements are equal or more specific /// than the `reference` sort requirements. pub fn requirements_compatible( @@ -694,8 +783,13 @@ impl EquivalenceProperties { given: &LexRequirement, reference: &LexRequirement, ) -> bool { - let normalized_given = self.normalize_sort_requirements(given); - let normalized_reference = self.normalize_sort_requirements(reference); + let Some(normalized_given) = self.normalize_sort_requirements(given) else { + return true; + }; + let Some(normalized_reference) = self.normalize_sort_requirements(reference) + else { + return true; + }; (normalized_reference.len() <= normalized_given.len()) && normalized_reference @@ -717,12 +811,12 @@ impl EquivalenceProperties { lhs: &LexOrdering, rhs: &LexOrdering, ) -> Option { - // Convert the given sort expressions to sort requirements: - let lhs = LexRequirement::from(lhs.clone()); - let rhs = LexRequirement::from(rhs.clone()); - let finer = self.get_finer_requirement(&lhs, &rhs); - // Convert the chosen sort requirements back to sort expressions: - finer.map(LexOrdering::from) + let mut lhs = self.normalize_sort_exprs(lhs); + let mut rhs = self.normalize_sort_exprs(rhs); + lhs.iter_mut() + .zip(rhs.iter_mut()) + .all(|(lhs, rhs)| lhs.expr.eq(&rhs.expr) && lhs.options == rhs.options) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) } /// Returns the finer ordering among the requirements `lhs` and `rhs`, @@ -735,11 +829,15 @@ impl EquivalenceProperties { /// is the latter. pub fn get_finer_requirement( &self, - req1: &LexRequirement, - req2: &LexRequirement, + lhs: &LexRequirement, + rhs: &LexRequirement, ) -> Option { - let mut lhs = self.normalize_sort_requirements(req1); - let mut rhs = self.normalize_sort_requirements(req2); + let Some(mut rhs) = self.normalize_sort_requirements(rhs) else { + return self.normalize_sort_requirements(lhs); + }; + let Some(mut lhs) = self.normalize_sort_requirements(lhs) else { + return Some(rhs); + }; lhs.iter_mut() .zip(rhs.iter_mut()) .all(|(lhs, rhs)| { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 7b6e86c7b4ce5..3948c762a38db 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -104,7 +104,7 @@ fn pushdown_sorts_helper( return internal_err!("SortExec should have output ordering"); }; let current_plan_ordering_as_req = - RequiredInputOrdering::from(current_plan_ordering.clone()).unwrap(); + RequiredInputOrdering::from(current_plan_ordering.clone()); let parent_is_stricter = parent_reqs.as_ref().is_some_and(|parent_req| { plan.equivalence_properties().requirements_compatible( @@ -246,17 +246,15 @@ fn pushdown_requirement_to_children( .cloned() .unwrap_or(LexOrdering::default()), ); - Ok(sort_req - .filter(|req| { - sort_exec - .properties() - .eq_properties - .requirements_compatible( - parent_required.lex_requirement(), - req.lex_requirement(), - ) - }) - .map(|_| vec![Some(parent_required.clone())])) + sort_exec + .properties() + .eq_properties + .requirements_compatible( + parent_required.lex_requirement(), + sort_req.lex_requirement(), + ) + .then(|| Ok(vec![Some(parent_required.clone())])) + .transpose() } else if plan.fetch().is_some() && plan.supports_limit_pushdown() && plan @@ -264,15 +262,13 @@ fn pushdown_requirement_to_children( .iter() .all(|maintain| *maintain) { - let output_req = LexRequirement::from( - plan.properties() - .output_ordering() - .cloned() - .unwrap_or(LexOrdering::default()), - ); // Push down through operator with fetch when: // - requirement is aligned with output ordering // - it preserves ordering during execution + let Some(ordering) = plan.properties().output_ordering() else { + return Ok(Some(vec![Some(parent_required.clone())])); + }; + let output_req = LexRequirement::from(ordering.clone()); if plan .properties() .eq_properties @@ -475,7 +471,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = RequiredInputOrdering::from(sort_expr.clone()); + let new_req = Some(RequiredInputOrdering::from(sort_expr.clone())); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; @@ -658,11 +654,9 @@ fn handle_custom_pushdown( let result = maintains_input_order .iter() .map(|&maintains_order| { - if maintains_order { + maintains_order.then(|| { RequiredInputOrdering::new(updated_parent_req.clone().into()) - } else { - None - } + }) }) .collect(); @@ -737,7 +731,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - RequiredInputOrdering::new(updated_parent_req.into()), + Some(RequiredInputOrdering::new(updated_parent_req.into())), ])) } else { Ok(None) diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 66eb45648f031..d5701289673ae 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -30,7 +30,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; +use datafusion_physical_expr::{Distribution, PhysicalSortRequirement}; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::projection::{ make_with_child, update_expr, ProjectionExec, @@ -213,7 +213,7 @@ impl ExecutionPlan for OutputRequirementExec { return Ok(None); } - let mut updated_sort_reqs = LexRequirement::new(vec![]); + let mut updated_sort_reqs = vec![]; // None or empty_vec can be treated in the same way. if let Some(reqs) = &self.required_input_ordering()[0] { for req in reqs.lex_requirement().iter() { @@ -247,7 +247,7 @@ impl ExecutionPlan for OutputRequirementExec { .map(|input| { OutputRequirementExec::new( input, - RequiredInputOrdering::new(updated_sort_reqs), + Some(RequiredInputOrdering::new(updated_sort_reqs.into())), dist_req, ) }) @@ -320,7 +320,7 @@ fn require_top_ordering_helper( let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = RequiredInputOrdering::from(req_ordering.clone()); Ok(( - Arc::new(OutputRequirementExec::new(plan, reqs, req_dist)) as _, + Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, true, )) } else if let Some(spm) = plan.as_any().downcast_ref::() { @@ -328,7 +328,7 @@ fn require_top_ordering_helper( Ok(( Arc::new(OutputRequirementExec::new( plan, - reqs, + Some(reqs), Distribution::SinglePartition, )) as _, true, diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 90bf131371f54..838e3a04432b3 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -178,7 +178,7 @@ fn try_convert_aggregate_if_better( aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) } else if eq_properties.ordering_satisfy_requirement( &LexRequirement::new(concat_slices( - &LexRequirement::from_iter(prefix_requirement.iter().cloned()), + prefix_requirement, &LexRequirement::from(reverse_order_bys(order_bys)), )), ) { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2f2acb059cd07..3cb94032dcf26 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -478,15 +478,13 @@ impl AggregateExec { // prefix requirements with this section. In this case, aggregation will // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let mut new_requirement = LexRequirement::new( - indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&groupby_exprs[idx]), - options: None, - }) - .collect::>(), - ); + let mut new_requirements = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: Arc::clone(&groupby_exprs[idx]), + options: None, + }) + .collect::>(); let req = get_finer_aggregate_exprs_requirement( &mut aggr_expr, @@ -494,8 +492,14 @@ impl AggregateExec { input_eq_properties, &mode, )?; - new_requirement.extend(req); - new_requirement = new_requirement.collapse(); + new_requirements.extend(req); + + let required_input_ordering = if new_requirements.is_empty() { + None + } else { + let reqs = LexRequirement::from(new_requirements).collapse(); + RequiredInputOrdering::new_with_alternatives(vec![reqs], true) + }; // If our aggregation has grouping sets then our base grouping exprs will // be expanded based on the flags in `group_by.groups` where for each @@ -522,9 +526,6 @@ impl AggregateExec { let group_expr_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; - let required_input_ordering = - RequiredInputOrdering::new_with_alternatives(vec![new_requirement], true); - let cache = Self::compute_properties( &input, Arc::clone(&schema), @@ -1118,14 +1119,15 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// # Returns /// -/// A `LexRequirement` instance, which is the requirement that satisfies all the -/// aggregate requirements. Returns an error in case of conflicting requirements. +/// A `Result>` instance, which is the requirement +/// that satisfies all the aggregate requirements. Returns an error in case of +/// conflicting requirements. pub fn get_finer_aggregate_exprs_requirement( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, -) -> Result { +) -> Result> { let mut requirement = LexOrdering::default(); for aggr_expr in aggr_exprs.iter_mut() { if let Some(finer_ordering) = @@ -1186,7 +1188,7 @@ pub fn get_finer_aggregate_exprs_requirement( ); } - Ok(LexRequirement::from(requirement)) + Ok(requirement.into_iter().map(Into::into).collect()) } /// Returns physical expressions for arguments to evaluate against a batch. @@ -2418,7 +2420,7 @@ mod tests { &eq_properties, &AggregateMode::Partial, )?; - let res = LexOrdering::from(res); + let res = LexOrdering::from_iter(res.into_iter().map(Into::into)); assert_eq!(res, common_requirement); Ok(()) } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 932ac1ae004d5..947f161273eda 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -1086,31 +1086,39 @@ impl RequiredInputOrdering { /// empty requirements are not allowed inside this type, /// if given [`None`] will be returned pub fn new_with_alternatives( - lex_requirements: Vec, + alternatives: Vec, soft: bool, ) -> Option { - (!(lex_requirements.is_empty() || lex_requirements[0].is_empty())).then(|| { + (!(alternatives.is_empty() || alternatives[0].is_empty())).then(|| { if soft { - Self::Soft(lex_requirements) + Self::Soft(alternatives) } else { - Self::Hard(lex_requirements) + Self::Hard(alternatives) } }) } - pub fn new(requirement: LexRequirement) -> Option { - (!requirement.is_empty()).then(|| Self::Hard(vec![requirement])) + pub fn new(requirement: LexRequirement) -> Self { + debug_assert!(!requirement.is_empty()); + Self::Hard(vec![requirement]) } - pub fn from(ordering: LexOrdering) -> Option { + pub fn from(ordering: LexOrdering) -> Self { Self::new(LexRequirement::from(ordering)) } + pub fn add_alternative(&mut self, requirement: LexRequirement) { + match self { + Self::Hard(alts) => alts.push(requirement), + Self::Soft(alts) => alts.push(requirement), + } + } + /// Returns the first (i.e. most preferred) among alternative requirements. pub fn lex_requirement(&self) -> &LexRequirement { match self { - Self::Hard(lex) => &lex[0], - Self::Soft(lex) => &lex[0], + Self::Hard(alts) => &alts[0], + Self::Soft(alts) => &alts[0], } } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 9882a57b2b2fb..83e94c880a6bf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -418,8 +418,8 @@ impl ExecutionPlan for SortMergeJoinExec { fn required_input_ordering(&self) -> Vec> { vec![ - RequiredInputOrdering::from(self.left_sort_exprs.clone()), - RequiredInputOrdering::from(self.right_sort_exprs.clone()), + Some(RequiredInputOrdering::from(self.left_sort_exprs.clone())), + Some(RequiredInputOrdering::from(self.right_sort_exprs.clone())), ] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 47613b857eb18..a6de0090a28b4 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -439,10 +439,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { vec![ self.left_sort_exprs .as_ref() - .and_then(|e| RequiredInputOrdering::from(e.clone())), + .map(|e| RequiredInputOrdering::from(e.clone())), self.right_sort_exprs .as_ref() - .and_then(|e| RequiredInputOrdering::from(e.clone())), + .map(|e| RequiredInputOrdering::from(e.clone())), ] } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 524e3c37b6770..26455a94b683f 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -242,7 +242,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn required_input_ordering(&self) -> Vec> { - vec![RequiredInputOrdering::from(self.expr.clone())] + vec![Some(RequiredInputOrdering::from(self.expr.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 3105eab4ae408..bf7a3022d171d 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -83,7 +83,7 @@ impl<'a> StreamingMergeBuilder<'a> { } pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { - self.expressions = (!expressions.is_empty()).then(|| expressions); + self.expressions = (!expressions.is_empty()).then_some(expressions); self } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0fead22facda4..fc799dff1c609 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -281,30 +281,36 @@ pub(crate) fn calc_requirements< partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, ) -> Option { - let mut sort_reqs_with_partition = LexRequirement::new( - partition_by_exprs - .into_iter() - .map(|partition_by| { - PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None) - }) - .collect::>(), - ); - let mut sort_reqs = LexRequirement::new(vec![]); + let mut sort_reqs_with_partition = partition_by_exprs + .into_iter() + .map(|partition_by| { + PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None) + }) + .collect::>(); + let mut sort_reqs = vec![]; for element in orderby_sort_exprs.into_iter() { let PhysicalSortExpr { expr, options } = element.borrow(); let sort_req = PhysicalSortRequirement::new(Arc::clone(expr), Some(*options)); if !sort_reqs_with_partition.iter().any(|e| e.expr.eq(expr)) { sort_reqs_with_partition.push(sort_req.clone()); } - if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { + if !sort_reqs + .iter() + .any(|e: &PhysicalSortRequirement| e.expr.eq(expr)) + { sort_reqs.push(sort_req); } } - RequiredInputOrdering::new_with_alternatives( - vec![sort_reqs_with_partition, sort_reqs], - false, - ) + let mut alternatives = vec![]; + if !sort_reqs_with_partition.is_empty() { + alternatives.push(sort_reqs_with_partition.into()); + } + if !sort_reqs.is_empty() { + alternatives.push(sort_reqs.into()); + } + + RequiredInputOrdering::new_with_alternatives(alternatives, false) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -592,31 +598,25 @@ pub fn get_window_mode( ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); - vec![].extend(indices.iter().map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&partitionby_exprs[idx]), - options: None, - })); - let partition_by_reqs = LexRequirement::new( - indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&partitionby_exprs[idx]), - options: None, - }) - .collect(), - ); + let partition_by_reqs = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: Arc::clone(&partitionby_exprs[idx]), + options: None, + }) + .collect::>(); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); - let order_by_reqs = LexRequirement::from(orderby_keys.clone()); - let reverse_order_by_reqs = LexRequirement::from(reverse_order_bys(orderby_keys)); - for (should_swap, order_by_reqs) in - [(false, order_by_reqs), (true, reverse_order_by_reqs)] + let reverse_orderby_keys = reverse_order_bys(orderby_keys); + for (should_swap, orderbys) in [(false, orderby_keys), (true, &reverse_orderby_keys)] { let mut req = partition_by_reqs.clone(); - req.extend(order_by_reqs.clone()); - req = req.collapse(); - if partition_by_eqs.ordering_satisfy_requirement(&req) { + req.extend(orderbys.iter().cloned().map(Into::into)); + if req.is_empty() + || partition_by_eqs + .ordering_satisfy_requirement(&LexRequirement::new(req).collapse()) + { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { InputOrderMode::Sorted @@ -778,24 +778,20 @@ mod tests { let mut expected: Option = None; for expected_param in expected_params.clone() { - let mut lex_requirement = LexRequirement::new(vec![]); + let mut requirements = vec![]; for (col_name, reqs) in expected_param { let options = reqs.map(|(descending, nulls_first)| SortOptions { descending, nulls_first, }); let expr = col(col_name, &schema)?; - let res = PhysicalSortRequirement::new(expr, options); - lex_requirement.push(res); + requirements.push(PhysicalSortRequirement::new(expr, options)); } - if !lex_requirement.is_empty() { - expected = if let Some(expect) = expected { - RequiredInputOrdering::new_with_alternatives( - vec![expect.lex_requirement().clone(), lex_requirement], - false, - ) + if !requirements.is_empty() { + if let Some(alts) = expected.as_mut() { + alts.add_alternative(requirements.into()); } else { - RequiredInputOrdering::new(lex_requirement) + expected = Some(RequiredInputOrdering::new(requirements.into())); } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 1fffc9d369dc0..2b8bacc149d34 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1094,7 +1094,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.and_then(RequiredInputOrdering::new), + sort_order.map(RequiredInputOrdering::new), ))) } PhysicalPlanType::CsvSink(sink) => { @@ -1123,7 +1123,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.and_then(RequiredInputOrdering::new), + sort_order.map(RequiredInputOrdering::new), ))) } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] @@ -1159,7 +1159,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.and_then(RequiredInputOrdering::new), + sort_order.map(RequiredInputOrdering::new), ))) } #[cfg(not(feature = "parquet"))] diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 8e1fd119bf3e0..829ec1698aebf 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -1320,7 +1320,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - RequiredInputOrdering::new(sort_order), + Some(RequiredInputOrdering::new(sort_order)), ))) } @@ -1359,7 +1359,7 @@ fn roundtrip_csv_sink() -> Result<()> { let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; let roundtrip_plan = roundtrip_test_and_return( - Arc::new(DataSinkExec::new(input, data_sink, sort_order)), + Arc::new(DataSinkExec::new(input, data_sink, Some(sort_order))), &ctx, &codec, ) @@ -1415,7 +1415,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - RequiredInputOrdering::new(sort_order), + Some(RequiredInputOrdering::new(sort_order)), ))) } From bade16da5999f6810becda0d5a9e0146a7381d81 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 27 Mar 2025 23:16:35 +0300 Subject: [PATCH 035/167] Enforce non-degeneracy for LexOrdering (Part 1) --- .../examples/function_factory.rs | 4 - datafusion/core/src/datasource/memory.rs | 17 +- datafusion/core/src/physical_planner.rs | 30 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 5 +- .../fuzz_cases/equivalence/properties.rs | 6 +- .../sort_preserving_repartition_fuzz.rs | 11 +- .../user_defined_scalar_functions.rs | 4 - datafusion/datasource/src/file_scan_config.rs | 3 +- datafusion/expr/src/test/function_stub.rs | 13 +- .../src/accumulator.rs | 4 +- .../functions-aggregate/benches/count.rs | 3 +- datafusion/functions-aggregate/benches/sum.rs | 3 +- .../functions-aggregate/src/array_agg.rs | 80 ++-- .../functions-aggregate/src/bool_and_or.rs | 8 - datafusion/functions-aggregate/src/count.rs | 4 - .../functions-aggregate/src/first_last.rs | 411 ++++++++++-------- datafusion/functions-aggregate/src/median.rs | 4 - datafusion/functions-aggregate/src/min_max.rs | 8 - .../functions-aggregate/src/nth_value.rs | 253 ++++++++--- datafusion/functions-aggregate/src/stddev.rs | 5 +- datafusion/functions-aggregate/src/sum.rs | 4 - .../functions/src/datetime/date_part.rs | 1 + datafusion/functions/src/datetime/to_char.rs | 1 + .../physical-expr-common/src/sort_expr.rs | 32 +- datafusion/physical-expr/src/aggregate.rs | 40 +- .../physical-expr/src/equivalence/mod.rs | 48 +- .../physical-expr/src/equivalence/ordering.rs | 4 +- .../src/equivalence/projection.rs | 6 +- .../src/equivalence/properties/dependency.rs | 8 +- .../src/equivalence/properties/mod.rs | 22 +- datafusion/physical-expr/src/planner.rs | 2 +- .../physical-expr/src/window/standard.rs | 2 +- .../src/enforce_distribution.rs | 13 +- .../src/enforce_sorting/sort_pushdown.rs | 18 +- .../physical-plan/benches/partial_ordering.rs | 23 +- .../physical-plan/src/aggregates/mod.rs | 7 +- .../physical-plan/src/aggregates/order/mod.rs | 13 +- .../src/aggregates/order/partial.rs | 41 +- .../physical-plan/src/aggregates/row_hash.rs | 13 +- .../physical-plan/src/repartition/mod.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 7 +- datafusion/physical-plan/src/windows/mod.rs | 2 +- .../src/windows/window_agg_exec.rs | 8 +- .../proto/src/physical_plan/from_proto.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 8 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- 46 files changed, 609 insertions(+), 596 deletions(-) diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index 06367f5c09e36..c4223bee025cc 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -150,10 +150,6 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { Ok(ExprSimplifyResult::Simplified(replacement)) } - fn aliases(&self) -> &[String] { - &[] - } - fn output_ordering(&self, _input: &[ExprProperties]) -> Result { Ok(SortProperties::Unordered) } diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 982cb79d9786b..0052869aa32cd 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -239,16 +239,13 @@ impl TableProvider for MemTable { if !sort_order.is_empty() { let df_schema = DFSchema::try_from(self.schema.as_ref().clone())?; - let file_sort_order = sort_order - .iter() - .map(|sort_exprs| { - create_physical_sort_exprs( - sort_exprs, - &df_schema, - state.execution_props(), - ) - }) - .collect::>>()?; + let eqp = state.execution_props(); + let mut file_sort_order = vec![]; + for sort_exprs in sort_order.iter() { + file_sort_order.push( + create_physical_sort_exprs(sort_exprs, &df_schema, eqp)?.into(), + ); + } source = source.try_with_sort_information(file_sort_order)?; } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index bc3c1d0ac99b2..569d14d2c7161 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -828,7 +828,7 @@ impl DefaultPhysicalPlanner { session_state.execution_props(), )?; let new_sort = - SortExec::new(sort_expr, physical_input).with_fetch(*fetch); + SortExec::new(sort_expr.into(), physical_input).with_fetch(*fetch); Arc::new(new_sort) } LogicalPlan::Subquery(_) => todo!(), @@ -1552,7 +1552,7 @@ pub fn create_window_expr_with_name( name, &physical_args, &partition_by, - order_by.as_ref(), + &order_by.into(), window_frame, physical_schema, ignore_nulls, @@ -1626,21 +1626,21 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( == NullTreatment::IgnoreNulls; let (agg_expr, filter, order_by) = { - let physical_sort_exprs = match order_by { - Some(exprs) => Some(create_physical_sort_exprs( - exprs, - logical_input_schema, - execution_props, - )?), + let ordering = match order_by { + Some(exprs) => Some( + create_physical_sort_exprs( + exprs, + logical_input_schema, + execution_props, + )? + .into(), + ), None => None, }; - let ordering_reqs: LexOrdering = - physical_sort_exprs.clone().unwrap_or_default(); - let agg_expr = AggregateExprBuilder::new(func.to_owned(), physical_args.to_vec()) - .order_by(ordering_reqs) + .order_by(ordering.clone()) .schema(Arc::new(physical_input_schema.to_owned())) .alias(name) .human_display(human_displan) @@ -1649,7 +1649,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( .build() .map(Arc::new)?; - (agg_expr, filter, physical_sort_exprs) + (agg_expr, filter, ordering) }; Ok((agg_expr, filter, order_by)) @@ -1713,11 +1713,11 @@ pub fn create_physical_sort_exprs( exprs: &[SortExpr], input_dfschema: &DFSchema, execution_props: &ExecutionProps, -) -> Result { +) -> Result> { exprs .iter() .map(|expr| create_physical_sort_expr(expr, input_dfschema, execution_props)) - .collect::>() + .collect() } impl DefaultPhysicalPlanner { diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 46221acfcc9b1..13f72b147b924 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -45,7 +45,6 @@ use datafusion_common_runtime::JoinSet; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; @@ -311,7 +310,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = LexOrdering::default(); + let mut sort_keys = vec![]; for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), @@ -331,7 +330,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str let running_source = Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None) .unwrap() - .try_with_sort_information(vec![sort_keys]) + .try_with_sort_information(vec![sort_keys.into()]) .unwrap(), ))); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 593e1c6c2dca0..93c6ce603eb1a 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -24,7 +24,7 @@ use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr::{PhysicalExprRef, ScalarFunctionExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use itertools::Itertools; use std::sync::Arc; @@ -77,7 +77,7 @@ fn test_find_longest_permutation_random() -> Result<()> { expr: Arc::clone(&exprs[idx]), options: sort_expr.options, }) - .collect::(); + .collect::>(); assert_eq!( ordering, ordering2, "indices and lexicographical ordering do not match" @@ -92,7 +92,7 @@ fn test_find_longest_permutation_random() -> Result<()> { // that result will be same after sort (e.g sort was unnecessary). assert!( is_table_same_after_sort( - ordering.clone(), + ordering.into(), table_data_with_properties.clone(), )?, "{}", diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 06b93d41af362..75ccce6fb5aff 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -23,6 +23,8 @@ mod sp_repartition_fuzz_tests { use arrow::compute::{concat_batches, lexsort, SortColumn, SortOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion::datasource::memory::MemorySourceConfig; + use datafusion::datasource::source::DataSourceExec; use datafusion::physical_plan::{ collect, metrics::{BaselineMetrics, ExecutionPlanMetricsSet}, @@ -44,9 +46,6 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; - use datafusion::datasource::memory::MemorySourceConfig; - use datafusion::datasource::source::DataSourceExec; - use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -347,7 +346,7 @@ mod sp_repartition_fuzz_tests { let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = LexOrdering::default(); + let mut sort_keys = vec![]; for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), @@ -360,7 +359,7 @@ mod sp_repartition_fuzz_tests { let running_source = Arc::new( MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None) .unwrap() - .try_with_sort_information(vec![sort_keys.clone()]) + .try_with_sort_information(vec![sort_keys.clone().into()]) .unwrap(), ); let running_source = Arc::new(DataSourceExec::new(running_source)); @@ -381,7 +380,7 @@ mod sp_repartition_fuzz_tests { sort_preserving_repartition_exec_hash(intermediate, hash_exprs.clone()) }; - let final_plan = sort_preserving_merge_exec(sort_keys.clone(), intermediate); + let final_plan = sort_preserving_merge_exec(sort_keys, intermediate); let task_ctx = ctx.task_ctx(); let collected_running = collect(final_plan, task_ctx.clone()).await.unwrap(); diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 264bd6b66a600..2872cca8f6cee 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -967,10 +967,6 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { Ok(ExprSimplifyResult::Simplified(replacement)) } - - fn aliases(&self) -> &[String] { - &[] - } } impl ScalarFunctionWrapper { diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 91b5f01577392..b18d7a76d04a2 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -996,7 +996,7 @@ fn get_projected_output_ordering( ) -> Vec { let mut all_orderings = vec![]; for output_ordering in &base_config.output_ordering { - let mut new_ordering = LexOrdering::default(); + let mut new_ordering = vec![]; for PhysicalSortExpr { expr, options } in output_ordering.iter() { if let Some(col) = expr.as_any().downcast_ref::() { let name = col.name(); @@ -1020,6 +1020,7 @@ fn get_projected_output_ordering( continue; } + let new_ordering = LexOrdering::from(new_ordering); // Check if any file groups are not sorted if base_config.file_groups.iter().any(|group| { if group.len() <= 1 { diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index a753f4c376c63..b7fb30383f1e1 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -179,10 +179,6 @@ impl AggregateUDFImpl for Sum { unreachable!("stub should not have state_fields()") } - fn aliases(&self) -> &[String] { - &[] - } - fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { false } @@ -344,10 +340,6 @@ impl AggregateUDFImpl for Min { not_impl_err!("no impl for stub") } - fn aliases(&self) -> &[String] { - &[] - } - fn create_groups_accumulator( &self, _args: AccumulatorArgs, @@ -429,10 +421,6 @@ impl AggregateUDFImpl for Max { not_impl_err!("no impl for stub") } - fn aliases(&self) -> &[String] { - &[] - } - fn create_groups_accumulator( &self, _args: AccumulatorArgs, @@ -494,6 +482,7 @@ impl AggregateUDFImpl for Avg { fn state_fields(&self, _args: StateFieldsArgs) -> Result> { not_impl_err!("no impl for stub") } + fn aliases(&self) -> &[String] { &self.aliases } diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index a230bb0289091..b9e4451f05002 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -50,9 +50,7 @@ pub struct AccumulatorArgs<'a> { /// ```sql /// SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; /// ``` - /// - /// If no `ORDER BY` is specified, `ordering_req` will be empty. - pub ordering_req: &'a LexOrdering, + pub ordering_req: Option<&'a LexOrdering>, /// Whether the aggregation is running in reverse order pub is_reversed: bool, diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 8bde7d04c44d9..44f362444bd21 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -22,7 +22,6 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; fn prepare_accumulator() -> Box { @@ -31,7 +30,7 @@ fn prepare_accumulator() -> Box { return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, - ordering_req: &LexOrdering::default(), + ordering_req: None, is_reversed: false, name: "COUNT(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index fab53ae94b25d..44aa932b2f843 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -22,7 +22,6 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { @@ -31,7 +30,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box { return_type: data_type, schema: &schema, ignore_nulls: false, - ordering_req: &LexOrdering::default(), + ordering_req: None, is_reversed: false, name: "SUM(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 0f12ac34bfd2c..e1686a5c5bb77 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -17,24 +17,25 @@ //! `ARRAY_AGG` aggregate implementation: [`ArrayAgg`] +use std::collections::{HashSet, VecDeque}; +use std::mem::{size_of, size_of_val}; +use std::sync::Arc; + use arrow::array::{new_empty_array, Array, ArrayRef, AsArray, ListArray, StructArray}; use arrow::datatypes::{DataType, Field, Fields}; use datafusion_common::cast::as_list_array; use datafusion_common::utils::{get_row_at_idx, SingleRowListArrayBuilder}; -use datafusion_common::{exec_err, ScalarValue}; -use datafusion_common::{internal_err, Result}; +use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; -use datafusion_expr::{Accumulator, Signature, Volatility}; -use datafusion_expr::{AggregateUDFImpl, Documentation}; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, Documentation, Signature, Volatility, +}; use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; use datafusion_functions_aggregate_common::utils::ordering_fields; use datafusion_macros::user_doc; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use std::collections::{HashSet, VecDeque}; -use std::mem::{size_of, size_of_val}; -use std::sync::Arc; make_udaf_expr_and_func!( ArrayAgg, @@ -81,10 +82,6 @@ impl AggregateUDFImpl for ArrayAgg { "array_agg" } - fn aliases(&self) -> &[String] { - &[] - } - fn signature(&self) -> &Signature { &self.signature } @@ -134,12 +131,11 @@ impl AggregateUDFImpl for ArrayAgg { return Ok(Box::new(DistinctArrayAggAccumulator::try_new(&data_type)?)); } - if acc_args.ordering_req.is_empty() { + let Some(ordering_req) = acc_args.ordering_req else { return Ok(Box::new(ArrayAggAccumulator::try_new(&data_type)?)); - } + }; - let ordering_dtypes = acc_args - .ordering_req + let ordering_dtypes = ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -147,7 +143,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - acc_args.ordering_req.clone(), + ordering_req.clone(), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) @@ -425,6 +421,31 @@ impl OrderSensitiveArrayAggAccumulator { reverse, }) } + + fn evaluate_orderings(&self) -> Result { + let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); + let num_columns = fields.len(); + let struct_field = Fields::from(fields.clone()); + + let mut column_wise_ordering_values = vec![]; + for i in 0..num_columns { + let column_values = self + .ordering_values + .iter() + .map(|x| x[i].clone()) + .collect::>(); + let array = if column_values.is_empty() { + new_empty_array(fields[i].data_type()) + } else { + ScalarValue::iter_to_array(column_values.into_iter())? + }; + column_wise_ordering_values.push(array); + } + + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; + Ok(SingleRowListArrayBuilder::new(Arc::new(ordering_array)).build_list_scalar()) + } } impl Accumulator for OrderSensitiveArrayAggAccumulator { @@ -568,33 +589,6 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } } -impl OrderSensitiveArrayAggAccumulator { - fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); - let num_columns = fields.len(); - let struct_field = Fields::from(fields.clone()); - - let mut column_wise_ordering_values = vec![]; - for i in 0..num_columns { - let column_values = self - .ordering_values - .iter() - .map(|x| x[i].clone()) - .collect::>(); - let array = if column_values.is_empty() { - new_empty_array(fields[i].data_type()) - } else { - ScalarValue::iter_to_array(column_values.into_iter())? - }; - column_wise_ordering_values.push(array); - } - - let ordering_array = - StructArray::try_new(struct_field, column_wise_ordering_values, None)?; - Ok(SingleRowListArrayBuilder::new(Arc::new(ordering_array)).build_list_scalar()) - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index 1b33a7900c000..74336749bf525 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -178,10 +178,6 @@ impl AggregateUDFImpl for BoolAnd { } } - fn aliases(&self) -> &[String] { - &[] - } - fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } @@ -317,10 +313,6 @@ impl AggregateUDFImpl for BoolOr { } } - fn aliases(&self) -> &[String] { - &[] - } - fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 2d995b4a41793..838a5cb45e69f 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -339,10 +339,6 @@ impl AggregateUDFImpl for Count { }) } - fn aliases(&self) -> &[String] { - &[] - } - fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { // groups accumulator only supports `COUNT(c1)`, not // `COUNT(c1, c2)`, etc diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 6df8ede4fc772..3dcc18fd4eb3e 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -32,8 +32,8 @@ use datafusion_common::{ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Documentation, Expr, ExprFunctionExt, Signature, - SortExpr, Volatility, + Accumulator, AggregateUDFImpl, Documentation, Expr, ExprFunctionExt, ReversedUDAF, + Signature, SortExpr, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; use datafusion_macros::user_doc; @@ -122,24 +122,23 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_dtypes = acc_args - .ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - - // When requirement is empty, or it is signalled by outside caller that - // the ordering requirement is/will be satisfied. - let requirement_satisfied = - acc_args.ordering_req.is_empty() || self.requirement_satisfied; - - FirstValueAccumulator::try_new( - acc_args.return_type, - &ordering_dtypes, - acc_args.ordering_req.clone(), - acc_args.ignore_nulls, - ) - .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) + if let Some(ordering_req) = acc_args.ordering_req { + if !self.requirement_satisfied { + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + return FirstValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + ordering_req.clone(), + acc_args.ignore_nulls, + ) + .map(|acc| Box::new(acc) as _); + } + } + TrivialFirstValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) + .map(|acc| Box::new(acc) as _) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { @@ -148,15 +147,11 @@ impl AggregateUDFImpl for FirstValue { args.return_type.clone(), true, )]; - fields.extend(args.ordering_fields.to_vec()); + fields.extend(args.ordering_fields.iter().cloned()); fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } - fn aliases(&self) -> &[String] { - &[] - } - fn with_beneficial_ordering( self: Arc, beneficial_ordering: bool, @@ -170,8 +165,8 @@ impl AggregateUDFImpl for FirstValue { AggregateOrderSensitivity::Beneficial } - fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(last_value_udaf()) + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Reversed(last_value_udaf()) } fn documentation(&self) -> Option<&Documentation> { @@ -179,19 +174,93 @@ impl AggregateUDFImpl for FirstValue { } } +#[derive(Debug)] +pub struct TrivialFirstValueAccumulator { + first: ScalarValue, + // Whether we have seen the first value yet. + is_set: bool, + // Ignore null values. + ignore_nulls: bool, +} + +impl TrivialFirstValueAccumulator { + /// Creates a new `TrivialFirstValueAccumulator` for the given `data_type`. + pub fn try_new(data_type: &DataType, ignore_nulls: bool) -> Result { + ScalarValue::try_from(data_type).map(|first| Self { + first, + is_set: false, + ignore_nulls, + }) + } +} + +impl Accumulator for TrivialFirstValueAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![self.first.clone(), ScalarValue::from(self.is_set)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if !self.is_set { + // Get first entry according to the pre-existing ordering (0th index): + let value = &values[0]; + let mut first_idx = None; + if self.ignore_nulls { + // If ignoring nulls, find the first non-null value. + for i in 0..value.len() { + if !value.is_null(i) { + first_idx = Some(i); + break; + } + } + } else if !value.is_empty() { + // If not ignoring nulls, return the first value if it exists. + first_idx = Some(0); + } + if let Some(first_idx) = first_idx { + let mut row = get_row_at_idx(values, first_idx)?; + self.first = row.swap_remove(0); + self.is_set = true; + } + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // FIRST_VALUE(first1, first2, first3, ...) + // Second index contains is_set flag. + if !self.is_set { + let flags = states[1].as_boolean(); + let filtered_states = + filter_states_according_to_is_set(&states[0..1], flags)?; + // If we have any values, take the first one we find: + if let Some(first) = filtered_states.first() { + self.first = ScalarValue::try_from_array(first, 0)?; + self.is_set = true; + } + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + Ok(self.first.clone()) + } + + fn size(&self) -> usize { + size_of_val(self) - size_of_val(&self.first) + self.first.size() + } +} + #[derive(Debug)] pub struct FirstValueAccumulator { first: ScalarValue, - // At the beginning, `is_set` is false, which means `first` is not seen yet. - // Once we see the first value, we set the `is_set` flag and do not update `first` anymore. + // Whether we have seen the first value yet. is_set: bool, - // Stores ordering values, of the aggregator requirement corresponding to first value - // of the aggregator. These values are used during merging of multiple partitions. + // Stores ordering values, of the aggregator requirement corresponding to + // first value of the aggregator. These values are used during merging of + // multiple partitions. orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, - // Stores whether incoming data already satisfies the ordering requirement. - requirement_satisfied: bool, // Ignore null values. ignore_nulls: bool, } @@ -208,22 +277,15 @@ impl FirstValueAccumulator { .iter() .map(ScalarValue::try_from) .collect::>>()?; - let requirement_satisfied = ordering_req.is_empty(); ScalarValue::try_from(data_type).map(|first| Self { first, is_set: false, orderings, ordering_req, - requirement_satisfied, ignore_nulls, }) } - pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; - self - } - // Updates state with the values in the given row. fn update_with_new_row(&mut self, row: &[ScalarValue]) { self.first = row[0].clone(); @@ -235,21 +297,6 @@ impl FirstValueAccumulator { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in FIRST_VALUE"); }; - if self.requirement_satisfied { - // Get first entry according to the pre-existing ordering (0th index): - if self.ignore_nulls { - // If ignoring nulls, find the first non-null value. - for i in 0..value.len() { - if !value.is_null(i) { - return Ok(Some(i)); - } - } - return Ok(None); - } else { - // If not ignoring nulls, return the first value if it exists. - return Ok((!value.is_empty()).then_some(0)); - } - } let sort_columns = ordering_values .iter() @@ -278,29 +325,22 @@ impl Accumulator for FirstValueAccumulator { fn state(&mut self) -> Result> { let mut result = vec![self.first.clone()]; result.extend(self.orderings.iter().cloned()); - result.push(ScalarValue::Boolean(Some(self.is_set))); + result.push(ScalarValue::from(self.is_set)); Ok(result) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if !self.is_set { - if let Some(first_idx) = self.get_first_idx(values)? { - let row = get_row_at_idx(values, first_idx)?; - self.update_with_new_row(&row); - } - } else if !self.requirement_satisfied { - if let Some(first_idx) = self.get_first_idx(values)? { - let row = get_row_at_idx(values, first_idx)?; - let orderings = &row[1..]; - if compare_rows( + if let Some(first_idx) = self.get_first_idx(values)? { + let row = get_row_at_idx(values, first_idx)?; + if !self.is_set + || compare_rows( &self.orderings, - orderings, + &row[1..], &get_sort_options(self.ordering_req.as_ref()), )? .is_gt() - { - self.update_with_new_row(&row); - } + { + self.update_with_new_row(&row); } } Ok(()) @@ -426,46 +466,36 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_dtypes = acc_args - .ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - - let requirement_satisfied = - acc_args.ordering_req.is_empty() || self.requirement_satisfied; - - LastValueAccumulator::try_new( - acc_args.return_type, - &ordering_dtypes, - acc_args.ordering_req.clone(), - acc_args.ignore_nulls, - ) - .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) + if let Some(ordering_req) = acc_args.ordering_req { + if !self.requirement_satisfied { + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + return LastValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + ordering_req.clone(), + acc_args.ignore_nulls, + ) + .map(|acc| Box::new(acc) as _); + } + } + TrivialLastValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) + .map(|acc| Box::new(acc) as _) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { - let StateFieldsArgs { - name, - input_types, - return_type: _, - ordering_fields, - is_distinct: _, - } = args; let mut fields = vec![Field::new( - format_state_name(name, "last_value"), - input_types[0].clone(), + format_state_name(args.name, "last_value"), + args.return_type.clone(), true, )]; - fields.extend(ordering_fields.to_vec()); + fields.extend(args.ordering_fields.iter().cloned()); fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } - fn aliases(&self) -> &[String] { - &[] - } - fn with_beneficial_ordering( self: Arc, beneficial_ordering: bool, @@ -479,8 +509,8 @@ impl AggregateUDFImpl for LastValue { AggregateOrderSensitivity::Beneficial } - fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(first_value_udaf()) + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Reversed(first_value_udaf()) } fn documentation(&self) -> Option<&Documentation> { @@ -488,6 +518,80 @@ impl AggregateUDFImpl for LastValue { } } +#[derive(Debug)] +pub struct TrivialLastValueAccumulator { + last: ScalarValue, + // The `is_set` flag keeps track of whether the last value is finalized. + // This information is used to discriminate genuine NULLs and NULLS that + // occur due to empty partitions. + is_set: bool, + // Ignore null values. + ignore_nulls: bool, +} + +impl TrivialLastValueAccumulator { + /// Creates a new `TrivialLastValueAccumulator` for the given `data_type`. + pub fn try_new(data_type: &DataType, ignore_nulls: bool) -> Result { + ScalarValue::try_from(data_type).map(|last| Self { + last, + is_set: false, + ignore_nulls, + }) + } +} + +impl Accumulator for TrivialLastValueAccumulator { + fn state(&mut self) -> Result> { + Ok(vec![self.last.clone(), ScalarValue::from(self.is_set)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + // Get last entry according to the pre-existing ordering (0th index): + let value = &values[0]; + let mut last_idx = None; + if self.ignore_nulls { + // If ignoring nulls, find the last non-null value. + for i in (0..value.len()).rev() { + if !value.is_null(i) { + last_idx = Some(i); + break; + } + } + } else if !value.is_empty() { + // If not ignoring nulls, return the last value if it exists. + last_idx = Some(value.len() - 1); + } + if let Some(last_idx) = last_idx { + let mut row = get_row_at_idx(values, last_idx)?; + self.last = row.swap_remove(0); + self.is_set = true; + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // LAST_VALUE(last1, last2, last3, ...) + // Second index contains is_set flag. + let flags = states[1].as_boolean(); + let filtered_states = + filter_states_according_to_is_set(&states[0..1], flags)?; + // If we have any values, take the last one we find: + if let Some(last) = filtered_states.last() { + self.last = ScalarValue::try_from_array(last, 0)?; + self.is_set = true; + } + Ok(()) + } + + fn evaluate(&mut self) -> Result { + Ok(self.last.clone()) + } + + fn size(&self) -> usize { + size_of_val(self) - size_of_val(&self.last) + self.last.size() + } +} + #[derive(Debug)] struct LastValueAccumulator { last: ScalarValue, @@ -498,8 +602,6 @@ struct LastValueAccumulator { orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, - // Stores whether incoming data already satisfies the ordering requirement. - requirement_satisfied: bool, // Ignore null values. ignore_nulls: bool, } @@ -515,14 +617,12 @@ impl LastValueAccumulator { let orderings = ordering_dtypes .iter() .map(ScalarValue::try_from) - .collect::>>()?; - let requirement_satisfied = ordering_req.is_empty(); + .collect::>()?; ScalarValue::try_from(data_type).map(|last| Self { last, is_set: false, orderings, ordering_req, - requirement_satisfied, ignore_nulls, }) } @@ -538,20 +638,7 @@ impl LastValueAccumulator { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in LAST_VALUE"); }; - if self.requirement_satisfied { - // Get last entry according to the order of data: - if self.ignore_nulls { - // If ignoring nulls, find the last non-null value. - for i in (0..value.len()).rev() { - if !value.is_null(i) { - return Ok(Some(i)); - } - } - return Ok(None); - } else { - return Ok((!value.is_empty()).then_some(value.len() - 1)); - } - } + let sort_columns = ordering_values .iter() .zip(self.ordering_req.iter()) @@ -572,32 +659,22 @@ impl LastValueAccumulator { Ok(max_ind) } - - fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; - self - } } impl Accumulator for LastValueAccumulator { fn state(&mut self) -> Result> { let mut result = vec![self.last.clone()]; result.extend(self.orderings.clone()); - result.push(ScalarValue::Boolean(Some(self.is_set))); + result.push(ScalarValue::from(self.is_set)); Ok(result) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - if !self.is_set || self.requirement_satisfied { - if let Some(last_idx) = self.get_last_idx(values)? { - let row = get_row_at_idx(values, last_idx)?; - self.update_with_new_row(&row); - } - } else if let Some(last_idx) = self.get_last_idx(values)? { + if let Some(last_idx) = self.get_last_idx(values)? { let row = get_row_at_idx(values, last_idx)?; let orderings = &row[1..]; // Update when there is a more recent entry - if compare_rows( + if !self.is_set || compare_rows( &self.orderings, orderings, &get_sort_options(self.ordering_req.as_ref()), @@ -607,7 +684,6 @@ impl Accumulator for LastValueAccumulator { self.update_with_new_row(&row); } } - Ok(()) } @@ -635,7 +711,6 @@ impl Accumulator for LastValueAccumulator { // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set - || self.requirement_satisfied || compare_rows(&self.orderings, last_ordering, &sort_options)?.is_lt() { // Update with last value in the state. Note that we should exclude the @@ -690,18 +765,10 @@ mod tests { #[test] fn test_first_last_value_value() -> Result<()> { - let mut first_accumulator = FirstValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; - let mut last_accumulator = LastValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut first_accumulator = + TrivialFirstValueAccumulator::try_new(&DataType::Int64, false)?; + let mut last_accumulator = + TrivialLastValueAccumulator::try_new(&DataType::Int64, false)?; // first value in the tuple is start of the range (inclusive), // second value in the tuple is end of the range (exclusive) let ranges: Vec<(i64, i64)> = vec![(0, 10), (1, 11), (2, 13)]; @@ -738,22 +805,14 @@ mod tests { .collect::>(); // FirstValueAccumulator - let mut first_accumulator = FirstValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut first_accumulator = + TrivialFirstValueAccumulator::try_new(&DataType::Int64, false)?; first_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = first_accumulator.state()?; - let mut first_accumulator = FirstValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut first_accumulator = + TrivialFirstValueAccumulator::try_new(&DataType::Int64, false)?; first_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = first_accumulator.state()?; @@ -768,34 +827,22 @@ mod tests { ])?); } - let mut first_accumulator = FirstValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut first_accumulator = + TrivialFirstValueAccumulator::try_new(&DataType::Int64, false)?; first_accumulator.merge_batch(&states)?; let merged_state = first_accumulator.state()?; assert_eq!(merged_state.len(), state1.len()); // LastValueAccumulator - let mut last_accumulator = LastValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut last_accumulator = + TrivialLastValueAccumulator::try_new(&DataType::Int64, false)?; last_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = last_accumulator.state()?; - let mut last_accumulator = LastValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut last_accumulator = + TrivialLastValueAccumulator::try_new(&DataType::Int64, false)?; last_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = last_accumulator.state()?; @@ -810,12 +857,8 @@ mod tests { ])?); } - let mut last_accumulator = LastValueAccumulator::try_new( - &DataType::Int64, - &[], - LexOrdering::default(), - false, - )?; + let mut last_accumulator = + TrivialLastValueAccumulator::try_new(&DataType::Int64, false)?; last_accumulator.merge_batch(&states)?; let merged_state = last_accumulator.state()?; diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index ba6b63260e068..98f590a780a81 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -213,10 +213,6 @@ impl AggregateUDFImpl for Median { } } - fn aliases(&self) -> &[String] { - &[] - } - fn documentation(&self) -> Option<&Documentation> { self.doc() } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 17ac8c43019c8..64c5b5ab8fa20 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -233,10 +233,6 @@ impl AggregateUDFImpl for Max { Ok(Box::new(MaxAccumulator::try_new(acc_args.return_type)?)) } - fn aliases(&self) -> &[String] { - &[] - } - fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; matches!( @@ -1087,10 +1083,6 @@ impl AggregateUDFImpl for Min { Ok(Box::new(MinAccumulator::try_new(acc_args.return_type)?)) } - fn aliases(&self) -> &[String] { - &[] - } - fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; matches!( diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index d84bd02a6bafe..cffc4d10f338f 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -86,7 +86,7 @@ pub fn nth_value( description = "The position (nth) of the value to retrieve, based on the ordering." ) )] -/// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi +/// Expression for a `NTH_VALUE(..., ... ORDER BY ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, /// and then their results are merged. #[derive(Debug)] @@ -148,8 +148,11 @@ impl AggregateUDFImpl for NthValueAgg { } }; - let ordering_dtypes = acc_args - .ordering_req + let Some(ordering_req) = acc_args.ordering_req else { + return TrivialNthValueAccumulator::try_new(n, acc_args.return_type) + .map(|acc| Box::new(acc) as _); + }; + let ordering_dtypes = ordering_req .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -159,7 +162,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &data_type, &ordering_dtypes, - acc_args.ordering_req.clone(), + ordering_req.clone(), ) .map(|acc| Box::new(acc) as _) } @@ -182,10 +185,6 @@ impl AggregateUDFImpl for NthValueAgg { Ok(fields) } - fn aliases(&self) -> &[String] { - &[] - } - fn reverse_expr(&self) -> ReversedUDAF { ReversedUDAF::Reversed(nth_value_udaf()) } @@ -195,6 +194,126 @@ impl AggregateUDFImpl for NthValueAgg { } } +#[derive(Debug)] +pub struct TrivialNthValueAccumulator { + /// The `N` value. + n: i64, + /// Stores entries in the `NTH_VALUE` result. + values: VecDeque, + /// Data types of the value. + datatype: DataType, +} + +impl TrivialNthValueAccumulator { + /// Create a new order-insensitive NTH_VALUE accumulator based on the given + /// item data type. + pub fn try_new(n: i64, datatype: &DataType) -> Result { + if n == 0 { + // n cannot be 0 + return internal_err!("Nth value indices are 1 based. 0 is invalid index"); + } + Ok(Self { + n, + values: VecDeque::new(), + datatype: datatype.clone(), + }) + } + + /// Updates state, with the `values`. Fetch contains missing number of entries for state to be complete + /// None represents all of the new `values` need to be added to the state. + fn append_new_data( + &mut self, + values: &[ArrayRef], + fetch: Option, + ) -> Result<()> { + let n_row = values[0].len(); + let n_to_add = if let Some(fetch) = fetch { + std::cmp::min(fetch, n_row) + } else { + n_row + }; + for index in 0..n_to_add { + let mut row = get_row_at_idx(values, index)?; + self.values.push_back(row.swap_remove(0)); + // At index 1, we have n index argument, which is constant. + } + Ok(()) + } +} + +impl Accumulator for TrivialNthValueAccumulator { + /// Updates its state with the `values`. Assumes data in the `values` satisfies the required + /// ordering for the accumulator (across consecutive batches, not just batch-wise). + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + if !values.is_empty() { + let n_required = self.n.unsigned_abs() as usize; + let from_start = self.n > 0; + if from_start { + // direction is from start + let n_remaining = n_required.saturating_sub(self.values.len()); + self.append_new_data(values, Some(n_remaining))?; + } else { + // direction is from end + self.append_new_data(values, None)?; + let start_offset = self.values.len().saturating_sub(n_required); + if start_offset > 0 { + self.values.drain(0..start_offset); + } + } + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + if !states.is_empty() { + // First entry in the state is the aggregation result. + let n_required = self.n.unsigned_abs() as usize; + let array_agg_res = ScalarValue::convert_array_to_scalar_vec(&states[0])?; + for v in array_agg_res.into_iter() { + self.values.extend(v); + if self.values.len() > n_required { + // There is enough data collected, can stop merging: + break; + } + } + } + Ok(()) + } + + fn state(&mut self) -> Result> { + let mut values_cloned = self.values.clone(); + let values_slice = values_cloned.make_contiguous(); + Ok(vec![ScalarValue::List(ScalarValue::new_list_nullable( + values_slice, + &self.datatype, + ))]) + } + + fn evaluate(&mut self) -> Result { + let n_required = self.n.unsigned_abs() as usize; + let from_start = self.n > 0; + let nth_value_idx = if from_start { + // index is from start + let forward_idx = n_required - 1; + (forward_idx < self.values.len()).then_some(forward_idx) + } else { + // index is from end + self.values.len().checked_sub(n_required) + }; + if let Some(idx) = nth_value_idx { + Ok(self.values[idx].clone()) + } else { + ScalarValue::try_from(self.datatype.clone()) + } + } + + fn size(&self) -> usize { + size_of_val(self) + ScalarValue::size_of_vec_deque(&self.values) + - size_of_val(&self.values) + + size_of::() + } +} + #[derive(Debug)] pub struct NthValueAccumulator { /// The `N` value. @@ -236,6 +355,64 @@ impl NthValueAccumulator { ordering_req, }) } + + fn evaluate_orderings(&self) -> Result { + let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); + let struct_field = Fields::from(fields.clone()); + + let mut column_wise_ordering_values = vec![]; + let num_columns = fields.len(); + for i in 0..num_columns { + let column_values = self + .ordering_values + .iter() + .map(|x| x[i].clone()) + .collect::>(); + let array = if column_values.is_empty() { + new_empty_array(fields[i].data_type()) + } else { + ScalarValue::iter_to_array(column_values.into_iter())? + }; + column_wise_ordering_values.push(array); + } + + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; + + Ok(SingleRowListArrayBuilder::new(Arc::new(ordering_array)).build_list_scalar()) + } + + fn evaluate_values(&self) -> ScalarValue { + let mut values_cloned = self.values.clone(); + let values_slice = values_cloned.make_contiguous(); + ScalarValue::List(ScalarValue::new_list_nullable( + values_slice, + &self.datatypes[0], + )) + } + + /// Updates state, with the `values`. Fetch contains missing number of entries for state to be complete + /// None represents all of the new `values` need to be added to the state. + fn append_new_data( + &mut self, + values: &[ArrayRef], + fetch: Option, + ) -> Result<()> { + let n_row = values[0].len(); + let n_to_add = if let Some(fetch) = fetch { + std::cmp::min(fetch, n_row) + } else { + n_row + }; + for index in 0..n_to_add { + let row = get_row_at_idx(values, index)?; + self.values.push_back(row[0].clone()); + // At index 1, we have n index argument. + // Ordering values cover starting from 2nd index to end + self.ordering_values.push_back(row[2..].to_vec()); + } + Ok(()) + } } impl Accumulator for NthValueAccumulator { @@ -396,63 +573,3 @@ impl Accumulator for NthValueAccumulator { total } } - -impl NthValueAccumulator { - fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); - let struct_field = Fields::from(fields.clone()); - - let mut column_wise_ordering_values = vec![]; - let num_columns = fields.len(); - for i in 0..num_columns { - let column_values = self - .ordering_values - .iter() - .map(|x| x[i].clone()) - .collect::>(); - let array = if column_values.is_empty() { - new_empty_array(fields[i].data_type()) - } else { - ScalarValue::iter_to_array(column_values.into_iter())? - }; - column_wise_ordering_values.push(array); - } - - let ordering_array = - StructArray::try_new(struct_field, column_wise_ordering_values, None)?; - - Ok(SingleRowListArrayBuilder::new(Arc::new(ordering_array)).build_list_scalar()) - } - - fn evaluate_values(&self) -> ScalarValue { - let mut values_cloned = self.values.clone(); - let values_slice = values_cloned.make_contiguous(); - ScalarValue::List(ScalarValue::new_list_nullable( - values_slice, - &self.datatypes[0], - )) - } - - /// Updates state, with the `values`. Fetch contains missing number of entries for state to be complete - /// None represents all of the new `values` need to be added to the state. - fn append_new_data( - &mut self, - values: &[ArrayRef], - fetch: Option, - ) -> Result<()> { - let n_row = values[0].len(); - let n_to_add = if let Some(fetch) = fetch { - std::cmp::min(fetch, n_row) - } else { - n_row - }; - for index in 0..n_to_add { - let row = get_row_at_idx(values, index)?; - self.values.push_back(row[0].clone()); - // At index 1, we have n index argument. - // Ordering values cover starting from 2nd index to end - self.ordering_values.push_back(row[2..].to_vec()); - } - Ok(()) - } -} diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index adf86a128cfb1..b26572c69c225 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -387,7 +387,6 @@ mod tests { use datafusion_expr::AggregateUDF; use datafusion_functions_aggregate_common::utils::get_accum_scalar_values_as_arrays; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; #[test] @@ -439,7 +438,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &LexOrdering::default(), + ordering_req: None, name: "a", is_distinct: false, is_reversed: false, @@ -450,7 +449,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &LexOrdering::default(), + ordering_req: None, name: "a", is_distinct: false, is_reversed: false, diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 76a1315c2d889..9295aa6fbba4f 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -208,10 +208,6 @@ impl AggregateUDFImpl for Sum { } } - fn aliases(&self) -> &[String] { - &[] - } - fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { !args.is_distinct } diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index bfd06b39d2067..b3c530416579a 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -231,6 +231,7 @@ impl ScalarUDFImpl for DatePartFunc { fn aliases(&self) -> &[String] { &self.aliases } + fn documentation(&self) -> Option<&Documentation> { self.doc() } diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index 8b2e5ad874717..6c8158a5932a2 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -165,6 +165,7 @@ impl ScalarUDFImpl for ToCharFunc { fn aliases(&self) -> &[String] { &self.aliases } + fn documentation(&self) -> Option<&Documentation> { self.doc() } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 3cebe48fd5bbd..5ac55fc62d597 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -429,13 +429,13 @@ impl LexOrdering { /// /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. pub fn collapse(self) -> Self { - let mut output = Self::default(); - for item in self { - if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); + let mut orderings = Vec::::new(); + for element in self { + if !orderings.iter().any(|item| item.expr.eq(&element.expr)) { + orderings.push(element); } } - output + Self::new(orderings) } /// Transforms each `PhysicalSortExpr` in the `LexOrdering` @@ -584,13 +584,13 @@ impl LexRequirement { /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a /// Some(ASC)]`. pub fn collapse(self) -> Self { - let mut output = Vec::::new(); - for item in self { - if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); + let mut reqs = Vec::::new(); + for element in self { + if !reqs.iter().any(|item| item.expr.eq(&element.expr)) { + reqs.push(element); } } - LexRequirement::new(output) + Self::new(reqs) } } @@ -636,8 +636,20 @@ impl From for LexRequirement { } } +impl From> for LexRequirement { + fn from(value: Vec) -> Self { + Self::new(value.into_iter().map(Into::into).collect()) + } +} + impl From for LexOrdering { fn from(value: LexRequirement) -> Self { value.into_iter().map(Into::into).collect() } } + +impl From> for LexOrdering { + fn from(value: Vec) -> Self { + Self::new(value.into_iter().map(Into::into).collect()) + } +} diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index ae3d9050fa628..e5687ae0cb22a 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -70,7 +70,7 @@ pub struct AggregateExprBuilder { /// Arrow Schema for the aggregate function schema: SchemaRef, /// The physical order by expressions - ordering_req: LexOrdering, + ordering_req: Option, /// Whether to ignore null values ignore_nulls: bool, /// Whether is distinct aggregate function @@ -87,7 +87,7 @@ impl AggregateExprBuilder { alias: None, human_display: String::default(), schema: Arc::new(Schema::empty()), - ordering_req: LexOrdering::default(), + ordering_req: None, ignore_nulls: false, is_distinct: false, is_reversed: false, @@ -115,14 +115,13 @@ impl AggregateExprBuilder { let mut ordering_fields = vec![]; - if !ordering_req.is_empty() { + if let Some(ordering_req) = &ordering_req { let ordering_types = ordering_req .iter() .map(|e| e.expr.data_type(&schema)) .collect::>>()?; - ordering_fields = - utils::ordering_fields(ordering_req.as_ref(), &ordering_types); + ordering_fields = utils::ordering_fields(ordering_req, &ordering_types); } let input_exprs_types = args @@ -179,7 +178,7 @@ impl AggregateExprBuilder { self } - pub fn order_by(mut self, order_by: LexOrdering) -> Self { + pub fn order_by(mut self, order_by: Option) -> Self { self.ordering_req = order_by; self } @@ -230,7 +229,7 @@ pub struct AggregateFunctionExpr { human_display: String, schema: Schema, // The physical order by expressions - ordering_req: LexOrdering, + ordering_req: Option, // Whether to ignore null values ignore_nulls: bool, // fields used for order sensitive aggregation functions @@ -323,22 +322,17 @@ impl AggregateFunctionExpr { /// By default it is `None` (there is no requirement) /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this pub fn order_bys(&self) -> Option<&LexOrdering> { - if self.ordering_req.is_empty() { + let Some(ordering_req) = &self.ordering_req else { return None; - } - - if !self.order_sensitivity().is_insensitive() { - return Some(self.ordering_req.as_ref()); - } - - None + }; + (!self.order_sensitivity().is_insensitive()).then_some(ordering_req) } /// Indicates whether aggregator can produce the correct result with any /// arbitrary input ordering. By default, we assume that aggregate expressions /// are order insensitive. pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { - if !self.ordering_req.is_empty() { + if self.ordering_req.is_some() { // If there is requirement, use the sensitivity of the implementation self.fun.order_sensitivity() } else { @@ -493,18 +487,16 @@ impl AggregateFunctionExpr { ReversedUDAF::NotSupported => None, ReversedUDAF::Identical => Some(self.clone()), ReversedUDAF::Reversed(reverse_udf) => { - let reverse_ordering_req = reverse_order_bys(self.ordering_req.as_ref()); let mut name = self.name().to_string(); // If the function is changed, we need to reverse order_by clause as well // i.e. First(a order by b asc null first) -> Last(a order by b desc null last) - if self.fun().name() == reverse_udf.name() { - } else { + if self.fun().name() != reverse_udf.name() { replace_order_by_clause(&mut name); } replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) - .order_by(reverse_ordering_req) + .order_by(self.ordering_req.as_ref().map(reverse_order_bys)) .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) @@ -520,14 +512,12 @@ impl AggregateFunctionExpr { /// These expressions are (1)function arguments, (2) order by expressions. pub fn all_expressions(&self) -> AggregatePhysicalExpressions { let args = self.expressions(); - let order_bys = self + let order_by_exprs = self .order_bys() - .cloned() - .unwrap_or_else(LexOrdering::default); - let order_by_exprs = order_bys + .map_or_else(Vec::new, |ordering| ordering.iter().collect()) .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect::>(); + .collect(); AggregatePhysicalExpressions { args, order_by_exprs, diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index c568fab51bd3d..b2cdd29881f97 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -55,6 +55,7 @@ pub fn add_offset_to_expr( #[cfg(test)] mod tests { + use std::borrow::Borrow; use super::*; use crate::expressions::col; @@ -200,50 +201,23 @@ mod tests { } // Convert each tuple to PhysicalSortExpr - pub fn convert_to_sort_exprs( - in_data: &[(&Arc, SortOptions)], - ) -> LexOrdering { - in_data - .iter() + pub fn convert_to_sort_exprs>>( + args: &[(T, SortOptions)], + ) -> Vec { + args.iter() .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(*expr), + expr: Arc::clone(expr.borrow()), options: *options, }) .collect() } - // Convert each inner tuple to PhysicalSortExpr - pub fn convert_to_orderings( - orderings: &[Vec<(&Arc, SortOptions)>], + // Convert each inner tuple to PhysicalSortExpr and then into LexOrdering + pub fn convert_to_orderings>>( + args: &[Vec<(T, SortOptions)>], ) -> Vec { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) - .collect() - } - - // Convert each tuple to PhysicalSortExpr - pub fn convert_to_sort_exprs_owned( - in_data: &[(Arc, SortOptions)], - ) -> LexOrdering { - LexOrdering::new( - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr), - options: *options, - }) - .collect(), - ) - } - - // Convert each inner tuple to PhysicalSortExpr - pub fn convert_to_orderings_owned( - orderings: &[Vec<(Arc, SortOptions)>], - ) -> Vec { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) + args.iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs).into()) .collect() } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 0efd46ad912e9..fbeab0c38cdd3 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -682,7 +682,7 @@ mod tests { let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(reqs.as_ref()), + eq_properties.ordering_satisfy(&reqs.into()), expected, "{}", err_msg @@ -742,7 +742,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(reqs.as_ref()), + eq_properties.ordering_satisfy(&reqs.into()), expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 035678fbf1f39..54ccd07ec4de9 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -138,9 +138,7 @@ fn project_index_to_exprs( #[cfg(test)] mod tests { use super::*; - use crate::equivalence::tests::{ - convert_to_orderings, convert_to_orderings_owned, output_schema, - }; + use crate::equivalence::tests::{convert_to_orderings, output_schema}; use crate::equivalence::EquivalenceProperties; use crate::expressions::{col, BinaryExpr}; use crate::utils::tests::TestScalarUDF; @@ -628,7 +626,7 @@ mod tests { .collect::>() }) .collect::>(); - let expected = convert_to_orderings_owned(&expected); + let expected = convert_to_orderings(&expected); let projected_eq = eq_properties.project(&projection_mapping, output_schema); let orderings = projected_eq.oeq_class(); diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 03f058a006876..fa75f06377fd9 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -615,7 +615,7 @@ mod tests { assert_eq!(idxs, vec![0, 1]); assert_eq!( result, - LexOrdering::new(vec![ + vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -624,7 +624,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ]) + ] ); let schema = Schema::new(vec![ @@ -656,7 +656,7 @@ mod tests { assert_eq!(idxs, vec![0, 1]); assert_eq!( result, - LexOrdering::new(vec![ + vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -665,7 +665,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ]) + ] ); let required_columns = [ diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index a8c29ca232d19..8a3b7320d7f68 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -211,9 +211,9 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { + // Prune out constant expressions: let constants = self.constants(); - let mut output_ordering = self.oeq_class().output_ordering().unwrap_or_default(); - // Prune out constant expressions + let mut output_ordering = self.oeq_class().output_ordering()?; output_ordering .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)); (!output_ordering.is_empty()).then_some(output_ordering) @@ -446,17 +446,16 @@ impl EquivalenceProperties { /// If the given ordering is already satisfied, the function does nothing. pub fn with_reorder(mut self, sort_exprs: LexOrdering) -> Self { // Filter out constant expressions as they don't affect ordering - let filtered_exprs = LexOrdering::new( - sort_exprs - .into_iter() - .filter(|expr| !self.is_expr_constant(&expr.expr)) - .collect(), - ); + let filtered_exprs = sort_exprs + .into_iter() + .filter(|expr| !self.is_expr_constant(&expr.expr)) + .collect::>(); if filtered_exprs.is_empty() { return self; } + let filtered_exprs = LexOrdering::new(filtered_exprs); let mut new_orderings = vec![filtered_exprs.clone()]; // Preserve valid suffixes from existing orderings @@ -607,7 +606,7 @@ impl EquivalenceProperties { // by any ordering. return true; }; - // Check whether given requirement is satisfied by constraints first: + // Then, check whether given requirement is satisfied by constraints: if self.satisfied_by_constraints(&normalized_reqs) { return true; } @@ -1227,7 +1226,7 @@ impl EquivalenceProperties { pub fn find_longest_permutation( &self, exprs: &[Arc], - ) -> (LexOrdering, Vec) { + ) -> (Vec, Vec) { let mut eq_properties = self.clone(); let mut result = vec![]; // The algorithm is as follows: @@ -1290,8 +1289,7 @@ impl EquivalenceProperties { // Add new ordered section to the state. result.extend(ordered_exprs); } - let (left, right) = result.into_iter().unzip(); - (LexOrdering::new(left), right) + result.into_iter().unzip() } /// This function determines whether the provided expression is constant diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index fac83dfc45247..47479cbee60b9 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -380,7 +380,7 @@ where exprs .into_iter() .map(|expr| create_physical_expr(expr, input_dfschema, execution_props)) - .collect::>>() + .collect() } /// Convert a logical expression to a physical expression (without any simplification, etc) diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 22e8aea83fe78..042041d5a3ed9 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -290,7 +290,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( let (mut ordering, _) = eqp.find_longest_permutation(partition_by); if ordering.len() == partition_by.len() { ordering.push(expr); - eqp.add_new_orderings([ordering]); + eqp.add_new_orderings([ordering.into()]); } } } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 465ed54556cc2..a66a77b3b24e3 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -42,7 +42,6 @@ use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, }; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -945,16 +944,10 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { // if any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.bounded_order_preserving_variants`) - let should_preserve_ordering = input.plan.output_ordering().is_some(); - - let new_plan = if should_preserve_ordering { + // (determined by flag `config.optimizer.prefer_existing_sort`) + let new_plan = if let Some(ordering) = input.plan.output_ordering() { Arc::new(SortPreservingMergeExec::new( - input - .plan - .output_ordering() - .unwrap_or(&LexOrdering::default()) - .clone(), + ordering.clone(), Arc::clone(&input.plan), )) as _ } else { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 3948c762a38db..6adf30f4c09a7 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -24,7 +24,9 @@ use crate::utils::{ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, Result}; +use datafusion_common::{ + internal_datafusion_err, internal_err, plan_err, HashSet, JoinSide, Result, +}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; @@ -244,7 +246,9 @@ fn pushdown_requirement_to_children( .properties() .output_ordering() .cloned() - .unwrap_or(LexOrdering::default()), + .ok_or_else(|| { + internal_datafusion_err!("SortExec should have output ordering") + })?, ); sort_exec .properties() @@ -336,9 +340,7 @@ fn pushdown_requirement_to_children( let mut spm_eqs = plan.equivalence_properties().clone(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); - if spm_eqs - .ordering_satisfy(plan.output_ordering().unwrap_or(&LexOrdering::default())) - { + if spm_eqs.ordering_satisfy(plan.output_ordering().unwrap()) { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. Ok(Some(vec![Some(parent_required.clone())])) @@ -419,8 +421,8 @@ fn try_pushdown_requirements_to_join( let mut smj_required_orderings = smj.required_input_ordering(); let right_requirement = smj_required_orderings.swap_remove(1); let left_requirement = smj_required_orderings.swap_remove(0); - let left_ordering = &smj.left().output_ordering().cloned().unwrap_or_default(); - let right_ordering = &smj.right().output_ordering().cloned().unwrap_or_default(); + let left_ordering = &smj.left().output_ordering().cloned().unwrap(); + let right_ordering = &smj.right().output_ordering().cloned().unwrap(); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { @@ -552,7 +554,7 @@ fn shift_right_required( }) .collect::>(); if new_right_required.len() == parent_required.len() { - Ok(LexRequirement::new(new_right_required)) + Ok(new_right_required.into()) } else { plan_err!( "Expect to shift all the parent required column indexes for SortMergeJoin" diff --git a/datafusion/physical-plan/benches/partial_ordering.rs b/datafusion/physical-plan/benches/partial_ordering.rs index 422826abcc8ba..117665f7bba56 100644 --- a/datafusion/physical-plan/benches/partial_ordering.rs +++ b/datafusion/physical-plan/benches/partial_ordering.rs @@ -18,11 +18,10 @@ use std::sync::Arc; use arrow::array::{ArrayRef, Int32Array}; -use arrow_schema::{DataType, Field, Schema, SortOptions}; -use criterion::{criterion_group, criterion_main, Criterion}; -use datafusion_physical_expr::{expressions::col, LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::order::GroupOrderingPartial; +use criterion::{criterion_group, criterion_main, Criterion}; + const BATCH_SIZE: usize = 8192; fn create_test_arrays(num_columns: usize) -> Vec { @@ -39,22 +38,7 @@ fn bench_new_groups(c: &mut Criterion) { // Test with 1, 2, 4, and 8 order indices for num_columns in [1, 2, 4, 8] { - let fields: Vec = (0..num_columns) - .map(|i| Field::new(format!("col{}", i), DataType::Int32, false)) - .collect(); - let schema = Schema::new(fields); - let order_indices: Vec = (0..num_columns).collect(); - let ordering = LexOrdering::new( - (0..num_columns) - .map(|i| { - PhysicalSortExpr::new( - col(&format!("col{}", i), &schema).unwrap(), - SortOptions::default(), - ) - }) - .collect(), - ); group.bench_function(format!("order_indices_{}", num_columns), |b| { let batch_group_values = create_test_arrays(num_columns); @@ -62,8 +46,7 @@ fn bench_new_groups(c: &mut Criterion) { b.iter(|| { let mut ordering = - GroupOrderingPartial::try_new(&schema, &order_indices, &ordering) - .unwrap(); + GroupOrderingPartial::try_new(order_indices.clone()).unwrap(); ordering .new_groups(&batch_group_values, &group_indices, BATCH_SIZE) .unwrap(); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3cb94032dcf26..384125ebbcdf0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2206,7 +2206,7 @@ mod tests { let args = [col("b", schema)?]; AggregateExprBuilder::new(first_value_udaf(), args.to_vec()) - .order_by(LexOrdering::new(ordering_req.to_vec())) + .order_by(Some(LexOrdering::from_iter(ordering_req))) .schema(Arc::new(schema.clone())) .alias(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2224,7 +2224,7 @@ mod tests { }]; let args = [col("b", schema)?]; AggregateExprBuilder::new(last_value_udaf(), args.to_vec()) - .order_by(LexOrdering::new(ordering_req.to_vec())) + .order_by(Some(LexOrdering::from_iter(ordering_req))) .schema(Arc::new(schema.clone())) .alias(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2403,10 +2403,9 @@ mod tests { let mut aggr_exprs = order_by_exprs .into_iter() .map(|order_by_expr| { - let ordering_req = order_by_expr.unwrap_or_default(); AggregateExprBuilder::new(array_agg_udaf(), vec![Arc::clone(col_a)]) .alias("a") - .order_by(LexOrdering::new(ordering_req.to_vec())) + .order_by(order_by_expr.map(LexOrdering::new)) .schema(Arc::clone(&test_schema)) .build() .map(Arc::new) diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 0b742b3d20fdc..bbcb30d877cf0 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -15,12 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::mem::size_of; + use arrow::array::ArrayRef; -use arrow::datatypes::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use std::mem::size_of; mod full; mod partial; @@ -42,15 +41,11 @@ pub enum GroupOrdering { impl GroupOrdering { /// Create a `GroupOrdering` for the specified ordering - pub fn try_new( - input_schema: &Schema, - mode: &InputOrderMode, - ordering: &LexOrdering, - ) -> Result { + pub fn try_new(mode: &InputOrderMode) -> Result { match mode { InputOrderMode::Linear => Ok(GroupOrdering::None), InputOrderMode::PartiallySorted(order_indices) => { - GroupOrderingPartial::try_new(input_schema, order_indices, ordering) + GroupOrderingPartial::try_new(order_indices.clone()) .map(GroupOrdering::Partial) } InputOrderMode::Sorted => Ok(GroupOrdering::Full(GroupOrderingFull::new())), diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index aff69277a4cef..13849d0ba5329 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -15,18 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::cmp::Ordering; +use std::mem::size_of; +use std::sync::Arc; + use arrow::array::ArrayRef; use arrow::compute::SortOptions; -use arrow::datatypes::Schema; use arrow_ord::partition::partition; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use std::cmp::Ordering; -use std::mem::size_of; -use std::sync::Arc; /// Tracks grouping state when the data is ordered by some subset of /// the group keys. @@ -118,17 +117,11 @@ impl State { impl GroupOrderingPartial { /// TODO: Remove unnecessary `input_schema` parameter. - pub fn try_new( - _input_schema: &Schema, - order_indices: &[usize], - ordering: &LexOrdering, - ) -> Result { - assert!(!order_indices.is_empty()); - assert!(order_indices.len() <= ordering.len()); - + pub fn try_new(order_indices: Vec) -> Result { + debug_assert!(!order_indices.is_empty()); Ok(Self { state: State::Start, - order_indices: order_indices.to_vec(), + order_indices, }) } @@ -276,29 +269,15 @@ impl GroupOrderingPartial { #[cfg(test)] mod tests { - use arrow::array::Int32Array; - use arrow_schema::{DataType, Field}; - use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; - use super::*; + use arrow::array::Int32Array; + #[test] fn test_group_ordering_partial() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - ]); - // Ordered on column a let order_indices = vec![0]; - - let ordering = LexOrdering::new(vec![PhysicalSortExpr::new( - col("a", &schema)?, - SortOptions::default(), - )]); - - let mut group_ordering = - GroupOrderingPartial::try_new(&schema, &order_indices, &ordering)?; + let mut group_ordering = GroupOrderingPartial::try_new(order_indices)?; let batch_group_values: Vec = vec![ Arc::new(Int32Array::from(vec![1, 2, 3])), diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 05122d5a5403d..15df982efdb2f 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -32,7 +32,7 @@ use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::{read_spill_as_stream, spill_record_batch_by_size}; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; +use crate::{aggregates, metrics, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; @@ -529,16 +529,7 @@ impl GroupedHashAggregateStream { let reservation = MemoryConsumer::new(name) .with_can_spill(true) .register(context.memory_pool()); - let (ordering, _) = agg - .properties() - .equivalence_properties() - .find_longest_permutation(&agg_group_by.output_exprs()); - let group_ordering = GroupOrdering::try_new( - &group_schema, - &agg.input_order_mode, - ordering.as_ref(), - )?; - + let group_ordering = GroupOrdering::try_new(&agg.input_order_mode)?; let group_values = new_group_values(group_schema, &group_ordering)?; timer.done(); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c27de77401eba..df5d366148af0 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -577,7 +577,7 @@ impl ExecutionPlan for RepartitionExec { let input = Arc::clone(&self.input); let partitioning = self.partitioning().clone(); let metrics = self.metrics.clone(); - let preserve_order = self.preserve_order; + let preserve_order = self.sort_exprs().is_some(); let name = self.name().to_owned(); let schema = self.schema(); let schema_captured = Arc::clone(&schema); diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 51e0803d000a0..28ac693af45de 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -61,8 +61,7 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use ahash::RandomState; use futures::stream::Stream; @@ -153,7 +152,7 @@ impl BoundedWindowAggExec { // We are sure that partition by columns are always at the beginning of sort_keys // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points - pub fn partition_by_sort_keys(&self) -> Result { + pub fn partition_by_sort_keys(&self) -> Result> { let partition_by = self.window_expr()[0].partition_by(); get_partition_by_sort_exprs( &self.input, @@ -744,7 +743,7 @@ impl LinearSearch { /// when computing partitions. pub struct SortedSearch { /// Stores partition by columns and their ordering information - partition_by_sort_keys: LexOrdering, + partition_by_sort_keys: Vec, /// Input ordering and partition by key ordering need not be the same, so /// this vector stores the mapping between them. For instance, if the input /// is ordered by a, b and the window expression contains a PARTITION BY b, a diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index fc799dff1c609..683d32bb9ccac 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -332,7 +332,7 @@ pub(crate) fn get_partition_by_sort_exprs( input: &Arc, partition_by_exprs: &[Arc], ordered_partition_by_indices: &[usize], -) -> Result { +) -> Result> { let ordered_partition_exprs = ordered_partition_by_indices .iter() .map(|idx| Arc::clone(&partition_by_exprs[*idx])) diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 73d45ea263550..b95347ec600ed 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -44,7 +44,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr::PhysicalSortExpr; use futures::{ready, Stream, StreamExt}; @@ -107,7 +107,7 @@ impl WindowAggExec { // We are sure that partition by columns are always at the beginning of sort_keys // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points - pub fn partition_by_sort_keys(&self) -> Result { + pub fn partition_by_sort_keys(&self) -> Result> { let partition_by = self.window_expr()[0].partition_by(); get_partition_by_sort_exprs( &self.input, @@ -307,7 +307,7 @@ pub struct WindowAggStream { batches: Vec, finished: bool, window_expr: Vec>, - partition_by_sort_keys: LexOrdering, + partition_by_sort_keys: Vec, baseline_metrics: BaselineMetrics, ordered_partition_by_indices: Vec, } @@ -319,7 +319,7 @@ impl WindowAggStream { window_expr: Vec>, input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, - partition_by_sort_keys: LexOrdering, + partition_by_sort_keys: Vec, ordered_partition_by_indices: Vec, ) -> Result { // In WindowAggExec all partition by columns should be ordered. diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 0bf9fdb63d593..b0b561d6d1815 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -105,7 +105,7 @@ pub fn parse_physical_sort_exprs( .map(|sort_expr| { parse_physical_sort_expr(sort_expr, registry, input_schema, codec) }) - .collect::>() + .collect() } /// Parses a physical window expr from a protobuf. diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 2b8bacc149d34..2cf581c824de4 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -538,11 +538,11 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { match expr_type { ExprType::AggregateExpr(agg_node) => { - let input_phy_expr: Vec> = agg_node.expr.iter() + let input_phy_expr = agg_node.expr.iter() .map(|e| parse_physical_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; - let ordering_req: LexOrdering = agg_node.ordering_req.iter() + let ordering_req = agg_node.ordering_req.iter() .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)) - .collect::>()?; + .collect::>>()?; agg_node.aggregate_function.as_ref().map(|func| { match func { AggregateFunction::UserDefinedAggrFunction(udaf_name) => { @@ -556,7 +556,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .alias(name) .with_ignore_nulls(agg_node.ignore_nulls) .with_distinct(agg_node.distinct) - .order_by(ordering_req) + .order_by((!ordering_req.is_empty()).then(|| ordering_req.into())) .build() .map(Arc::new) } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 829ec1698aebf..5a634ace5c296 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -539,7 +539,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { AggregateExprBuilder::new(array_agg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) .alias("ARRAY_AGG(b)") - .order_by(sort_exprs) + .order_by(Some(sort_exprs)) .build() .map(Arc::new)?, ]; From 0ba2db4aab8932419a22198ce9ff8cd1b6102d30 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 28 Mar 2025 14:39:09 +0300 Subject: [PATCH 036/167] Enforce non-degeneracy for LexOrdering (Part 2) --- datafusion/core/src/physical_planner.rs | 2 +- .../tests/fuzz_cases/equivalence/ordering.rs | 4 +- .../fuzz_cases/equivalence/projection.rs | 2 +- .../fuzz_cases/equivalence/properties.rs | 18 ++- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 7 +- .../core/tests/fuzz_cases/window_fuzz.rs | 16 +- .../physical_optimizer/enforce_sorting.rs | 59 +++----- .../tests/physical_optimizer/test_utils.rs | 18 ++- datafusion/datasource/src/memory.rs | 4 +- datafusion/ffi/src/plan_properties.rs | 16 +- .../functions-aggregate/src/first_last.rs | 16 +- .../physical-expr-common/src/sort_expr.rs | 3 +- .../physical-expr/src/equivalence/mod.rs | 12 +- .../physical-expr/src/equivalence/ordering.rs | 18 +-- .../src/equivalence/properties/dependency.rs | 21 ++- .../src/equivalence/properties/mod.rs | 48 ++++-- .../src/equivalence/properties/union.rs | 4 +- datafusion/physical-expr/src/physical_expr.rs | 4 +- .../physical-expr/src/window/aggregate.rs | 12 +- .../src/window/sliding_aggregate.rs | 36 +++-- .../physical-expr/src/window/standard.rs | 25 +-- .../physical-expr/src/window/window_expr.rs | 41 +++-- .../src/enforce_sorting/sort_pushdown.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 142 ++++++++---------- .../src/joins/nested_loop_join.rs | 8 +- .../physical-plan/src/repartition/mod.rs | 4 +- datafusion/physical-plan/src/sorts/sort.rs | 11 +- .../src/sorts/sort_preserving_merge.rs | 30 +--- datafusion/physical-plan/src/streaming.rs | 4 +- .../src/windows/bounded_window_agg_exec.rs | 13 +- datafusion/physical-plan/src/windows/mod.rs | 26 ++-- .../src/windows/window_agg_exec.rs | 10 +- .../proto/src/physical_plan/from_proto.rs | 8 +- datafusion/proto/src/physical_plan/mod.rs | 4 +- .../proto/src/physical_plan/to_proto.rs | 11 +- .../tests/cases/roundtrip_physical_plan.rs | 20 +-- 37 files changed, 347 insertions(+), 338 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 569d14d2c7161..26bea002655ce 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1552,7 +1552,7 @@ pub fn create_window_expr_with_name( name, &physical_args, &partition_by, - &order_by.into(), + (!order_by.is_empty()).then(|| order_by.into()), window_frame, physical_schema, ignore_nulls, diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 769deef1187d6..c1e7e4e553d5d 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -55,7 +55,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { col("f", &test_schema)?, ]; - for n_req in 0..=col_exprs.len() { + for n_req in 1..=col_exprs.len() { for exprs in col_exprs.iter().combinations(n_req) { let requirement = exprs .into_iter() @@ -127,7 +127,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { a_plus_b, ]; - for n_req in 0..=exprs.len() { + for n_req in 1..=exprs.len() { for exprs in exprs.iter().combinations(n_req) { let requirement = exprs .into_iter() diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index a3fa1157b38f4..b43e104b90246 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -164,7 +164,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { .map(|(_source, target)| Arc::clone(target)) .collect::>(); - for n_req in 0..=projected_exprs.len() { + for n_req in 1..=projected_exprs.len() { for exprs in projected_exprs.iter().combinations(n_req) { let requirement = exprs .into_iter() diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 93c6ce603eb1a..7362fb5b326b6 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -90,14 +90,16 @@ fn test_find_longest_permutation_random() -> Result<()> { assert_eq!(ordering.len(), indices.len(), "{}", err_msg); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). - assert!( - is_table_same_after_sort( - ordering.into(), - table_data_with_properties.clone(), - )?, - "{}", - err_msg - ); + if !ordering.is_empty() { + assert!( + is_table_same_after_sort( + ordering.into(), + table_data_with_properties.clone(), + )?, + "{}", + err_msg + ); + } } } } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index d4b41b6866315..89b5a7d60a0ba 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -114,7 +114,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti }; while !remaining_exprs.is_empty() { - let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + let n_sort_expr = rng.gen_range(1..remaining_exprs.len() + 1); remaining_exprs.shuffle(&mut rng); let ordering = remaining_exprs diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 75ccce6fb5aff..ca802211afcb2 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -93,7 +93,7 @@ mod sp_repartition_fuzz_tests { }; while !remaining_exprs.is_empty() { - let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + let n_sort_expr = rng.gen_range(1..remaining_exprs.len() + 1); remaining_exprs.shuffle(&mut rng); let ordering = remaining_exprs @@ -237,10 +237,7 @@ mod sp_repartition_fuzz_tests { .collect::>(); // Returns concatenated version of the all available orderings - let exprs = eq_properties - .oeq_class() - .output_ordering() - .unwrap_or_default(); + let exprs = eq_properties.oeq_class().output_ordering().unwrap(); let context = SessionContext::new().task_ctx(); let mem_reservation = diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index a7f9e38c9ae31..092b53bae2a51 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -252,7 +252,6 @@ async fn bounded_window_causal_non_causal() -> Result<()> { ]; let partitionby_exprs = vec![]; - let orderby_exprs = LexOrdering::default(); // Window frame starts with "UNBOUNDED PRECEDING": let start_bound = WindowFrameBound::Preceding(ScalarValue::UInt64(None)); @@ -285,7 +284,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { fn_name.to_string(), &args, &partitionby_exprs, - orderby_exprs.as_ref(), + None, Arc::new(window_frame), &extended_schema, false, @@ -594,7 +593,7 @@ async fn run_window_test( let ctx = SessionContext::new_with_config(session_config); let (window_fn, args, fn_name) = get_random_function(&schema, &mut rng, is_linear); let window_frame = get_random_window_frame(&mut rng, is_linear); - let mut orderby_exprs = LexOrdering::default(); + let mut orderby_exprs = vec![]; for column in &orderby_columns { orderby_exprs.push(PhysicalSortExpr { expr: col(column, &schema)?, @@ -602,13 +601,13 @@ async fn run_window_test( }) } if orderby_exprs.len() > 1 && !window_frame.can_accept_multi_orderby() { - orderby_exprs = LexOrdering::new(orderby_exprs[0..1].to_vec()); + orderby_exprs = orderby_exprs[0..1].to_vec(); } let mut partitionby_exprs = vec![]; for column in &partition_by_columns { partitionby_exprs.push(col(column, &schema)?); } - let mut sort_keys = LexOrdering::default(); + let mut sort_keys = vec![]; for partition_by_expr in &partitionby_exprs { sort_keys.push(PhysicalSortExpr { expr: partition_by_expr.clone(), @@ -643,18 +642,19 @@ async fn run_window_test( // Table is ordered according to ORDER BY a, b, c In linear test we use PARTITION BY b, ORDER BY a // For WindowAggExec to produce correct result it need table to be ordered by b,a. Hence add a sort. if is_linear { - exec1 = Arc::new(SortExec::new(sort_keys, exec1)) as _; + exec1 = Arc::new(SortExec::new(sort_keys.into(), exec1)) as _; } let extended_schema = schema_add_window_field(&args, &schema, &window_fn, &fn_name)?; + let orderby = (!orderby_exprs.is_empty()).then(|| orderby_exprs.into()); let usual_window_exec = Arc::new(WindowAggExec::try_new( vec![create_window_expr( &window_fn, fn_name.clone(), &args, &partitionby_exprs, - orderby_exprs.as_ref(), + orderby.clone(), Arc::new(window_frame.clone()), &extended_schema, false, @@ -672,7 +672,7 @@ async fn run_window_test( fn_name, &args, &partitionby_exprs, - orderby_exprs.as_ref(), + orderby, Arc::new(window_frame.clone()), &extended_schema, false, diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 34306280f14a0..fca2ccc13431c 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -62,39 +62,23 @@ use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use rstest::rstest; -/// Create a csv exec for tests -fn csv_exec_ordered( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - Arc::new(CsvSource::new(true, 0, b'"')), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]) - .build() -} - /// Created a sorted parquet exec -pub fn parquet_exec_sorted( +fn parquet_exec_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - + let sort_exprs = sort_exprs.into_iter().collect::>(); let source = Arc::new(ParquetSource::default()); - FileScanConfig::new( + let mut builder = FileScanConfig::new( ObjectStoreUrl::parse("test:///").unwrap(), schema.clone(), source, ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]) - .build() + .with_file(PartitionedFile::new("x".to_string(), 100)); + if !sort_exprs.is_empty() { + builder = builder.with_output_ordering(vec![sort_exprs.into()]); + } + builder.build() } /// Create a sorted Csv exec @@ -102,16 +86,17 @@ fn csv_exec_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - FileScanConfig::new( + let sort_exprs = sort_exprs.into_iter().collect::>(); + let mut builder = FileScanConfig::new( ObjectStoreUrl::parse("test:///").unwrap(), schema.clone(), Arc::new(CsvSource::new(false, 0, 0)), ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]) - .build() + .with_file(PartitionedFile::new("x".to_string(), 100)); + if !sort_exprs.is_empty() { + builder = builder.with_output_ordering(vec![sort_exprs.into()]); + } + builder.build() } /// Runs the sort enforcement optimizer and asserts the plan @@ -1461,7 +1446,7 @@ async fn test_with_lost_ordering_unbounded_bounded( let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_ordered(&schema, sort_exprs) + csv_exec_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( @@ -1484,7 +1469,7 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false", ]; // Expected unbounded result (same for with and without flag) @@ -1501,14 +1486,14 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false", ]; let expected_optimized_bounded_parallelize_sort = vec![ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false", ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = if source_unbounded { @@ -2400,7 +2385,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { let plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(sort_exprs_a) + .with_required_input_ordering(Some(sort_exprs_a)) .with_maintains_input_order(true) .into_arc(); let plan = sort_exec(sort_exprs_b, plan); @@ -2431,7 +2416,7 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { let plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(sort_exprs_a) + .with_required_input_ordering(Some(sort_exprs_a)) .with_maintains_input_order(true) .into_arc(); let plan = sort_exec(sort_exprs_ab, plan); @@ -3712,7 +3697,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { case.func.1, &case.func.2, &partition_by, - &LexOrdering::default(), + None, case.window_frame, input_schema.as_ref(), false, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 00011c8d6de0a..7f68cb4183284 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -205,14 +205,14 @@ pub fn bounded_window_exec_with_partition( partition_by: &[Arc], input: Arc, ) -> Arc { - let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); + let sort_exprs = sort_exprs.into_iter().collect::>(); let schema = input.schema(); let window_expr = create_window_expr( &WindowFunctionDefinition::AggregateUDF(count_udaf()), "count".to_owned(), &[col(col_name, &schema).unwrap()], partition_by, - sort_exprs.as_ref(), + (!sort_exprs.is_empty()).then(|| sort_exprs.into()), Arc::new(WindowFrame::new(Some(false))), schema.as_ref(), false, @@ -327,7 +327,7 @@ pub fn projection_exec( /// A test [`ExecutionPlan`] whose requirements can be configured. #[derive(Debug)] pub struct RequirementsTestExec { - required_input_ordering: LexOrdering, + required_input_ordering: Option, maintains_input_order: bool, input: Arc, } @@ -335,7 +335,7 @@ pub struct RequirementsTestExec { impl RequirementsTestExec { pub fn new(input: Arc) -> Self { Self { - required_input_ordering: LexOrdering::default(), + required_input_ordering: None, maintains_input_order: true, input, } @@ -344,7 +344,7 @@ impl RequirementsTestExec { /// sets the required input ordering pub fn with_required_input_ordering( mut self, - required_input_ordering: LexOrdering, + required_input_ordering: Option, ) -> Self { self.required_input_ordering = required_input_ordering; self @@ -390,9 +390,11 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - vec![Some(RequiredInputOrdering::from( - self.required_input_ordering.clone(), - ))] + if let Some(ordering) = &self.required_input_ordering { + vec![Some(RequiredInputOrdering::from(ordering.clone()))] + } else { + vec![None] + } } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 64fd56971b292..14873d7560242 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -755,7 +755,7 @@ mod memory_source_tests { expr: col("c", &schema)?, options: SortOptions::default(), }]); - let mut expected_output_order = LexOrdering::default(); + let mut expected_output_order = vec![]; expected_output_order.extend(sort1.clone()); expected_output_order.extend(sort2.clone()); @@ -767,7 +767,7 @@ mod memory_source_tests { assert_eq!( mem_exec.properties().output_ordering().unwrap(), - &expected_output_order + &expected_output_order.into() ); let eq_properties = mem_exec.properties().equivalence_properties(); assert!(eq_properties.oeq_class().contains(&sort1)); diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 3592c16b8fab0..568cacc5a2694 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -188,12 +188,12 @@ impl TryFrom for PlanProperties { let proto_output_ordering = PhysicalSortExprNodeCollection::decode(df_result!(ffi_orderings)?.as_ref()) .map_err(|e| DataFusionError::External(Box::new(e)))?; - let orderings = Some(parse_physical_sort_exprs( + let sort_exprs = parse_physical_sort_exprs( &proto_output_ordering.physical_sort_expr_nodes, &default_ctx, &schema, &codex, - )?); + )?; let partitioning_vec = unsafe { df_result!((ffi_props.output_partitioning)(&ffi_props))? }; @@ -211,11 +211,13 @@ impl TryFrom for PlanProperties { .to_string(), ))?; - let eq_properties = match orderings { - Some(ordering) => { - EquivalenceProperties::new_with_orderings(Arc::new(schema), &[ordering]) - } - None => EquivalenceProperties::new(Arc::new(schema)), + let eq_properties = if sort_exprs.is_empty() { + EquivalenceProperties::new(Arc::new(schema)) + } else { + EquivalenceProperties::new_with_orderings( + Arc::new(schema), + &[sort_exprs.into()], + ) }; let emission_type: EmissionType = diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 3dcc18fd4eb3e..bb312a8fb0773 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -573,8 +573,7 @@ impl Accumulator for TrivialLastValueAccumulator { // LAST_VALUE(last1, last2, last3, ...) // Second index contains is_set flag. let flags = states[1].as_boolean(); - let filtered_states = - filter_states_according_to_is_set(&states[0..1], flags)?; + let filtered_states = filter_states_according_to_is_set(&states[0..1], flags)?; // If we have any values, take the last one we find: if let Some(last) = filtered_states.last() { self.last = ScalarValue::try_from_array(last, 0)?; @@ -674,12 +673,13 @@ impl Accumulator for LastValueAccumulator { let row = get_row_at_idx(values, last_idx)?; let orderings = &row[1..]; // Update when there is a more recent entry - if !self.is_set || compare_rows( - &self.orderings, - orderings, - &get_sort_options(self.ordering_req.as_ref()), - )? - .is_lt() + if !self.is_set + || compare_rows( + &self.orderings, + orderings, + &get_sort_options(self.ordering_req.as_ref()), + )? + .is_lt() { self.update_with_new_row(&row); } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 5ac55fc62d597..2087a0d91c752 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -335,7 +335,7 @@ fn to_str(options: &SortOptions) -> &str { /// For example, `vec![a ASC, b DESC]` represents a lexicographical ordering /// that first sorts by column `a` in ascending order, then by column `b` in /// descending order. -#[derive(Debug, Default, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LexOrdering { inner: Vec, } @@ -349,6 +349,7 @@ impl AsRef for LexOrdering { impl LexOrdering { /// Creates a new [`LexOrdering`] from a vector pub fn new(inner: Vec) -> Self { + debug_assert!(!inner.is_empty()); Self { inner } } diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index b2cdd29881f97..8c28fd96d2e38 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -188,19 +188,19 @@ mod tests { Ok((test_schema, eq_properties)) } - // Convert each tuple to PhysicalSortRequirement + // Convert each tuple to a `PhysicalSortRequirement` and construct a + // a `LexRequirement` from them. pub fn convert_to_sort_reqs( - in_data: &[(&Arc, Option)], + args: &[(&Arc, Option)], ) -> LexRequirement { - in_data - .iter() + args.iter() .map(|(expr, options)| { PhysicalSortRequirement::new(Arc::clone(*expr), *options) }) .collect() } - // Convert each tuple to PhysicalSortExpr + // Convert each tuple to a `PhysicalSortExpr` and construct a vector. pub fn convert_to_sort_exprs>>( args: &[(T, SortOptions)], ) -> Vec { @@ -212,7 +212,7 @@ mod tests { .collect() } - // Convert each inner tuple to PhysicalSortExpr and then into LexOrdering + // Convert each vector of tuples to a `LexOrdering`. pub fn convert_to_orderings>>( args: &[Vec<(T, SortOptions)>], ) -> Vec { diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index fbeab0c38cdd3..00d74f897bd7b 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -181,14 +181,14 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - let output_ordering = self - .orderings - .iter() - .flatten() - .cloned() - .collect::() - .collapse(); - (!output_ordering.is_empty()).then_some(output_ordering) + (!self.orderings.is_empty()).then(|| { + self.orderings + .iter() + .flatten() + .cloned() + .collect::() + .collapse() + }) } // Append orderings in `other` to all existing orderings in this equivalence @@ -856,7 +856,7 @@ mod tests { // ------- TEST CASE 5 --------- // Empty ordering ( - vec![vec![]], + vec![], // No ordering in the state (empty ordering is ignored). vec![], ), diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index fa75f06377fd9..dcf3378f2da8e 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -397,13 +397,10 @@ pub fn generate_dependency_orderings( (!prefixes.is_empty()).then_some(prefixes) }) .collect::>(); + // Note that if relevant prefixes are empty, there is no dependency, + // meaning that dependent is a leading ordering. - // No dependency, dependent is a leading ordering. - if relevant_prefixes.is_empty() { - // Return an empty ordering: - return vec![LexOrdering::default()]; - } - + // Generate all possible valid orderings: relevant_prefixes .into_iter() .multi_cartesian_product() @@ -411,11 +408,13 @@ pub fn generate_dependency_orderings( prefix_orderings .iter() .permutations(prefix_orderings.len()) - .map(|prefixes| { - prefixes - .into_iter() - .flat_map(|ordering| ordering.clone()) - .collect() + .filter_map(|prefixes| { + (!prefixes.is_empty()).then(|| { + prefixes + .into_iter() + .flat_map(|ordering| ordering.clone()) + .collect() + }) }) .collect::>() }) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 8a3b7320d7f68..4629505d608a3 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -226,7 +226,7 @@ impl EquivalenceProperties { OrderingEquivalenceClass::new( self.oeq_class .iter() - .map(|ordering| self.normalize_sort_exprs(ordering)) + .filter_map(|ordering| self.normalize_sort_exprs(ordering)) .collect(), ) } @@ -497,7 +497,7 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { + pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> Option { let normalized_sort_exprs = self.eq_group.normalize_sort_exprs(sort_exprs); let mut constant_exprs = vec![]; constant_exprs.extend( @@ -507,12 +507,12 @@ impl EquivalenceProperties { ); let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); // Prune redundant sections in the ordering: - normalized_sort_exprs + let sort_exprs = normalized_sort_exprs .iter() .filter(|&order| !physical_exprs_contains(&constants_normalized, &order.expr)) .cloned() - .collect::() - .collapse() + .collect::>(); + (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the @@ -552,8 +552,12 @@ impl EquivalenceProperties { /// orderings. pub fn ordering_satisfy(&self, given: &LexOrdering) -> bool { // First, standardize the given ordering: - let normalized_ordering = self.normalize_sort_exprs(given); - // Check whether given ordering is satisfied by constraints first: + let Some(normalized_ordering) = self.normalize_sort_exprs(given) else { + // If the requirement vanishes after normalization, it is satisfied + // by any ordering. + return true; + }; + // Then, check whether given ordering is satisfied by constraints: if self.satisfied_by_constraints_ordering(&normalized_ordering) { return true; } @@ -810,8 +814,12 @@ impl EquivalenceProperties { lhs: &LexOrdering, rhs: &LexOrdering, ) -> Option { - let mut lhs = self.normalize_sort_exprs(lhs); - let mut rhs = self.normalize_sort_exprs(rhs); + let Some(mut rhs) = self.normalize_sort_exprs(rhs) else { + return self.normalize_sort_exprs(lhs); + }; + let Some(mut lhs) = self.normalize_sort_exprs(lhs) else { + return Some(rhs); + }; lhs.iter_mut() .zip(rhs.iter_mut()) .all(|(lhs, rhs)| lhs.expr.eq(&rhs.expr) && lhs.options == rhs.options) @@ -1083,8 +1091,13 @@ impl EquivalenceProperties { let mut dependency_orderings = generate_dependency_orderings(&relevant_deps, &dependency_map); // Append `sort_expr` to the dependent orderings: - for ordering in dependency_orderings.iter_mut() { - ordering.push(sort_expr.clone()); + if dependency_orderings.is_empty() { + dependency_orderings + .push(LexOrdering::new(vec![sort_expr.clone()])); + } else { + for ordering in dependency_orderings.iter_mut() { + ordering.push(sort_expr.clone()); + } } dependency_orderings }) @@ -1100,12 +1113,15 @@ impl EquivalenceProperties { if prefixes.is_empty() { // If prefix is empty, there is no dependency. Insert // empty ordering: - prefixes = vec![LexOrdering::default()]; - } - // Append current ordering on top its dependencies: - for ordering in prefixes.iter_mut() { if let Some(target) = &node.target_sort_expr { - ordering.push(target.clone()) + prefixes.push(LexOrdering::new(vec![target.clone()])); + } + } else { + // Append current ordering on top its dependencies: + for ordering in prefixes.iter_mut() { + if let Some(target) = &node.target_sort_expr { + ordering.push(target.clone()) + } } } prefixes diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 64ef9278e248b..78edf4af21b5a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -247,7 +247,7 @@ impl UnionEquivalentOrderingBuilder { existing_ordering: &LexOrdering, existing_constants: &[ConstExpr], ) -> Option { - let mut augmented_ordering = LexOrdering::default(); + let mut augmented_ordering = vec![]; let mut sort_expr_iter = ordering.iter().peekable(); let mut existing_sort_expr_iter = existing_ordering.iter().peekable(); @@ -274,7 +274,7 @@ impl UnionEquivalentOrderingBuilder { } } - Some(augmented_ordering) + (!augmented_ordering.is_empty()).then(|| LexOrdering::new(augmented_ordering)) } fn build(self) -> Vec { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 2221bc980f6ce..a01a8c3cdac19 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -112,7 +112,7 @@ pub fn create_ordering( for (group_idx, exprs) in sort_order.iter().enumerate() { // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = LexOrdering::default(); + let mut sort_exprs = vec![]; for (expr_idx, sort) in exprs.iter().enumerate() { match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { @@ -140,7 +140,7 @@ pub fn create_ordering( } } if !sort_exprs.is_empty() { - all_sort_orders.push(sort_exprs); + all_sort_orders.push(LexOrdering::new(sort_exprs)); } } Ok(all_sort_orders) diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index a94d5b1212f52..4a0218e6e2a3e 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -43,7 +43,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; pub struct PlainAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: LexOrdering, + order_by: Option, window_frame: Arc, } @@ -52,13 +52,13 @@ impl PlainAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &LexOrdering, + order_by: Option, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.clone(), + order_by, window_frame, } } @@ -141,7 +141,7 @@ impl WindowExpr for PlainAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &LexOrdering { + fn order_by(&self) -> Option<&LexOrdering> { self.order_by.as_ref() } @@ -156,14 +156,14 @@ impl WindowExpr for PlainAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), + self.order_by.as_ref().map(reverse_order_bys), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), + self.order_by.as_ref().map(reverse_order_bys), Arc::new(self.window_frame.reverse()), )) as _ } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 23967e78f07a7..b775d4094bb7e 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -43,7 +43,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; pub struct SlidingAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: LexOrdering, + order_by: Option, window_frame: Arc, } @@ -52,7 +52,7 @@ impl SlidingAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &LexOrdering, + order_by: Option, window_frame: Arc, ) -> Self { Self { @@ -108,7 +108,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &LexOrdering { + fn order_by(&self) -> Option<&LexOrdering> { self.order_by.as_ref() } @@ -123,14 +123,14 @@ impl WindowExpr for SlidingAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), + self.order_by.as_ref().map(reverse_order_bys), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), + self.order_by.as_ref().map(reverse_order_bys), Arc::new(self.window_frame.reverse()), )) as _ } @@ -147,17 +147,21 @@ impl WindowExpr for SlidingAggregateWindowExpr { partition_bys: Vec>, order_by_exprs: Vec>, ) -> Option> { - debug_assert_eq!(self.order_by.len(), order_by_exprs.len()); - - let new_order_by = self - .order_by - .iter() - .zip(order_by_exprs) - .map(|(req, new_expr)| PhysicalSortExpr { - expr: new_expr, - options: req.options, - }) - .collect::(); + debug_assert!(self.order_by.as_ref().map_or_else( + || order_by_exprs.is_empty(), + |old_exprs| old_exprs.len() == order_by_exprs.len() + )); + + let new_order_by = self.order_by().map(|ob_exprs| { + ob_exprs + .iter() + .zip(order_by_exprs) + .map(|(sort_expr, new_expr)| PhysicalSortExpr { + expr: new_expr, + options: sort_expr.options, + }) + .collect::() + }); Some(Arc::new(SlidingAggregateWindowExpr { aggregate: self .aggregate diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 042041d5a3ed9..1430ff676644f 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -25,8 +25,8 @@ use super::{StandardWindowFunctionExpr, WindowExpr}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; + use arrow::array::{new_empty_array, ArrayRef}; -use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; use datafusion_common::utils::evaluate_partition_ranges; @@ -40,7 +40,7 @@ use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; pub struct StandardWindowExpr { expr: Arc, partition_by: Vec>, - order_by: LexOrdering, + order_by: Option, window_frame: Arc, } @@ -49,13 +49,13 @@ impl StandardWindowExpr { pub fn new( expr: Arc, partition_by: &[Arc], - order_by: &LexOrdering, + order_by: Option, window_frame: Arc, ) -> Self { Self { expr, partition_by: partition_by.to_vec(), - order_by: order_by.clone(), + order_by, window_frame, } } @@ -104,7 +104,7 @@ impl WindowExpr for StandardWindowExpr { &self.partition_by } - fn order_by(&self) -> &LexOrdering { + fn order_by(&self) -> Option<&LexOrdering> { self.order_by.as_ref() } @@ -112,8 +112,11 @@ impl WindowExpr for StandardWindowExpr { let mut evaluator = self.expr.create_evaluator()?; let num_rows = batch.num_rows(); if evaluator.uses_window_frame() { - let sort_options: Vec = - self.order_by.iter().map(|o| o.options).collect(); + let sort_options = if let Some(ob) = &self.order_by { + ob.iter().map(|o| o.options).collect() + } else { + vec![] + }; let mut row_wise_results = vec![]; let mut values = self.evaluate_args(batch)?; @@ -157,7 +160,11 @@ impl WindowExpr for StandardWindowExpr { ) -> Result<()> { let field = self.expr.field()?; let out_type = field.data_type(); - let sort_options = self.order_by.iter().map(|o| o.options).collect::>(); + let sort_options = if let Some(ob) = &self.order_by { + ob.iter().map(|o| o.options).collect() + } else { + vec![] + }; for (partition_row, partition_batch_state) in partition_batches.iter() { let window_state = if let Some(window_state) = window_agg_state.get_mut(partition_row) { @@ -253,7 +260,7 @@ impl WindowExpr for StandardWindowExpr { Arc::new(StandardWindowExpr::new( reverse_expr, &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), + self.order_by.as_ref().map(reverse_order_bys), Arc::new(self.window_frame.reverse()), )) as _ }) diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 793f2e5ee5867..a92dea6ee7297 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -109,14 +109,17 @@ pub trait WindowExpr: Send + Sync + Debug { fn partition_by(&self) -> &[Arc]; /// Expressions that's from the window function's order by clause, empty if absent - fn order_by(&self) -> &LexOrdering; + fn order_by(&self) -> Option<&LexOrdering>; /// Get order by columns, empty if absent fn order_by_columns(&self, batch: &RecordBatch) -> Result> { - self.order_by() + let Some(order_bys) = self.order_by() else { + return Ok(vec![]); + }; + order_bys .iter() .map(|e| e.evaluate_to_sort_column(batch)) - .collect::>>() + .collect() } /// Get the window frame of this [WindowExpr]. @@ -134,11 +137,13 @@ pub trait WindowExpr: Send + Sync + Debug { fn all_expressions(&self) -> WindowPhysicalExpressions { let args = self.expressions(); let partition_by_exprs = self.partition_by().to_vec(); - let order_by_exprs = self - .order_by() - .iter() - .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect::>(); + let order_by_exprs = if let Some(ob) = self.order_by() { + ob.iter() + .map(|sort_expr| Arc::clone(&sort_expr.expr)) + .collect() + } else { + vec![] + }; WindowPhysicalExpressions { args, partition_by_exprs, @@ -190,8 +195,11 @@ pub trait AggregateWindowExpr: WindowExpr { fn aggregate_evaluate(&self, batch: &RecordBatch) -> Result { let mut accumulator = self.get_accumulator()?; let mut last_range = Range { start: 0, end: 0 }; - let sort_options: Vec = - self.order_by().iter().map(|o| o.options).collect(); + let sort_options = if let Some(ob) = self.order_by() { + ob.iter().map(|sort_expr| sort_expr.options).collect() + } else { + vec![] + }; let mut window_frame_ctx = WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options); self.get_result_column( @@ -239,8 +247,11 @@ pub trait AggregateWindowExpr: WindowExpr { // If there is no window state context, initialize it. let window_frame_ctx = state.window_frame_ctx.get_or_insert_with(|| { - let sort_options: Vec = - self.order_by().iter().map(|o| o.options).collect(); + let sort_options = if let Some(ob) = self.order_by() { + ob.iter().map(|sort_expr| sort_expr.options).collect() + } else { + vec![] + }; WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options) }); let out_col = self.get_result_column( @@ -344,13 +355,13 @@ pub(crate) fn is_end_bound_safe( window_frame_ctx: &WindowFrameContext, order_bys: &[ArrayRef], most_recent_order_bys: Option<&[ArrayRef]>, - sort_exprs: &LexOrdering, + sort_exprs: Option<&LexOrdering>, idx: usize, ) -> Result { - if sort_exprs.is_empty() { + let Some(sort_exprs) = sort_exprs else { // Early return if no sort expressions are present: return Ok(false); - } + }; match window_frame_ctx { WindowFrameContext::Rows(window_frame) => { diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 6adf30f4c09a7..a6eac4dd03121 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -468,8 +468,10 @@ fn try_pushdown_requirements_to_join( Some(probe_side), ); let mut smj_eqs = smj.properties().equivalence_properties().clone(); - // smj will have this ordering when its input changes. - smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); + if let Some(new_output_ordering) = new_output_ordering { + // smj will have this ordering when its input changes. + smj_eqs = smj_eqs.with_reorder(new_output_ordering); + } let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 384125ebbcdf0..55aadd83d9d1b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1045,15 +1045,14 @@ fn get_aggregate_expr_req( aggr_expr: &AggregateFunctionExpr, group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, -) -> LexOrdering { +) -> Option { // If the aggregation function is ordering requirement is not absolutely // necessary, or the aggregation is performing a "second stage" calculation, // then ignore the ordering requirement. if !aggr_expr.order_sensitivity().hard_requires() || !agg_mode.is_first_stage() { - return LexOrdering::default(); + return None; } - - let mut req = aggr_expr.order_bys().cloned().unwrap_or_default(); + let mut req = aggr_expr.order_bys().cloned()?.to_vec(); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we @@ -1069,34 +1068,7 @@ fn get_aggregate_expr_req( !physical_exprs_contains(&physical_exprs, &sort_expr.expr) }); } - req -} - -/// Computes the finer ordering for between given existing ordering requirement -/// of aggregate expression. -/// -/// # Parameters -/// -/// * `existing_req` - The existing lexical ordering that needs refinement. -/// * `aggr_expr` - A reference to an aggregate expression trait object. -/// * `group_by` - Information about the physical grouping (e.g group by expression). -/// * `eq_properties` - Equivalence properties relevant to the computation. -/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). -/// -/// # Returns -/// -/// An `Option` representing the computed finer lexical ordering, -/// or `None` if there is no finer ordering; e.g. the existing requirement and -/// the aggregator requirement is incompatible. -fn finer_ordering( - existing_req: &LexOrdering, - aggr_expr: &AggregateFunctionExpr, - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Option { - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req, aggr_req.as_ref()) + (!req.is_empty()).then(|| req.into()) } /// Concatenates the given slices. @@ -1128,67 +1100,79 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - let mut requirement = LexOrdering::default(); + let mut requirement = None; for aggr_expr in aggr_exprs.iter_mut() { - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + let (mut conflict, mut conflict_rev) = (false, false); + let mut finer = if let Some(aggr_req) = + get_aggregate_expr_req(aggr_expr, group_by, agg_mode) { - if eq_properties.ordering_satisfy(finer_ordering.as_ref()) { - // Requirement is satisfied by existing ordering - requirement = finer_ordering; - continue; + if let Some(req) = requirement.as_ref() { + let finer = eq_properties.get_finer_ordering(req, &aggr_req); + conflict = finer.is_none(); + finer + } else { + eq_properties.normalize_sort_exprs(&aggr_req) } + } else { + requirement.clone() + }; + if let Some(finer_ordering) = finer.take_if(|o| eq_properties.ordering_satisfy(o)) + { + // Requirement is satisfied by the existing ordering: + requirement = Some(finer_ordering); + continue; } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - if eq_properties.ordering_satisfy(finer_ordering.as_ref()) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - requirement = finer_ordering; - *aggr_expr = Arc::new(reverse_aggr_expr); - continue; + let mut finer_rev = None; + let mut rev_expr = aggr_expr.reverse_expr(); + if let Some(reverse_aggr_expr) = rev_expr.take() { + finer_rev = if let Some(aggr_req) = + get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) + { + if let Some(req) = requirement.as_ref() { + let finer = eq_properties.get_finer_ordering(req, &aggr_req); + conflict_rev = finer.is_none(); + finer + } else { + eq_properties.normalize_sort_exprs(&aggr_req) } + } else { + requirement.clone() + }; + if let Some(finer_ordering) = + finer_rev.take_if(|o| eq_properties.ordering_satisfy(o)) + { + // Reverse requirement is satisfied by the existing ordering. + // Hence, we need to reverse the aggregate expression: + requirement = Some(finer_ordering); + *aggr_expr = Arc::new(reverse_aggr_expr); + continue; } + let _ = rev_expr.insert(reverse_aggr_expr); } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current - // aggregate requirement. Use updated requirement - requirement = finer_ordering; + if finer.is_some() { + // There is a requirement that satisfies both the existing requirement + // and the aggregate requirement. Use updated requirement: + requirement = finer; continue; - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse - // aggregate requirement. Use updated requirement - requirement = finer_ordering; + } else if let Some(reverse_aggr_expr) = rev_expr { + if finer_rev.is_some() { + // There is a requirement that satisfies both the existing requirement + // and the reverse aggregate requirement. Use updated requirement: *aggr_expr = Arc::new(reverse_aggr_expr); + requirement = finer_rev; continue; } + } else if conflict && conflict_rev { + // Neither the existing requirement nor the aggregate requirement + // satisfy the other, this means requirements are conflicting. + // Currently, we do not support conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); } - - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); } - Ok(requirement.into_iter().map(Into::into).collect()) + Ok(requirement.map_or_else(Vec::new, |o| o.into_iter().map(Into::into).collect())) } /// Returns physical expressions for arguments to evaluate against a batch. diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 88d3ea9e7e1ee..373d83b982d71 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1052,7 +1052,7 @@ pub(crate) mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use rstest::rstest; @@ -1082,7 +1082,7 @@ pub(crate) mod tests { let mut source = TestMemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { - let mut sort_info = LexOrdering::default(); + let mut sort_info = vec![]; for name in sorted_column_names { let index = schema.index_of(name).unwrap(); let sort_expr = PhysicalSortExpr { @@ -1094,7 +1094,9 @@ pub(crate) mod tests { }; sort_info.push(sort_expr); } - source = source.try_with_sort_information(vec![sort_info]).unwrap(); + source = source + .try_with_sort_information(vec![sort_info.into()]) + .unwrap(); } Arc::new(TestMemoryExec::update_cache(Arc::new(source))) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index df5d366148af0..93b1b4df3a023 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -583,7 +583,7 @@ impl ExecutionPlan for RepartitionExec { let schema_captured = Arc::clone(&schema); // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); + let sort_exprs = self.sort_exprs().cloned(); let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); @@ -650,7 +650,7 @@ impl ExecutionPlan for RepartitionExec { StreamingMergeBuilder::new() .with_streams(input_streams) .with_schema(schema_captured) - .with_expressions(&sort_exprs) + .with_expressions(&sort_exprs.unwrap()) .with_metrics(BaselineMetrics::new(&metrics, partition)) .with_batch_size(context.session_config().batch_size()) .with_fetch(fetch) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 3d2323eb43367..e39834b5c76f7 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1171,7 +1171,7 @@ impl ExecutionPlan for SortExec { return Ok(None); } - let mut updated_exprs = LexOrdering::default(); + let mut updated_exprs = vec![]; for sort in self.expr() { let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { @@ -1184,9 +1184,12 @@ impl ExecutionPlan for SortExec { } Ok(Some(Arc::new( - SortExec::new(updated_exprs, make_with_child(projection, self.input())?) - .with_fetch(self.fetch()) - .with_preserve_partitioning(self.preserve_partitioning()), + SortExec::new( + updated_exprs.into(), + make_with_child(projection, self.input())?, + ) + .with_fetch(self.fetch()) + .with_preserve_partitioning(self.preserve_partitioning()), ))) } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 26455a94b683f..cbb4f0eec20e0 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -363,7 +363,7 @@ impl ExecutionPlan for SortPreservingMergeExec { return Ok(None); } - let mut updated_exprs = LexOrdering::default(); + let mut updated_exprs = vec![]; for sort in self.expr() { let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? else { @@ -377,7 +377,7 @@ impl ExecutionPlan for SortPreservingMergeExec { Ok(Some(Arc::new( SortPreservingMergeExec::new( - updated_exprs, + updated_exprs.into(), make_with_child(projection, self.input())?, ) .with_fetch(self.fetch()), @@ -413,7 +413,7 @@ mod tests { }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; + use datafusion_common::{assert_batches_eq, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; @@ -549,30 +549,6 @@ mod tests { .await; } - #[tokio::test] - async fn test_merge_no_exprs() { - let task_ctx = Arc::new(TaskContext::default()); - let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); - let batch = RecordBatch::try_from_iter(vec![("a", a)]).unwrap(); - - let schema = batch.schema(); - let sort = LexOrdering::default(); // no sort expressions - let exec = TestMemoryExec::try_new_exec( - &[vec![batch.clone()], vec![batch]], - schema, - None, - ) - .unwrap(); - - let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); - - let res = collect(merge, task_ctx).await.unwrap_err(); - assert_contains!( - res.to_string(), - "Internal error: Sort expressions cannot be empty for streaming merge" - ); - } - #[tokio::test] async fn test_merge_some_overlap() { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 18c472a7e1874..455d4ecfc846d 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -307,7 +307,7 @@ impl ExecutionPlan for StreamingTableExec { let mut lex_orderings = vec![]; for lex_ordering in self.projected_output_ordering().into_iter() { - let mut orderings = LexOrdering::default(); + let mut orderings = vec![]; for order in lex_ordering { let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? @@ -319,7 +319,7 @@ impl ExecutionPlan for StreamingTableExec { options: order.options, }); } - lex_orderings.push(orderings); + lex_orderings.push(orderings.into()); } StreamingTableExec::try_new( diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 28ac693af45de..6119a98e31771 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -294,7 +294,10 @@ impl ExecutionPlan for BoundedWindowAggExec { .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements( + partition_bys, + order_keys.map_or_else(Vec::new, |o| o.to_vec()), + )] } fn required_input_distribution(&self) -> Vec { @@ -1360,7 +1363,7 @@ mod tests { fn_name, &args, &partitionby_exprs, - orderby_exprs.as_ref(), + Some(orderby_exprs), Arc::new(window_frame), &input.schema(), false, @@ -1614,7 +1617,7 @@ mod tests { Arc::new(StandardWindowExpr::new( last_value_func, &[], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1625,7 +1628,7 @@ mod tests { Arc::new(StandardWindowExpr::new( nth_value_func1, &[], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1636,7 +1639,7 @@ mod tests { Arc::new(StandardWindowExpr::new( nth_value_func2, &[], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 683d32bb9ccac..58cc9ae820ca2 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -101,7 +101,7 @@ pub fn create_window_expr( name: String, args: &[Arc], partition_by: &[Arc], - order_by: &LexOrdering, + order_by: Option, window_frame: Arc, input_schema: &Schema, ignore_nulls: bool, @@ -133,7 +133,7 @@ pub fn create_window_expr( /// Creates an appropriate [`WindowExpr`] based on the window frame and fn window_expr_from_aggregate_expr( partition_by: &[Arc], - order_by: &LexOrdering, + order_by: Option, window_frame: Arc, aggregate: Arc, ) -> Arc { @@ -374,6 +374,7 @@ pub(crate) fn window_equivalence_properties( .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))); let all_satisfied_lexs = partition_by_orders .multi_cartesian_product() + .filter(|lex| !lex.is_empty()) .map(LexOrdering::new) .filter(|lex| window_eq_properties.ordering_satisfy(lex)) .collect::>(); @@ -593,7 +594,7 @@ pub fn get_best_fitting_window( /// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], - orderby_keys: &LexOrdering, + orderby_keys: Option<&LexOrdering>, input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); @@ -608,11 +609,14 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); - let reverse_orderby_keys = reverse_order_bys(orderby_keys); - for (should_swap, orderbys) in [(false, orderby_keys), (true, &reverse_orderby_keys)] + let reverse_orderby_keys = orderby_keys.map(reverse_order_bys); + for (should_swap, orderbys) in + [(false, orderby_keys), (true, reverse_orderby_keys.as_ref())] { let mut req = partition_by_reqs.clone(); - req.extend(orderbys.iter().cloned().map(Into::into)); + if let Some(ob) = orderbys { + req.extend(ob.iter().cloned().map(Into::into)); + } if req.is_empty() || partition_by_eqs .ordering_satisfy_requirement(&LexRequirement::new(req).collapse()) @@ -817,7 +821,7 @@ mod tests { "count".to_owned(), &[col("a", &schema)?], &[], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new(None)), schema.as_ref(), false, @@ -1014,7 +1018,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = LexOrdering::default(); + let mut order_by_exprs = vec![]; for col_name in order_by_params { let expr = col(col_name, &test_schema)?; // Give default ordering, this is same with input ordering direction @@ -1024,7 +1028,7 @@ mod tests { } let res = get_window_mode( &partition_by_exprs, - order_by_exprs.as_ref(), + Some(&order_by_exprs.into()), &exec_unbounded, ); // Since reversibility is not important in this test. Convert Option<(bool, InputOrderMode)> to Option @@ -1179,7 +1183,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = LexOrdering::default(); + let mut order_by_exprs = vec![]; for (col_name, descending, nulls_first) in order_by_params { let expr = col(col_name, &test_schema)?; let options = SortOptions { @@ -1190,7 +1194,7 @@ mod tests { } assert_eq!( - get_window_mode(&partition_by_exprs, order_by_exprs.as_ref(), &exec_unbounded), + get_window_mode(&partition_by_exprs, Some(&order_by_exprs.into()), &exec_unbounded), *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index b95347ec600ed..14b7ad543b85a 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -220,13 +220,19 @@ impl ExecutionPlan for WindowAggExec { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements( + partition_bys, + order_keys.map_or_else(Vec::new, |o| o.to_vec()), + )] } else { let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys.iter())] + vec![calc_requirements( + partition_bys, + order_keys.map_or_else(Vec::new, |o| o.to_vec()), + )] } } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index b0b561d6d1815..7c149ab830a80 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -35,7 +35,7 @@ use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig, FileSource}; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; -use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, Literal, NegativeExpr, NotExpr, TryCastExpr, UnKnownColumn, @@ -99,7 +99,7 @@ pub fn parse_physical_sort_exprs( registry: &dyn FunctionRegistry, input_schema: &Schema, codec: &dyn PhysicalExtensionCodec, -) -> Result { +) -> Result> { proto .iter() .map(|sort_expr| { @@ -172,7 +172,7 @@ pub fn parse_physical_window_expr( name, &window_node_expr, &partition_by, - order_by.as_ref(), + (!order_by.is_empty()).then(|| order_by.into()), Arc::new(window_frame), &extended_schema, false, @@ -534,7 +534,7 @@ pub fn parse_protobuf_file_scan_config( &schema, codec, )?; - output_ordering.push(sort_expr); + output_ordering.push(sort_expr.into()); } let config = FileScanConfig::new(object_store_url, file_schema, file_source) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 2cf581c824de4..cb7c1293c571f 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -828,8 +828,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { filter, &join_type.into(), sym_join.null_equals_null, - left_sort_exprs, - right_sort_exprs, + left_sort_exprs.map(Into::into), + right_sort_exprs.map(Into::into), partition_mode, ) .map(|e| Arc::new(e) as _) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 3f67842fe625c..568a8e148c29a 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::physical_plan::FileSink; use datafusion::physical_expr::window::{SlidingAggregateWindowExpr, StandardWindowExpr}; -use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, Literal, NegativeExpr, NotExpr, TryCastExpr, UnKnownColumn, @@ -53,8 +53,8 @@ pub fn serialize_physical_aggr_expr( ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; let ordering_req = match aggr_expr.order_bys() { - Some(order) => order.clone(), - None => LexOrdering::default(), + Some(order) => order.to_vec(), + None => vec![], }; let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; @@ -146,7 +146,10 @@ pub fn serialize_physical_window_expr( let args = serialize_physical_exprs(&args, codec)?; let partition_by = serialize_physical_exprs(window_expr.partition_by(), codec)?; - let order_by = serialize_physical_sort_exprs(window_expr.order_by().to_vec(), codec)?; + let order_by = serialize_physical_sort_exprs( + window_expr.order_by().map_or_else(Vec::new, |o| o.to_vec()), + codec, + )?; let window_frame: protobuf::WindowFrame = window_frame .as_ref() .try_into() diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 5a634ace5c296..ec78df3350308 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -319,9 +319,9 @@ fn roundtrip_udwf() -> Result<()> { &[ col("a", &schema)? ], - &LexOrdering::new(vec![ + Some(LexOrdering::new(vec![ PhysicalSortExpr::new(col("b", &schema)?, SortOptions::new(true, true)), - ]), + ])), Arc::new(WindowFrame::new(None)), )); @@ -358,13 +358,13 @@ fn roundtrip_window() -> Result<()> { let udwf_expr = Arc::new(StandardWindowExpr::new( nth_value_window, &[col("b", &schema)?], - &LexOrdering::new(vec![PhysicalSortExpr { + Some(LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]), + }])), Arc::new(window_frame), )); @@ -378,7 +378,7 @@ fn roundtrip_window() -> Result<()> { .build() .map(Arc::new)?, &[], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new(None)), )); @@ -398,7 +398,7 @@ fn roundtrip_window() -> Result<()> { let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( sum_expr, &[], - &LexOrdering::default(), + None, Arc::new(window_frame), )); @@ -1105,7 +1105,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), &[col("author", &schema)?], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new(None)), ))], filter, @@ -1150,13 +1150,13 @@ fn roundtrip_udwf_extension_codec() -> Result<()> { let udwf_expr = Arc::new(StandardWindowExpr::new( udwf, &[col("b", &schema)?], - &LexOrdering::new(vec![PhysicalSortExpr { + Some(LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]), + }])), Arc::new(window_frame), )); @@ -1213,7 +1213,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr, &[col("author", &schema)?], - &LexOrdering::default(), + None, Arc::new(WindowFrame::new(None)), ))], filter, From 669a649165de966e2510b7d1dbb1b891f4c5e6e9 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 28 Mar 2025 17:54:37 +0300 Subject: [PATCH 037/167] fix first phase of merge conflicts and other bugs --- datafusion/catalog/src/stream.rs | 1 - .../core/src/datasource/file_format/arrow.rs | 1 - .../core/src/datasource/listing/table.rs | 2 +- datafusion/core/src/physical_planner.rs | 11 ++----- datafusion/datasource-json/src/file_format.rs | 1 - datafusion/datasource/src/sink.rs | 9 ++--- .../functions-aggregate/src/array_agg.rs | 33 ++++++++++--------- .../functions-aggregate/src/first_last.rs | 16 ++++++--- .../physical-plan/src/aggregates/mod.rs | 3 +- .../src/sorts/sort_preserving_merge.rs | 2 +- 10 files changed, 38 insertions(+), 41 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index c77be3273f2fd..9e24ffb5b034b 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -36,7 +36,6 @@ use datafusion_expr::dml::InsertOp; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_physical_expr::create_ordering; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index cc894c0300a57..1de4f046aff5c 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -55,7 +55,6 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use async_trait::async_trait; use bytes::Bytes; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 6466025f10a61..4a9e67fe038ea 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -36,8 +36,8 @@ use datafusion_common::{ config_datafusion_err, config_err, internal_err, plan_err, project_schema, Constraints, DataFusionError, Result, SchemaExt, ToDFSchema, }; -use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_groups::FileGroup; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::cache::{ cache_manager::FileStatisticsCache, cache_unit::DefaultFileStatisticsCache, }; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1404714a5f327..3c1c1d3db602d 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -77,12 +77,13 @@ use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ Analyze, DescribeTable, DmlStatement, Explain, ExplainFormat, Extension, FetchType, - Filter, JoinType, RecursiveQuery, SkipType, StringifiedPlan, WindowFrame, + Filter, JoinType, RecursiveQuery, SkipType, SortExpr, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; @@ -1681,14 +1682,6 @@ pub fn create_aggregate_expr_and_maybe_filter( ) } -#[deprecated( - since = "47.0.0", - note = "use datafusion::create_physical_sort_expr" -)] -pub use datafusion_physical_expr::{ - create_physical_sort_expr, -}; - /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( e: &SortExpr, diff --git a/datafusion/datasource-json/src/file_format.rs b/datafusion/datasource-json/src/file_format.rs index 5ebff8524b9ff..6ad54f4ae0286 100644 --- a/datafusion/datasource-json/src/file_format.rs +++ b/datafusion/datasource-json/src/file_format.rs @@ -54,7 +54,6 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index bb420eddc0e1b..1cab769665f56 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -26,13 +26,10 @@ use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::{ - execute_input_stream, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, + execute_input_stream, execution_plan::RequiredInputOrdering, DisplayAs, + DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, + SendableRecordBatchStream, }; -use crate::execution_plan::RequiredInputOrdering; -use crate::metrics::MetricsSet; -use crate::stream::RecordBatchStreamAdapter; -use crate::ExecutionPlanProperties; use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 609555a9921a7..26dab6d1fcd81 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -17,6 +17,7 @@ //! `ARRAY_AGG` aggregate implementation: [`ArrayAgg`] +use std::cmp::Ordering; use std::collections::{HashSet, VecDeque}; use std::mem::{size_of, size_of_val}; use std::sync::Arc; @@ -37,10 +38,6 @@ use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; use datafusion_functions_aggregate_common::utils::ordering_fields; use datafusion_macros::user_doc; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use std::cmp::Ordering; -use std::collections::{HashSet, VecDeque}; -use std::mem::{size_of, size_of_val}; -use std::sync::Arc; make_udaf_expr_and_func!( ArrayAgg, @@ -156,15 +153,15 @@ impl AggregateUDFImpl for ArrayAgg { // ARRAY_AGG(DISTINCT concat(col, '') ORDER BY concat(col, '')) <- Valid // ARRAY_AGG(DISTINCT col ORDER BY other_col) <- Invalid // ARRAY_AGG(DISTINCT col ORDER BY concat(col, '')) <- Invalid - if acc_args.ordering_req.len() > 1 { - return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); - } let mut sort_option: Option = None; - if let Some(order) = acc_args.ordering_req.first() { - if !order.expr.eq(&acc_args.exprs[0]) { + if let Some(ordering_req) = acc_args.ordering_req { + if ordering_req.len() > 1 { return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); } - sort_option = Some(order.options) + if !ordering_req[0].expr.eq(&acc_args.exprs[0]) { + return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); + } + sort_option = Some(ordering_req[0].options) } return Ok(Box::new(DistinctArrayAggAccumulator::try_new( &data_type, @@ -931,7 +928,7 @@ mod tests { struct ArrayAggAccumulatorBuilder { data_type: DataType, distinct: bool, - ordering: LexOrdering, + ordering: Option, schema: Schema, } @@ -944,7 +941,7 @@ mod tests { Self { data_type: data_type.clone(), distinct: Default::default(), - ordering: Default::default(), + ordering: None, schema: Schema { fields: Fields::from(vec![Field::new( "col", @@ -964,13 +961,19 @@ mod tests { } fn order_by_col(mut self, col: &str, sort_options: SortOptions) -> Self { - self.ordering.extend([PhysicalSortExpr::new( + let new_order = PhysicalSortExpr::new( Arc::new( Column::new_with_schema(col, &self.schema) .expect("column not available in schema"), ), sort_options, - )]); + ); + if let Some(mut existing_ordering) = self.ordering { + existing_ordering.extend([new_order]); + self.ordering = Some(existing_ordering); + } else { + self.ordering = Some(LexOrdering::from(vec![new_order])); + } self } @@ -979,7 +982,7 @@ mod tests { return_type: &self.data_type, schema: &self.schema, ignore_nulls: false, - ordering_req: &self.ordering, + ordering_req: self.ordering.as_ref(), is_reversed: false, name: "", is_distinct: self.distinct, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 9ebebd13fa687..afaef77baa6cb 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -44,8 +44,8 @@ use datafusion_common::{ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Documentation, EmitTo, Expr, ExprFunctionExt, ReversedUDAF, - GroupsAccumulator, Signature, SortExpr, Volatility, + Accumulator, AggregateUDFImpl, Documentation, EmitTo, Expr, ExprFunctionExt, + GroupsAccumulator, ReversedUDAF, Signature, SortExpr, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; use datafusion_macros::user_doc; @@ -198,14 +198,20 @@ impl AggregateUDFImpl for FirstValue { where T: ArrowPrimitiveType + Send, { - let ordering_dtypes = args - .ordering_req + let Some(ordering_req) = args.ordering_req else { + // return TrivialFirstValueAccumulator::try_new(args.return_type, args.ignore_nulls) + // .map(|acc| Box::new(acc) as _) + // TODO Fix + return internal_err!("Order by is a must!"); + }; + + let ordering_dtypes = ordering_req .iter() .map(|e| e.expr.data_type(args.schema)) .collect::>>()?; Ok(Box::new(FirstPrimitiveGroupsAccumulator::::try_new( - args.ordering_req.clone(), + ordering_req.clone(), args.ignore_nulls, args.return_type, &ordering_dtypes, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f26518b3fcfd2..1da75bc5037d5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1162,7 +1162,8 @@ pub fn get_finer_aggregate_exprs_requirement( requirement = finer_rev; continue; } - } else if conflict && conflict_rev { + } + if conflict && conflict_rev { // Neither the existing requirement nor the aggregate requirement // satisfy the other, this means requirements are conflicting. // Currently, we do not support conflicting requirements. diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index cd012769da454..c0e350d4f8418 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -414,7 +414,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::test_util::batches_to_string; - use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; + use datafusion_common::{assert_batches_eq, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; From cf1c527b539c41fc749dd0dc975f63f09c520468 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 28 Mar 2025 19:23:49 +0300 Subject: [PATCH 038/167] Fix sqllogictests except the schema mismatch --- .../functions-aggregate/src/first_last.rs | 18 ++--- .../src/enforce_sorting/sort_pushdown.rs | 18 ++--- datafusion/physical-plan/src/joins/utils.rs | 68 ++++++++++--------- 3 files changed, 51 insertions(+), 53 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index afaef77baa6cb..5908727174250 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -804,10 +804,11 @@ impl Accumulator for TrivialFirstValueAccumulator { let flags = states[1].as_boolean(); let filtered_states = filter_states_according_to_is_set(&states[0..1], flags)?; - // If we have any values, take the first one we find: if let Some(first) = filtered_states.first() { - self.first = ScalarValue::try_from_array(first, 0)?; - self.is_set = true; + if !first.is_empty() { + self.first = ScalarValue::try_from_array(first, 0)?; + self.is_set = true; + } } } Ok(()) @@ -1146,10 +1147,11 @@ impl Accumulator for TrivialLastValueAccumulator { // Second index contains is_set flag. let flags = states[1].as_boolean(); let filtered_states = filter_states_according_to_is_set(&states[0..1], flags)?; - // If we have any values, take the last one we find: if let Some(last) = filtered_states.last() { - self.last = ScalarValue::try_from_array(last, 0)?; - self.is_set = true; + if !last.is_empty() { + self.last = ScalarValue::try_from_array(last, 0)?; + self.is_set = true; + } } Ok(()) } @@ -1315,7 +1317,7 @@ fn filter_states_according_to_is_set( states .iter() .map(|state| compute::filter(state, flags).map_err(|e| arrow_datafusion_err!(e))) - .collect::>>() + .collect() } /// Combines array refs and their corresponding orderings to construct `SortColumn`s. @@ -1326,7 +1328,7 @@ fn convert_to_sort_cols(arrs: &[ArrayRef], sort_exprs: &LexOrdering) -> Vec>() + .collect() } #[cfg(test)] diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index a6eac4dd03121..b67061c71461e 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -416,19 +416,13 @@ fn try_pushdown_requirements_to_join( sort_expr: &LexOrdering, push_side: JoinSide, ) -> Result>>> { - let left_eq_properties = smj.left().equivalence_properties(); - let right_eq_properties = smj.right().equivalence_properties(); let mut smj_required_orderings = smj.required_input_ordering(); - let right_requirement = smj_required_orderings.swap_remove(1); - let left_requirement = smj_required_orderings.swap_remove(0); - let left_ordering = &smj.left().output_ordering().cloned().unwrap(); - let right_ordering = &smj.right().output_ordering().cloned().unwrap(); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { let left_eq_properties = - left_eq_properties.clone().with_reorder(sort_expr.clone()); - let Some(left_requirement) = left_requirement else { + smj.left().equivalence_properties().clone().with_reorder(sort_expr.clone()); + let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; if !left_eq_properties @@ -437,13 +431,13 @@ fn try_pushdown_requirements_to_join( return Ok(None); } // After re-ordering requirement is still satisfied - (sort_expr, right_ordering) + (Some(sort_expr), smj.right().output_ordering()) } JoinSide::Right => { let right_eq_properties = - right_eq_properties.clone().with_reorder(sort_expr.clone()); + smj.right().equivalence_properties().clone().with_reorder(sort_expr.clone()); - let Some(right_requirement) = right_requirement else { + let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); }; if !right_eq_properties @@ -452,7 +446,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); } // After re-ordering requirement is still satisfied - (left_ordering, sort_expr) + (smj.left().output_ordering(), Some(sort_expr)) } JoinSide::None => return Ok(None), }; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index f6c720dbb707a..76c685be21f40 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -136,7 +136,7 @@ pub fn adjust_right_output_partitioning( fn replace_on_columns_of_right_ordering( on_columns: &[(PhysicalExprRef, PhysicalExprRef)], right_ordering: &mut LexOrdering, -) -> Result<()> { +) { for (left_col, right_col) in on_columns { right_ordering.transform(|item| { let new_expr = Arc::clone(&item.expr) @@ -152,7 +152,6 @@ fn replace_on_columns_of_right_ordering( item.expr = new_expr; }); } - Ok(()) } fn offset_ordering( @@ -176,55 +175,58 @@ fn offset_ordering( /// Calculate the output ordering of a given join operation. pub fn calculate_join_output_ordering( - left_ordering: &LexOrdering, - right_ordering: &LexOrdering, + left_ordering: Option<&LexOrdering>, + right_ordering: Option<&LexOrdering>, join_type: JoinType, on_columns: &[(PhysicalExprRef, PhysicalExprRef)], left_columns_len: usize, maintains_input_order: &[bool], probe_side: Option, ) -> Option { - let output_ordering = match maintains_input_order { + match maintains_input_order { [true, false] => { // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { - replace_on_columns_of_right_ordering( - on_columns, - &mut right_ordering.clone(), - ) - .ok()?; - merge_vectors( - left_ordering, - offset_ordering(right_ordering, &join_type, left_columns_len) - .as_ref(), - ) - } else { - left_ordering.clone() + if let Some(right_ordering) = right_ordering { + replace_on_columns_of_right_ordering( + on_columns, + &mut right_ordering.clone(), + ); + let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); + return if let Some(left_ordering) = left_ordering { + Some(merge_vectors(left_ordering, &right_offset)) + } else { + Some(right_offset) + }; + } } + left_ordering.cloned() } [false, true] => { // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { - replace_on_columns_of_right_ordering( - on_columns, - &mut right_ordering.clone(), - ) - .ok()?; - merge_vectors( - offset_ordering(right_ordering, &join_type, left_columns_len) - .as_ref(), - left_ordering, - ) - } else { - offset_ordering(right_ordering, &join_type, left_columns_len) + return if let Some(right_ordering) = right_ordering { + replace_on_columns_of_right_ordering( + on_columns, + &mut right_ordering.clone(), + ); + let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); + if let Some(left_ordering) = left_ordering { + Some(merge_vectors(&right_offset, left_ordering)) + } else { + Some(right_offset) + } + } else { + left_ordering.cloned() + }; } + right_ordering.map(|o| offset_ordering(o, &join_type, left_columns_len)) } // Doesn't maintain ordering, output ordering is None. [false, false] => return None, [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), - }; - (!output_ordering.is_empty()).then_some(output_ordering) + } } /// Information about the index and placement (left or right) of the columns @@ -2406,8 +2408,8 @@ mod tests { { assert_eq!( calculate_join_output_ordering( - left_ordering.as_ref(), - right_ordering.as_ref(), + Some(&left_ordering), + Some(&right_ordering), join_type, &on_columns, left_columns_len, From 45bc4ec6c6bd75c1cf0d59365228699c0fc93bdf Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 28 Mar 2025 23:31:23 +0300 Subject: [PATCH 039/167] Cleanup Part 1 --- .../examples/custom_file_format.rs | 10 ++--- datafusion/catalog/src/stream.rs | 6 +-- .../core/src/datasource/file_format/arrow.rs | 5 +-- .../core/src/datasource/listing/table.rs | 6 +-- datafusion/core/src/datasource/memory.rs | 8 ++-- datafusion/core/src/physical_planner.rs | 37 +------------------ .../fuzz_cases/equivalence/properties.rs | 25 +++++++------ .../sort_preserving_repartition_fuzz.rs | 15 ++++---- .../core/tests/fuzz_cases/window_fuzz.rs | 2 +- .../tests/physical_optimizer/test_utils.rs | 9 ++--- datafusion/datasource-csv/src/file_format.rs | 5 +-- datafusion/datasource-json/src/file_format.rs | 5 +-- .../datasource-parquet/src/file_format.rs | 5 +-- datafusion/datasource/src/file_format.rs | 5 +-- datafusion/datasource/src/file_scan_config.rs | 35 ++++-------------- datafusion/datasource/src/sink.rs | 29 ++++++++------- datafusion/physical-expr/src/physical_expr.rs | 23 +++++------- .../src/enforce_sorting/sort_pushdown.rs | 14 +++++-- datafusion/physical-plan/src/joins/utils.rs | 8 ++-- datafusion/proto/src/physical_plan/mod.rs | 8 ++-- .../tests/cases/roundtrip_physical_plan.rs | 21 +++++------ 21 files changed, 107 insertions(+), 174 deletions(-) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 30033cba4d2d8..b19ebddfd5a91 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -21,14 +21,9 @@ use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use datafusion::execution::session_state::SessionStateBuilder; -use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::{ catalog::Session, common::{GetExt, Statistics}, -}; -use datafusion::{ datasource::{ file_format::{ csv::CsvFormatFactory, file_compression_type::FileCompressionType, @@ -38,9 +33,12 @@ use datafusion::{ MemTable, }, error::Result, + execution::session_state::SessionStateBuilder, + physical_expr::{LexRequirement, PhysicalExpr}, physical_plan::ExecutionPlan, prelude::SessionContext, }; + use object_store::{ObjectMeta, ObjectStore}; use tempfile::tempdir; @@ -122,7 +120,7 @@ impl FileFormat for TSVFileFormat { input: Arc, state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { self.csv_file_format .create_writer_physical_plan(input, state, conf, order_requirements) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index 9e24ffb5b034b..e713538d65d03 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -35,7 +35,6 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_physical_expr::create_ordering; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; @@ -354,10 +353,7 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, &self.0.order)?; // It is sufficient to pass only one of the equivalent orderings: - let order_requirements = orders.first().map(|ordering| { - let reqs = ordering.iter().cloned().map(Into::into).collect(); - RequiredInputOrdering::new(reqs) - }); + let order_requirements = orders.into_iter().next().map(Into::into); Ok(Arc::new(DataSinkExec::new( input, diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 1de4f046aff5c..0cd10a6bd93f0 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -53,8 +53,7 @@ use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; use async_trait::async_trait; use bytes::Bytes; @@ -181,7 +180,7 @@ impl FileFormat for ArrowFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Arrow format"); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 4a9e67fe038ea..5a8eb4ef9f6ea 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -46,7 +46,6 @@ use datafusion_expr::utils::conjunction; use datafusion_expr::{Expr, SortExpr, TableProviderFilterPushDown, TableType}; use datafusion_physical_expr::{create_physical_expr, LexOrdering}; use datafusion_physical_plan::empty::EmptyExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::{ExecutionPlan, Statistics}; use async_trait::async_trait; @@ -1045,10 +1044,7 @@ impl TableProvider for ListingTable { let orderings = self.try_create_output_ordering()?; // It is sufficient to pass only one of the equivalent orderings: - let order_requirements = orderings.first().map(|ordering| { - let reqs = ordering.iter().cloned().map(Into::into).collect(); - RequiredInputOrdering::new(reqs) - }); + let order_requirements = orderings.into_iter().next().map(Into::into); self.options() .format diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 0304f8f2641a3..689ce9209dd51 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -30,19 +30,17 @@ use crate::physical_plan::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, SendableRecordBatchStream, }; -use crate::physical_planner::create_physical_sort_exprs; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; use datafusion_common_runtime::JoinSet; -pub use datafusion_datasource::memory::MemorySourceConfig; use datafusion_datasource::sink::{DataSink, DataSinkExec}; -pub use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_expr::SortExpr; +use datafusion_physical_expr::create_physical_sort_exprs; use async_trait::async_trait; use futures::StreamExt; @@ -50,6 +48,10 @@ use log::debug; use parking_lot::Mutex; use tokio::sync::RwLock; +// Exports: +pub use datafusion_datasource::memory::MemorySourceConfig; +pub use datafusion_datasource::source::DataSourceExec; + /// Type alias for partition data pub type PartitionData = Arc>>; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 3c1c1d3db602d..3b0a950ec7968 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -77,13 +77,12 @@ use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ Analyze, DescribeTable, DmlStatement, Explain, ExplainFormat, Extension, FetchType, - Filter, JoinType, RecursiveQuery, SkipType, SortExpr, StringifiedPlan, WindowFrame, + Filter, JoinType, RecursiveQuery, SkipType, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr::{create_physical_sort_exprs, LexOrdering}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; @@ -1682,38 +1681,6 @@ pub fn create_aggregate_expr_and_maybe_filter( ) } -/// Create a physical sort expression from a logical expression -pub fn create_physical_sort_expr( - e: &SortExpr, - input_dfschema: &DFSchema, - execution_props: &ExecutionProps, -) -> Result { - let SortExpr { - expr, - asc, - nulls_first, - } = e; - Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - }) -} - -/// Create vector of physical sort expression from a vector of logical expression -pub fn create_physical_sort_exprs( - exprs: &[SortExpr], - input_dfschema: &DFSchema, - execution_props: &ExecutionProps, -) -> Result> { - exprs - .iter() - .map(|expr| create_physical_sort_expr(expr, input_dfschema, execution_props)) - .collect() -} - impl DefaultPhysicalPlanner { /// Handles capturing the various plans for EXPLAIN queries /// diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 7362fb5b326b6..34d58456ac732 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -15,18 +15,20 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use crate::fuzz_cases::equivalence::utils::{ create_random_schema, generate_table_for_eq_properties, is_table_same_after_sort, TestScalarUDF, }; + use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; -use datafusion_physical_expr::{PhysicalExprRef, ScalarFunctionExpr}; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + use itertools::Itertools; -use std::sync::Arc; #[test] fn test_find_longest_permutation_random() -> Result<()> { @@ -47,13 +49,13 @@ fn test_find_longest_permutation_random() -> Result<()> { Arc::clone(&test_fun), vec![col_a], &test_schema, - )?) as PhysicalExprRef; + )?) as _; let a_plus_b = Arc::new(BinaryExpr::new( col("a", &test_schema)?, Operator::Plus, col("b", &test_schema)?, - )) as Arc; + )) as _; let exprs = [ col("a", &test_schema)?, col("b", &test_schema)?, @@ -90,16 +92,15 @@ fn test_find_longest_permutation_random() -> Result<()> { assert_eq!(ordering.len(), indices.len(), "{}", err_msg); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). - if !ordering.is_empty() { - assert!( - is_table_same_after_sort( + assert!( + ordering.is_empty() + || is_table_same_after_sort( ordering.into(), table_data_with_properties.clone(), )?, - "{}", - err_msg - ); - } + "{}", + err_msg + ); } } } diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index ca802211afcb2..647f498ba4803 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -36,9 +36,7 @@ mod sp_repartition_fuzz_tests { }; use datafusion::prelude::SessionContext; use datafusion_common::Result; - use datafusion_execution::{ - config::SessionConfig, memory_pool::MemoryConsumer, SendableRecordBatchStream, - }; + use datafusion_execution::{config::SessionConfig, memory_pool::MemoryConsumer}; use datafusion_physical_expr::{ equivalence::{EquivalenceClass, EquivalenceProperties}, expressions::{col, Column}, @@ -226,18 +224,21 @@ mod sp_repartition_fuzz_tests { let table_data_with_properties = generate_table_for_eq_properties(&eq_properties, N_ELEM, N_DISTINCT)?; let schema = table_data_with_properties.schema(); - let streams: Vec = (0..N_PARTITION) + let streams = (0..N_PARTITION) .map(|_idx| { let batch = table_data_with_properties.clone(); Box::pin(RecordBatchStreamAdapter::new( schema.clone(), futures::stream::once(async { Ok(batch) }), - )) as SendableRecordBatchStream + )) as _ }) .collect::>(); - // Returns concatenated version of the all available orderings - let exprs = eq_properties.oeq_class().output_ordering().unwrap(); + // Returns concatenated version of the all available orderings: + let Some(exprs) = eq_properties.oeq_class().output_ordering() else { + // We always should have an ordering due to the way we generate the schema: + unreachable!("No ordering found in eq_properties: {:?}", eq_properties); + }; let context = SessionContext::new().task_ctx(); let mem_reservation = diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 092b53bae2a51..b9f94f1682ee5 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -601,7 +601,7 @@ async fn run_window_test( }) } if orderby_exprs.len() > 1 && !window_frame.can_accept_multi_orderby() { - orderby_exprs = orderby_exprs[0..1].to_vec(); + orderby_exprs.truncate(1); } let mut partitionby_exprs = vec![]; for column in &partition_by_columns { diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 7f68cb4183284..a622d96f4f11c 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -390,11 +390,10 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - if let Some(ordering) = &self.required_input_ordering { - vec![Some(RequiredInputOrdering::from(ordering.clone()))] - } else { - vec![None] - } + vec![self + .required_input_ordering + .as_ref() + .map(|ordering| RequiredInputOrdering::from(ordering.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/datasource-csv/src/file_format.rs b/datafusion/datasource-csv/src/file_format.rs index 961b59c1196ef..e6dbf73d3a7d5 100644 --- a/datafusion/datasource-csv/src/file_format.rs +++ b/datafusion/datasource-csv/src/file_format.rs @@ -50,8 +50,7 @@ use datafusion_datasource::write::orchestration::spawn_writer_tasks_and_join; use datafusion_datasource::write::BatchSerializer; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; @@ -435,7 +434,7 @@ impl FileFormat for CsvFormat { input: Arc, state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for CSV"); diff --git a/datafusion/datasource-json/src/file_format.rs b/datafusion/datasource-json/src/file_format.rs index 6ad54f4ae0286..0f0294f19b3f9 100644 --- a/datafusion/datasource-json/src/file_format.rs +++ b/datafusion/datasource-json/src/file_format.rs @@ -52,8 +52,7 @@ use datafusion_datasource::write::orchestration::spawn_writer_tasks_and_join; use datafusion_datasource::write::BatchSerializer; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; @@ -259,7 +258,7 @@ impl FileFormat for JsonFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Json"); diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 026b289a7fd92..576c6ba4a81e5 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -55,8 +55,7 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_expr::Expr; use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; use datafusion_physical_plan::Accumulator; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use datafusion_session::Session; @@ -425,7 +424,7 @@ impl FileFormat for ParquetFormat { input: Arc, _state: &dyn Session, conf: FileSinkConfig, - order_requirements: Option, + order_requirements: Option, ) -> Result> { if conf.insert_op != InsertOp::Append { return not_impl_err!("Overwrites are not implemented yet for Parquet"); diff --git a/datafusion/datasource/src/file_format.rs b/datafusion/datasource/src/file_format.rs index 2957c5fe5e733..0e0b7b12e16a0 100644 --- a/datafusion/datasource/src/file_format.rs +++ b/datafusion/datasource/src/file_format.rs @@ -32,8 +32,7 @@ use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt, Result, Statistics}; use datafusion_expr::Expr; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{LexRequirement, PhysicalExpr}; use datafusion_physical_plan::ExecutionPlan; use datafusion_session::Session; @@ -105,7 +104,7 @@ pub trait FileFormat: Send + Sync + fmt::Debug { _input: Arc, _state: &dyn Session, _conf: FileSinkConfig, - _order_requirements: Option, + _order_requirements: Option, ) -> Result> { not_impl_err!("Writer not implemented for this format") } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 0ece40e8bf3ed..643267cd21711 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -1095,38 +1095,17 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { #[cfg(test)] mod tests { - use crate::{test_util::MockSource, tests::aggr_test_schema}; + use std::collections::HashMap; use super::*; - use arrow::{ - array::{Int32Array, RecordBatch}, - compute::SortOptions, - }; + use crate::{test_util::MockSource, tests::aggr_test_schema}; - use datafusion_common::stats::Precision; - use datafusion_common::{assert_batches_eq, DFSchema}; - use datafusion_expr::{execution_props::ExecutionProps, SortExpr}; - use datafusion_physical_expr::create_physical_expr; - use std::collections::HashMap; + use arrow::array::{Int32Array, RecordBatch}; - fn create_physical_sort_expr( - e: &SortExpr, - input_dfschema: &DFSchema, - execution_props: &ExecutionProps, - ) -> Result { - let SortExpr { - expr, - asc, - nulls_first, - } = e; - Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - }) - } + use datafusion_common::assert_batches_eq; + use datafusion_common::stats::Precision; + use datafusion_expr::SortExpr; + use datafusion_physical_expr::create_physical_sort_expr; /// Returns the column names on the schema pub fn columns(schema: &Schema) -> Vec { diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index 1cab769665f56..ffeb517f53344 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -22,20 +22,18 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; -use datafusion_physical_plan::metrics::MetricsSet; -use datafusion_physical_plan::stream::RecordBatchStreamAdapter; -use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_plan::{ - execute_input_stream, execution_plan::RequiredInputOrdering, DisplayAs, - DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, -}; - use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{Distribution, EquivalenceProperties}; +use datafusion_physical_expr::{Distribution, EquivalenceProperties, LexRequirement}; +use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::stream::RecordBatchStreamAdapter; +use datafusion_physical_plan::{ + execute_input_stream, DisplayAs, DisplayFormatType, ExecutionPlan, + ExecutionPlanProperties, Partitioning, PlanProperties, SendableRecordBatchStream, +}; use async_trait::async_trait; use futures::StreamExt; @@ -89,7 +87,7 @@ pub struct DataSinkExec { /// Schema describing the structure of the output data. count_schema: SchemaRef, /// Optional required sort order for output data. - sort_order: Option, + sort_order: Option, cache: PlanProperties, } @@ -104,7 +102,7 @@ impl DataSinkExec { pub fn new( input: Arc, sink: Arc, - sort_order: Option, + sort_order: Option, ) -> Self { let count_schema = make_count_schema(); let cache = Self::create_schema(&input, count_schema); @@ -128,7 +126,7 @@ impl DataSinkExec { } /// Optional sort order for output data - pub fn sort_order(&self) -> &Option { + pub fn sort_order(&self) -> &Option { &self.sort_order } @@ -187,7 +185,10 @@ impl ExecutionPlan for DataSinkExec { fn required_input_ordering(&self) -> Vec> { // The required input ordering is set externally (e.g. by a `ListingTable`). // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). - vec![self.sort_order.clone()] + vec![self + .sort_order + .as_ref() + .map(|req| RequiredInputOrdering::new(req.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 7038bf3a3b2e3..9ca22c7f7714d 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -154,17 +154,12 @@ pub fn create_physical_sort_expr( input_dfschema: &DFSchema, execution_props: &ExecutionProps, ) -> Result { - let SortExpr { - expr, - asc, - nulls_first, - } = e; - Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, + create_physical_expr(&e.expr, input_dfschema, execution_props).map(|expr| { + let options = SortOptions { + descending: !e.asc, + nulls_first: e.nulls_first, + }; + PhysicalSortExpr { expr, options } }) } @@ -173,11 +168,11 @@ pub fn create_physical_sort_exprs( exprs: &[SortExpr], input_dfschema: &DFSchema, execution_props: &ExecutionProps, -) -> Result { +) -> Result> { exprs .iter() - .map(|expr| create_physical_sort_expr(expr, input_dfschema, execution_props)) - .collect::>() + .map(|e| create_physical_sort_expr(e, input_dfschema, execution_props)) + .collect() } #[cfg(test)] diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index b67061c71461e..ff906e999adb0 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -420,8 +420,11 @@ fn try_pushdown_requirements_to_join( let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { - let left_eq_properties = - smj.left().equivalence_properties().clone().with_reorder(sort_expr.clone()); + let left_eq_properties = smj + .left() + .equivalence_properties() + .clone() + .with_reorder(sort_expr.clone()); let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; @@ -434,8 +437,11 @@ fn try_pushdown_requirements_to_join( (Some(sort_expr), smj.right().output_ordering()) } JoinSide::Right => { - let right_eq_properties = - smj.right().equivalence_properties().clone().with_reorder(sort_expr.clone()); + let right_eq_properties = smj + .right() + .equivalence_properties() + .clone() + .with_reorder(sort_expr.clone()); let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 76c685be21f40..6a563186d06f9 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -192,7 +192,8 @@ pub fn calculate_join_output_ordering( on_columns, &mut right_ordering.clone(), ); - let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); + let right_offset = + offset_ordering(right_ordering, &join_type, left_columns_len); return if let Some(left_ordering) = left_ordering { Some(merge_vectors(left_ordering, &right_offset)) } else { @@ -210,7 +211,8 @@ pub fn calculate_join_output_ordering( on_columns, &mut right_ordering.clone(), ); - let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); + let right_offset = + offset_ordering(right_ordering, &join_type, left_columns_len); if let Some(left_ordering) = left_ordering { Some(merge_vectors(&right_offset, left_ordering)) } else { @@ -223,7 +225,7 @@ pub fn calculate_join_output_ordering( right_ordering.map(|o| offset_ordering(o, &join_type, left_columns_len)) } // Doesn't maintain ordering, output ordering is None. - [false, false] => return None, + [false, false] => None, [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 3f7308470685b..56b124ef8dfc6 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -63,7 +63,6 @@ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::empty::EmptyExec; -use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::filter::FilterExec; @@ -1093,7 +1092,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.map(RequiredInputOrdering::new), + sort_order, ))) } PhysicalPlanType::CsvSink(sink) => { @@ -1122,7 +1121,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.map(RequiredInputOrdering::new), + sort_order, ))) } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] @@ -1158,7 +1157,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), - sort_order.map(RequiredInputOrdering::new), + sort_order, ))) } #[cfg(not(feature = "parquet"))] @@ -2030,7 +2029,6 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let sort_order = match exec.sort_order() { Some(requirements) => { let expr = requirements - .lex_requirement() .iter() .map(|requirement| { let expr: PhysicalSortExpr = requirement.to_owned().into(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index cf8b498d48093..8a0d40335a043 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -66,7 +66,6 @@ use datafusion::physical_plan::aggregates::{ }; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; -use datafusion::physical_plan::execution_plan::RequiredInputOrdering; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; @@ -1321,7 +1320,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::new(sort_order)), + Some(sort_order), ))) } @@ -1347,15 +1346,13 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = RequiredInputOrdering::new(LexRequirement::new(vec![ - PhysicalSortRequirement::new( - Arc::new(Column::new("plan_type", 0)), - Some(SortOptions { - descending: true, - nulls_first: false, - }), - ), - ])); + let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( + Arc::new(Column::new("plan_type", 0)), + Some(SortOptions { + descending: true, + nulls_first: false, + }), + )]); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1416,7 +1413,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - Some(RequiredInputOrdering::new(sort_order)), + Some(sort_order), ))) } From 923115cd8a1af8c297572df4a6be7ecabbea9448 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 28 Mar 2025 23:54:27 +0300 Subject: [PATCH 040/167] Cleanup Part 2 --- datafusion/catalog/src/stream.rs | 4 ++-- datafusion/functions-aggregate/src/array_agg.rs | 9 +++------ 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index e713538d65d03..94685c73548d1 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -353,12 +353,12 @@ impl TableProvider for StreamTable { let schema = self.0.source.schema(); let orders = create_ordering(schema, &self.0.order)?; // It is sufficient to pass only one of the equivalent orderings: - let order_requirements = orders.into_iter().next().map(Into::into); + let ordering = orders.into_iter().next().map(Into::into); Ok(Arc::new(DataSinkExec::new( input, Arc::new(StreamWrite(Arc::clone(&self.0))), - order_requirements, + ordering, ))) } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 26dab6d1fcd81..adb51b9443d93 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -155,10 +155,8 @@ impl AggregateUDFImpl for ArrayAgg { // ARRAY_AGG(DISTINCT col ORDER BY concat(col, '')) <- Invalid let mut sort_option: Option = None; if let Some(ordering_req) = acc_args.ordering_req { - if ordering_req.len() > 1 { - return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); - } - if !ordering_req[0].expr.eq(&acc_args.exprs[0]) { + if ordering_req.len() > 1 || !ordering_req[0].expr.eq(&acc_args.exprs[0]) + { return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); } sort_option = Some(ordering_req[0].options) @@ -968,9 +966,8 @@ mod tests { ), sort_options, ); - if let Some(mut existing_ordering) = self.ordering { + if let Some(existing_ordering) = self.ordering.as_mut() { existing_ordering.extend([new_order]); - self.ordering = Some(existing_ordering); } else { self.ordering = Some(LexOrdering::from(vec![new_order])); } From ef738ead17ddc81922f6077ff450000bcd2374e9 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 29 Mar 2025 00:21:57 +0300 Subject: [PATCH 041/167] Cleanup Part 3 --- .../functions-aggregate/src/array_agg.rs | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index adb51b9443d93..d23f8c6385d50 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -488,27 +488,27 @@ impl OrderSensitiveArrayAggAccumulator { } fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); - let num_columns = fields.len(); - let struct_field = Fields::from(fields.clone()); - - let mut column_wise_ordering_values = vec![]; - for i in 0..num_columns { - let column_values = self - .ordering_values + let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); + + let column_wise_ordering_values = if self.ordering_values.is_empty() { + fields .iter() - .map(|x| x[i].clone()) - .collect::>(); - let array = if column_values.is_empty() { - new_empty_array(fields[i].data_type()) - } else { - ScalarValue::iter_to_array(column_values.into_iter())? - }; - column_wise_ordering_values.push(array); - } + .map(|f| new_empty_array(f.data_type())) + .collect::>() + } else { + (0..fields.len()) + .map(|i| { + let column_values = self.ordering_values.iter().map(|x| x[i].clone()); + ScalarValue::iter_to_array(column_values) + }) + .collect::>()? + }; - let ordering_array = - StructArray::try_new(struct_field, column_wise_ordering_values, None)?; + let ordering_array = StructArray::try_new( + Fields::from(fields), + column_wise_ordering_values, + None, + )?; Ok(SingleRowListArrayBuilder::new(Arc::new(ordering_array)).build_list_scalar()) } } From 272a0989b1e9b491b7c99e960dfca7e864e59707 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Sat, 29 Mar 2025 00:35:01 +0300 Subject: [PATCH 042/167] do not initialize Trivial accumulators if ordering is set --- .../functions-aggregate/src/first_last.rs | 48 +++++++++---------- 1 file changed, 22 insertions(+), 26 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 5908727174250..f3589c9f622ba 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -135,19 +135,17 @@ impl AggregateUDFImpl for FirstValue { fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { if let Some(ordering_req) = acc_args.ordering_req { - if !self.requirement_satisfied { - let ordering_dtypes = ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - return FirstValueAccumulator::try_new( - acc_args.return_type, - &ordering_dtypes, - ordering_req.clone(), - acc_args.ignore_nulls, - ) - .map(|acc| Box::new(acc) as _); - } + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + return FirstValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + ordering_req.clone(), + acc_args.ignore_nulls, + ) + .map(|acc| Box::new(acc) as _); } TrivialFirstValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) .map(|acc| Box::new(acc) as _) @@ -1040,19 +1038,17 @@ impl AggregateUDFImpl for LastValue { fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { if let Some(ordering_req) = acc_args.ordering_req { - if !self.requirement_satisfied { - let ordering_dtypes = ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - return LastValueAccumulator::try_new( - acc_args.return_type, - &ordering_dtypes, - ordering_req.clone(), - acc_args.ignore_nulls, - ) - .map(|acc| Box::new(acc) as _); - } + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + return LastValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + ordering_req.clone(), + acc_args.ignore_nulls, + ) + .map(|acc| Box::new(acc) as _); } TrivialLastValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) .map(|acc| Box::new(acc) as _) From 23f1108d9de98c571459f3b0ec2ab260012568d5 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Sat, 29 Mar 2025 10:17:26 +0300 Subject: [PATCH 043/167] initialize TrivialFirstPrimitiveGroupsAccumulator struct and return --- .../functions-aggregate/src/first_last.rs | 290 +++++++++++++++++- 1 file changed, 286 insertions(+), 4 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index f3589c9f622ba..bb6daafe5a8be 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -197,10 +197,12 @@ impl AggregateUDFImpl for FirstValue { T: ArrowPrimitiveType + Send, { let Some(ordering_req) = args.ordering_req else { - // return TrivialFirstValueAccumulator::try_new(args.return_type, args.ignore_nulls) - // .map(|acc| Box::new(acc) as _) - // TODO Fix - return internal_err!("Order by is a must!"); + return Ok(Box::new( + TrivialFirstPrimitiveGroupsAccumulator::::try_new( + args.ignore_nulls, + args.return_type, + )?, + )); }; let ordering_dtypes = ordering_req @@ -744,6 +746,285 @@ where } } } + +/// See [`FirstPrimitiveGroupsAccumulator`] for original, this is a copy of it with no orderings +struct TrivialFirstPrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, +{ + vals: Vec, + is_sets: BooleanBufferBuilder, + null_builder: BooleanBufferBuilder, + min_of_each_group_buf: (Vec, BooleanBufferBuilder), + + ignore_nulls: bool, + data_type: DataType, +} + +impl TrivialFirstPrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, +{ + fn try_new(ignore_nulls: bool, data_type: &DataType) -> Result { + Ok(Self { + null_builder: BooleanBufferBuilder::new(0), + ignore_nulls, + data_type: data_type.clone(), + vals: Vec::new(), + is_sets: BooleanBufferBuilder::new(0), + min_of_each_group_buf: (Vec::new(), BooleanBufferBuilder::new(0)), + }) + } + + fn need_update(&self, group_idx: usize) -> bool { + if !self.is_sets.get_bit(group_idx) { + return true; + } + + if self.ignore_nulls && !self.null_builder.get_bit(group_idx) { + return true; + } + + false + } + + fn should_update_state(&self, group_idx: usize) -> Result { + if !self.is_sets.get_bit(group_idx) { + return Ok(true); + } + + Ok(false) + } + + fn take_need( + bool_buf_builder: &mut BooleanBufferBuilder, + emit_to: EmitTo, + ) -> BooleanBuffer { + let bool_buf = bool_buf_builder.finish(); + match emit_to { + EmitTo::All => bool_buf, + EmitTo::First(n) => { + // split off the first N values in seen_values + // + // TODO make this more efficient rather than two + // copies and bitwise manipulation + let first_n: BooleanBuffer = bool_buf.iter().take(n).collect(); + // reset the existing buffer + for b in bool_buf.iter().skip(n) { + bool_buf_builder.append(b); + } + first_n + } + } + } + + fn resize_states(&mut self, new_size: usize) { + self.vals.resize(new_size, T::default_value()); + self.null_builder.resize(new_size); + + self.is_sets.resize(new_size); + + self.min_of_each_group_buf.0.resize(new_size, 0); + self.min_of_each_group_buf.1.resize(new_size); + } + + fn update_state(&mut self, group_idx: usize, new_val: T::Native, is_null: bool) { + self.vals[group_idx] = new_val; + self.is_sets.set_bit(group_idx, true); + + self.null_builder.set_bit(group_idx, !is_null); + } + + fn take_state(&mut self, emit_to: EmitTo) -> (ArrayRef, BooleanBuffer) { + emit_to.take_needed(&mut self.min_of_each_group_buf.0); + self.min_of_each_group_buf + .1 + .truncate(self.min_of_each_group_buf.0.len()); + + ( + self.take_vals_and_null_buf(emit_to), + Self::take_need(&mut self.is_sets, emit_to), + ) + } + + fn get_filtered_min_of_each_group( + &mut self, + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + vals: &PrimitiveArray, + is_set_arr: Option<&BooleanArray>, + ) -> Result> { + self.min_of_each_group_buf.1.truncate(0); + self.min_of_each_group_buf + .1 + .append_n(self.vals.len(), false); + + for (idx_in_val, group_idx) in group_indices.iter().enumerate() { + let group_idx = *group_idx; + + let passed_filter = opt_filter.is_none_or(|x| x.value(idx_in_val)); + + let is_set = is_set_arr.is_none_or(|x| x.value(idx_in_val)); + + if !passed_filter || !is_set { + continue; + } + + if !self.need_update(group_idx) { + continue; + } + + if self.ignore_nulls && vals.is_null(idx_in_val) { + continue; + } + + let is_valid = self.min_of_each_group_buf.1.get_bit(group_idx); + if is_valid { + self.min_of_each_group_buf.0[group_idx] = idx_in_val; + } else if !is_valid { + self.min_of_each_group_buf.1.set_bit(group_idx, true); + self.min_of_each_group_buf.0[group_idx] = idx_in_val; + } + } + + Ok(self + .min_of_each_group_buf + .0 + .iter() + .enumerate() + .filter(|(group_idx, _)| self.min_of_each_group_buf.1.get_bit(*group_idx)) + .map(|(group_idx, idx_in_val)| (group_idx, *idx_in_val)) + .collect::>()) + } + + fn take_vals_and_null_buf(&mut self, emit_to: EmitTo) -> ArrayRef { + let r = emit_to.take_needed(&mut self.vals); + + let null_buf = NullBuffer::new(Self::take_need(&mut self.null_builder, emit_to)); + + let values = PrimitiveArray::::new(r.into(), Some(null_buf)) // no copy + .with_data_type(self.data_type.clone()); + Arc::new(values) + } +} + +impl GroupsAccumulator for TrivialFirstPrimitiveGroupsAccumulator +where + T: ArrowPrimitiveType + Send, +{ + fn update_batch( + &mut self, + values_and_order_cols: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.resize_states(total_num_groups); + + let vals = values_and_order_cols[0].as_primitive::(); + + let mut ordering_buf = Vec::with_capacity(0); + + for (group_idx, idx) in self + .get_filtered_min_of_each_group(group_indices, opt_filter, vals, None)? + .into_iter() + { + extract_row_at_idx_to_buf( + &values_and_order_cols[1..], + idx, + &mut ordering_buf, + )?; + + if self.should_update_state(group_idx)? { + self.update_state(group_idx, vals.value(idx), vals.is_null(idx)); + } + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + Ok(self.take_state(emit_to).0) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let (val_arr, is_sets) = self.take_state(emit_to); + let mut result = Vec::with_capacity(2); + + result.push(val_arr); + result.push(Arc::new(BooleanArray::new(is_sets, None))); + + Ok(result) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.resize_states(total_num_groups); + + let mut ordering_buf = Vec::with_capacity(0); + + let (is_set_arr, val_and_order_cols) = match values.split_last() { + Some(result) => result, + None => return internal_err!("Empty row in FISRT_VALUE"), + }; + + let is_set_arr = as_boolean_array(is_set_arr)?; + + let vals = values[0].as_primitive::(); + let groups = self.get_filtered_min_of_each_group( + group_indices, + opt_filter, + vals, + Some(is_set_arr), + )?; + + for (group_idx, idx) in groups.into_iter() { + extract_row_at_idx_to_buf(&val_and_order_cols[1..], idx, &mut ordering_buf)?; + + if self.should_update_state(group_idx)? { + self.update_state(group_idx, vals.value(idx), vals.is_null(idx)); + } + } + + Ok(()) + } + + fn size(&self) -> usize { + self.vals.capacity() * size_of::() + + self.null_builder.capacity() / 8 + + self.is_sets.capacity() / 8 + + self.min_of_each_group_buf.0.capacity() * size_of::() + + self.min_of_each_group_buf.1.capacity() / 8 + } + + fn supports_convert_to_state(&self) -> bool { + true + } + + fn convert_to_state( + &self, + values: &[ArrayRef], + opt_filter: Option<&BooleanArray>, + ) -> Result> { + let mut result = values.to_vec(); + match opt_filter { + Some(f) => { + result.push(Arc::new(f.clone())); + Ok(result) + } + None => { + result.push(Arc::new(BooleanArray::from(vec![true; values[0].len()]))); + Ok(result) + } + } + } +} + #[derive(Debug)] pub struct TrivialFirstValueAccumulator { first: ScalarValue, @@ -1293,6 +1574,7 @@ impl Accumulator for LastValueAccumulator { } fn evaluate(&mut self) -> Result { + println!("Last value eval {:?}", self.last); Ok(self.last.clone()) } From 63385fa1dd8dc339457c2f25174948ee2804ea9e Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 2 Apr 2025 10:12:50 +0300 Subject: [PATCH 044/167] fix clippy --- datafusion/functions-aggregate/src/first_last.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index bb6daafe5a8be..ef759e6aa058f 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -949,12 +949,8 @@ where fn state(&mut self, emit_to: EmitTo) -> Result> { let (val_arr, is_sets) = self.take_state(emit_to); - let mut result = Vec::with_capacity(2); - result.push(val_arr); - result.push(Arc::new(BooleanArray::new(is_sets, None))); - - Ok(result) + Ok(vec![val_arr, Arc::new(BooleanArray::new(is_sets, None))]) } fn merge_batch( From f70197f05f04a3c8128a598837033947e4f4cef7 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 2 Apr 2025 10:56:22 +0300 Subject: [PATCH 045/167] fix merge conflicts --- datafusion/catalog/src/memory/table.rs | 17 +++++++---------- datafusion/core/src/datasource/listing/table.rs | 11 +++-------- datafusion/datasource/src/file_scan_config.rs | 6 ++++-- 3 files changed, 14 insertions(+), 20 deletions(-) diff --git a/datafusion/catalog/src/memory/table.rs b/datafusion/catalog/src/memory/table.rs index 81243e2c4889e..99bd71c0caa2b 100644 --- a/datafusion/catalog/src/memory/table.rs +++ b/datafusion/catalog/src/memory/table.rs @@ -239,16 +239,13 @@ impl TableProvider for MemTable { if !sort_order.is_empty() { let df_schema = DFSchema::try_from(self.schema.as_ref().clone())?; - let file_sort_order = sort_order - .iter() - .map(|sort_exprs| { - create_physical_sort_exprs( - sort_exprs, - &df_schema, - state.execution_props(), - ) - }) - .collect::>>()?; + let eqp = state.execution_props(); + let mut file_sort_order = vec![]; + for sort_exprs in sort_order.iter() { + file_sort_order.push( + create_physical_sort_exprs(sort_exprs, &df_schema, eqp)?.into(), + ); + } source = source.try_with_sort_information(file_sort_order)?; } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 9731903b5cd65..72214f4a784c9 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -29,17 +29,18 @@ use crate::datasource::{ }, physical_plan::FileSinkConfig, }; -use crate::datasource::physical_plan::FileSinkConfig; -use crate::datasource::{create_ordering, get_statistics_with_limit}; use crate::execution::context::SessionState; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; use arrow_schema::Schema; use datafusion_catalog::{Session, TableProvider}; +use datafusion_common::stats::Precision; use datafusion_common::{ config_datafusion_err, config_err, internal_err, plan_err, project_schema, Constraints, DataFusionError, Result, SchemaExt, ToDFSchema, }; +use datafusion_datasource::add_row_stats; +use datafusion_datasource::compute_all_files_statistics; use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_execution::cache::{ @@ -53,12 +54,6 @@ use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::{ExecutionPlan, Statistics}; use async_trait::async_trait; -use datafusion_catalog::Session; -use datafusion_common::stats::Precision; -use datafusion_datasource::add_row_stats; -use datafusion_datasource::compute_all_files_statistics; -use datafusion_datasource::file_groups::FileGroup; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{future, stream, Stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index e954c48c06afa..6e3cf7cfa9a69 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -89,7 +89,7 @@ use crate::{ /// # // Note: crate mock ParquetSource, as ParquetSource is not in the datasource crate /// # struct ParquetSource { /// # projected_statistics: Option -/// # }; +/// # } /// # impl FileSource for ParquetSource { /// # fn create_file_opener(&self, _: Arc, _: &FileScanConfig, _: usize) -> Arc { unimplemented!() } /// # fn as_any(&self) -> &dyn Any { self } @@ -2085,7 +2085,9 @@ mod tests { "test.parquet".to_string(), 1024, )])]) - .with_output_ordering(vec![LexOrdering::default()]) + .with_output_ordering(vec![LexOrdering::new(vec![ + PhysicalSortExpr::new_default(Arc::new(Column::new("date", 0))), + ])]) .with_file_compression_type(FileCompressionType::UNCOMPRESSED) .with_newlines_in_values(true) .build(); From 4e5d09442ba893ee869aecdcd29846b2eb86304e Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 2 Apr 2025 17:58:41 +0300 Subject: [PATCH 046/167] fix typos remove TrivialFirstPrimitiveGroupsAccumulator make groups accumulator available only when order requirement is set --- .../functions-aggregate/src/first_last.rs | 290 +----------------- .../src/update_aggr_exprs.rs | 8 +- 2 files changed, 9 insertions(+), 289 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index ef759e6aa058f..19df5f2900887 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -164,7 +164,7 @@ impl AggregateUDFImpl for FirstValue { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; - matches!( + args.ordering_req.is_some() && !self.requirement_satisfied && matches!( args.return_type, Int8 | Int16 | Int32 @@ -197,12 +197,7 @@ impl AggregateUDFImpl for FirstValue { T: ArrowPrimitiveType + Send, { let Some(ordering_req) = args.ordering_req else { - return Ok(Box::new( - TrivialFirstPrimitiveGroupsAccumulator::::try_new( - args.ignore_nulls, - args.return_type, - )?, - )); + unreachable!("Groups accumulator must have ordering."); }; let ordering_dtypes = ordering_req @@ -684,7 +679,7 @@ where let (is_set_arr, val_and_order_cols) = match values.split_last() { Some(result) => result, - None => return internal_err!("Empty row in FISRT_VALUE"), + None => return internal_err!("Empty row in FIRST_VALUE"), }; let is_set_arr = as_boolean_array(is_set_arr)?; @@ -747,280 +742,6 @@ where } } -/// See [`FirstPrimitiveGroupsAccumulator`] for original, this is a copy of it with no orderings -struct TrivialFirstPrimitiveGroupsAccumulator -where - T: ArrowPrimitiveType + Send, -{ - vals: Vec, - is_sets: BooleanBufferBuilder, - null_builder: BooleanBufferBuilder, - min_of_each_group_buf: (Vec, BooleanBufferBuilder), - - ignore_nulls: bool, - data_type: DataType, -} - -impl TrivialFirstPrimitiveGroupsAccumulator -where - T: ArrowPrimitiveType + Send, -{ - fn try_new(ignore_nulls: bool, data_type: &DataType) -> Result { - Ok(Self { - null_builder: BooleanBufferBuilder::new(0), - ignore_nulls, - data_type: data_type.clone(), - vals: Vec::new(), - is_sets: BooleanBufferBuilder::new(0), - min_of_each_group_buf: (Vec::new(), BooleanBufferBuilder::new(0)), - }) - } - - fn need_update(&self, group_idx: usize) -> bool { - if !self.is_sets.get_bit(group_idx) { - return true; - } - - if self.ignore_nulls && !self.null_builder.get_bit(group_idx) { - return true; - } - - false - } - - fn should_update_state(&self, group_idx: usize) -> Result { - if !self.is_sets.get_bit(group_idx) { - return Ok(true); - } - - Ok(false) - } - - fn take_need( - bool_buf_builder: &mut BooleanBufferBuilder, - emit_to: EmitTo, - ) -> BooleanBuffer { - let bool_buf = bool_buf_builder.finish(); - match emit_to { - EmitTo::All => bool_buf, - EmitTo::First(n) => { - // split off the first N values in seen_values - // - // TODO make this more efficient rather than two - // copies and bitwise manipulation - let first_n: BooleanBuffer = bool_buf.iter().take(n).collect(); - // reset the existing buffer - for b in bool_buf.iter().skip(n) { - bool_buf_builder.append(b); - } - first_n - } - } - } - - fn resize_states(&mut self, new_size: usize) { - self.vals.resize(new_size, T::default_value()); - self.null_builder.resize(new_size); - - self.is_sets.resize(new_size); - - self.min_of_each_group_buf.0.resize(new_size, 0); - self.min_of_each_group_buf.1.resize(new_size); - } - - fn update_state(&mut self, group_idx: usize, new_val: T::Native, is_null: bool) { - self.vals[group_idx] = new_val; - self.is_sets.set_bit(group_idx, true); - - self.null_builder.set_bit(group_idx, !is_null); - } - - fn take_state(&mut self, emit_to: EmitTo) -> (ArrayRef, BooleanBuffer) { - emit_to.take_needed(&mut self.min_of_each_group_buf.0); - self.min_of_each_group_buf - .1 - .truncate(self.min_of_each_group_buf.0.len()); - - ( - self.take_vals_and_null_buf(emit_to), - Self::take_need(&mut self.is_sets, emit_to), - ) - } - - fn get_filtered_min_of_each_group( - &mut self, - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - vals: &PrimitiveArray, - is_set_arr: Option<&BooleanArray>, - ) -> Result> { - self.min_of_each_group_buf.1.truncate(0); - self.min_of_each_group_buf - .1 - .append_n(self.vals.len(), false); - - for (idx_in_val, group_idx) in group_indices.iter().enumerate() { - let group_idx = *group_idx; - - let passed_filter = opt_filter.is_none_or(|x| x.value(idx_in_val)); - - let is_set = is_set_arr.is_none_or(|x| x.value(idx_in_val)); - - if !passed_filter || !is_set { - continue; - } - - if !self.need_update(group_idx) { - continue; - } - - if self.ignore_nulls && vals.is_null(idx_in_val) { - continue; - } - - let is_valid = self.min_of_each_group_buf.1.get_bit(group_idx); - if is_valid { - self.min_of_each_group_buf.0[group_idx] = idx_in_val; - } else if !is_valid { - self.min_of_each_group_buf.1.set_bit(group_idx, true); - self.min_of_each_group_buf.0[group_idx] = idx_in_val; - } - } - - Ok(self - .min_of_each_group_buf - .0 - .iter() - .enumerate() - .filter(|(group_idx, _)| self.min_of_each_group_buf.1.get_bit(*group_idx)) - .map(|(group_idx, idx_in_val)| (group_idx, *idx_in_val)) - .collect::>()) - } - - fn take_vals_and_null_buf(&mut self, emit_to: EmitTo) -> ArrayRef { - let r = emit_to.take_needed(&mut self.vals); - - let null_buf = NullBuffer::new(Self::take_need(&mut self.null_builder, emit_to)); - - let values = PrimitiveArray::::new(r.into(), Some(null_buf)) // no copy - .with_data_type(self.data_type.clone()); - Arc::new(values) - } -} - -impl GroupsAccumulator for TrivialFirstPrimitiveGroupsAccumulator -where - T: ArrowPrimitiveType + Send, -{ - fn update_batch( - &mut self, - values_and_order_cols: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - self.resize_states(total_num_groups); - - let vals = values_and_order_cols[0].as_primitive::(); - - let mut ordering_buf = Vec::with_capacity(0); - - for (group_idx, idx) in self - .get_filtered_min_of_each_group(group_indices, opt_filter, vals, None)? - .into_iter() - { - extract_row_at_idx_to_buf( - &values_and_order_cols[1..], - idx, - &mut ordering_buf, - )?; - - if self.should_update_state(group_idx)? { - self.update_state(group_idx, vals.value(idx), vals.is_null(idx)); - } - } - - Ok(()) - } - - fn evaluate(&mut self, emit_to: EmitTo) -> Result { - Ok(self.take_state(emit_to).0) - } - - fn state(&mut self, emit_to: EmitTo) -> Result> { - let (val_arr, is_sets) = self.take_state(emit_to); - - Ok(vec![val_arr, Arc::new(BooleanArray::new(is_sets, None))]) - } - - fn merge_batch( - &mut self, - values: &[ArrayRef], - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) -> Result<()> { - self.resize_states(total_num_groups); - - let mut ordering_buf = Vec::with_capacity(0); - - let (is_set_arr, val_and_order_cols) = match values.split_last() { - Some(result) => result, - None => return internal_err!("Empty row in FISRT_VALUE"), - }; - - let is_set_arr = as_boolean_array(is_set_arr)?; - - let vals = values[0].as_primitive::(); - let groups = self.get_filtered_min_of_each_group( - group_indices, - opt_filter, - vals, - Some(is_set_arr), - )?; - - for (group_idx, idx) in groups.into_iter() { - extract_row_at_idx_to_buf(&val_and_order_cols[1..], idx, &mut ordering_buf)?; - - if self.should_update_state(group_idx)? { - self.update_state(group_idx, vals.value(idx), vals.is_null(idx)); - } - } - - Ok(()) - } - - fn size(&self) -> usize { - self.vals.capacity() * size_of::() - + self.null_builder.capacity() / 8 - + self.is_sets.capacity() / 8 - + self.min_of_each_group_buf.0.capacity() * size_of::() - + self.min_of_each_group_buf.1.capacity() / 8 - } - - fn supports_convert_to_state(&self) -> bool { - true - } - - fn convert_to_state( - &self, - values: &[ArrayRef], - opt_filter: Option<&BooleanArray>, - ) -> Result> { - let mut result = values.to_vec(); - match opt_filter { - Some(f) => { - result.push(Arc::new(f.clone())); - Ok(result) - } - None => { - result.push(Arc::new(BooleanArray::from(vec![true; values[0].len()]))); - Ok(result) - } - } - } -} - #[derive(Debug)] pub struct TrivialFirstValueAccumulator { first: ScalarValue, @@ -1570,7 +1291,6 @@ impl Accumulator for LastValueAccumulator { } fn evaluate(&mut self) -> Result { - println!("Last value eval {:?}", self.last); Ok(self.last.clone()) } @@ -1717,7 +1437,7 @@ mod tests { } #[test] - fn test_frist_group_acc() -> Result<()> { + fn test_first_group_acc() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), @@ -1810,7 +1530,7 @@ mod tests { } #[test] - fn test_frist_group_acc_size_of_ordering() -> Result<()> { + fn test_first_group_acc_size_of_ordering() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 838e3a04432b3..eeb202b8a410b 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -90,7 +90,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { return Ok(Transformed::no(plan)); } let input = aggr_exec.input(); - let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + let mut aggr_exprs = aggr_exec.aggr_expr().to_vec(); let groupby_exprs = aggr_exec.group_expr().input_exprs(); // If the existing ordering satisfies a prefix of the GROUP BY @@ -109,13 +109,13 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { }) .collect::>(); - aggr_expr = try_convert_aggregate_if_better( - aggr_expr, + aggr_exprs = try_convert_aggregate_if_better( + aggr_exprs, &requirement, input.equivalence_properties(), )?; - let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_expr); + let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_exprs); Ok(Transformed::yes(Arc::new(aggr_exec) as _)) } else { From 30594f02a6943e3f1a8f9b3d8729f254cf38b7c6 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 2 Apr 2025 18:03:44 +0300 Subject: [PATCH 047/167] format code --- .../functions-aggregate/src/first_last.rs | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 19df5f2900887..f8a85990035c0 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -164,26 +164,28 @@ impl AggregateUDFImpl for FirstValue { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; - args.ordering_req.is_some() && !self.requirement_satisfied && matches!( - args.return_type, - Int8 | Int16 - | Int32 - | Int64 - | UInt8 - | UInt16 - | UInt32 - | UInt64 - | Float16 - | Float32 - | Float64 - | Decimal128(_, _) - | Decimal256(_, _) - | Date32 - | Date64 - | Time32(_) - | Time64(_) - | Timestamp(_, _) - ) + args.ordering_req.is_some() + && !self.requirement_satisfied + && matches!( + args.return_type, + Int8 | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float16 + | Float32 + | Float64 + | Decimal128(_, _) + | Decimal256(_, _) + | Date32 + | Date64 + | Time32(_) + | Time64(_) + | Timestamp(_, _) + ) } fn create_groups_accumulator( From ebda66957e8cdb6f141e17a753b984f251dd6672 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 2 Apr 2025 20:56:15 +0300 Subject: [PATCH 048/167] Add requirement_satisfied back in --- .../core/src/datasource/listing/table.rs | 4 +- .../physical_optimizer/enforce_sorting.rs | 4 +- datafusion/datasource/src/file_scan_config.rs | 30 ++++--- .../functions-aggregate/src/first_last.rs | 87 +++++++++++++++---- .../sqllogictest/test_files/my_group_by.slt | 57 ++++++++++++ 5 files changed, 145 insertions(+), 37 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/my_group_by.slt diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 72214f4a784c9..d831e22b0bd45 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -17,11 +17,11 @@ //! The table implementation. -use super::helpers::{expr_applicable_for_cols, pruned_partition_list}; -use super::{ListingTableUrl, PartitionedFile}; use std::collections::HashMap; use std::{any::Any, str::FromStr, sync::Arc}; +use super::helpers::{expr_applicable_for_cols, pruned_partition_list}; +use super::{ListingTableUrl, PartitionedFile}; use crate::datasource::{ create_ordering, file_format::{ diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index b341f2efb2cba..8247336d29459 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -33,6 +33,8 @@ use arrow::datatypes::{DataType, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; use datafusion_common::{Result, ScalarValue}; +use datafusion_datasource::file_scan_config::FileScanConfigBuilder; +use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{JoinType, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; @@ -60,8 +62,6 @@ use datafusion_expr_common::operator::Operator; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_plan::execution_plan::RequiredInputOrdering; -use datafusion_datasource::file_scan_config::FileScanConfigBuilder; -use datafusion_datasource::source::DataSourceExec; use rstest::rstest; /// Created a sorted parquet exec diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 6e3cf7cfa9a69..ed7aaad2dd560 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -23,6 +23,17 @@ use std::{ fmt::Result as FmtResult, marker::PhantomData, sync::Arc, }; +use crate::file_groups::FileGroup; +use crate::{ + display::FileGroupsDisplay, + file::FileSource, + file_compression_type::FileCompressionType, + file_stream::FileStream, + source::{DataSource, DataSourceExec}, + statistics::MinMaxStatistics, + PartitionedFile, +}; + use arrow::{ array::{ ArrayData, ArrayRef, BufferBuilder, DictionaryArray, RecordBatch, @@ -31,8 +42,10 @@ use arrow::{ buffer::Buffer, datatypes::{ArrowNativeType, DataType, Field, Schema, SchemaRef, UInt16Type}, }; -use datafusion_common::{exec_err, ColumnStatistics, Constraints, Result, Statistics}; -use datafusion_common::{DataFusionError, ScalarValue}; +use datafusion_common::{ + exec_err, ColumnStatistics, Constraints, DataFusionError, Result, ScalarValue, + Statistics, +}; use datafusion_execution::{ object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, }; @@ -46,18 +59,8 @@ use datafusion_physical_plan::{ projection::{all_alias_free_columns, new_projections_for_columns, ProjectionExec}, DisplayAs, DisplayFormatType, ExecutionPlan, }; -use log::{debug, warn}; -use crate::file_groups::FileGroup; -use crate::{ - display::FileGroupsDisplay, - file::FileSource, - file_compression_type::FileCompressionType, - file_stream::FileStream, - source::{DataSource, DataSourceExec}, - statistics::MinMaxStatistics, - PartitionedFile, -}; +use log::{debug, warn}; /// The base configurations for a [`DataSourceExec`], the a physical plan for /// any given file format. @@ -1384,7 +1387,6 @@ mod tests { use crate::{test_util::MockSource, tests::aggr_test_schema}; use arrow::array::{Int32Array, RecordBatch}; - use datafusion_common::assert_batches_eq; use datafusion_common::stats::Precision; use datafusion_expr::SortExpr; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index f8a85990035c0..a91bf071001c4 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -145,7 +145,9 @@ impl AggregateUDFImpl for FirstValue { ordering_req.clone(), acc_args.ignore_nulls, ) - .map(|acc| Box::new(acc) as _); + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }); } TrivialFirstValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) .map(|acc| Box::new(acc) as _) @@ -165,7 +167,6 @@ impl AggregateUDFImpl for FirstValue { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; args.ordering_req.is_some() - && !self.requirement_satisfied && matches!( args.return_type, Int8 | Int16 @@ -192,14 +193,11 @@ impl AggregateUDFImpl for FirstValue { &self, args: AccumulatorArgs, ) -> Result> { - fn create_accumulator( + fn create_accumulator( args: AccumulatorArgs, - ) -> Result> - where - T: ArrowPrimitiveType + Send, - { + ) -> Result> { let Some(ordering_req) = args.ordering_req else { - unreachable!("Groups accumulator must have ordering."); + return internal_err!("Groups accumulator must have an ordering."); }; let ordering_dtypes = ordering_req @@ -261,7 +259,7 @@ impl AggregateUDFImpl for FirstValue { } _ => internal_err!( - "GroupsAccumulator not supported for first({})", + "GroupsAccumulator not supported for first_value({})", args.return_type ), } @@ -826,12 +824,13 @@ pub struct FirstValueAccumulator { first: ScalarValue, // Whether we have seen the first value yet. is_set: bool, - // Stores ordering values, of the aggregator requirement corresponding to - // first value of the aggregator. These values are used during merging of - // multiple partitions. + // Stores values of the ordering columns corresponding to the first value. + // These values are used during merging of multiple partitions. orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, + // Stores whether incoming data already satisfies the ordering requirement. + requirement_satisfied: bool, // Ignore null values. ignore_nulls: bool, } @@ -853,10 +852,16 @@ impl FirstValueAccumulator { is_set: false, orderings, ordering_req, + requirement_satisfied: false, ignore_nulls, }) } + pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } + // Updates state with the values in the given row. fn update_with_new_row(&mut self, row: &[ScalarValue]) { self.first = row[0].clone(); @@ -868,6 +873,21 @@ impl FirstValueAccumulator { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in FIRST_VALUE"); }; + if self.requirement_satisfied { + // Get first entry according to the pre-existing ordering (0th index): + if self.ignore_nulls { + // If ignoring nulls, find the first non-null value. + for i in 0..value.len() { + if !value.is_null(i) { + return Ok(Some(i)); + } + } + return Ok(None); + } else { + // If not ignoring nulls, return the first value if it exists. + return Ok((!value.is_empty()).then_some(0)); + } + } let sort_columns = ordering_values .iter() @@ -904,12 +924,13 @@ impl Accumulator for FirstValueAccumulator { if let Some(first_idx) = self.get_first_idx(values)? { let row = get_row_at_idx(values, first_idx)?; if !self.is_set - || compare_rows( - &self.orderings, - &row[1..], - &get_sort_options(self.ordering_req.as_ref()), - )? - .is_gt() + || (!self.requirement_satisfied + && compare_rows( + &self.orderings, + &row[1..], + &get_sort_options(self.ordering_req.as_ref()), + )? + .is_gt()) { self.update_with_new_row(&row); } @@ -1048,7 +1069,9 @@ impl AggregateUDFImpl for LastValue { ordering_req.clone(), acc_args.ignore_nulls, ) - .map(|acc| Box::new(acc) as _); + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }); } TrivialLastValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) .map(|acc| Box::new(acc) as _) @@ -1168,9 +1191,13 @@ struct LastValueAccumulator { // This information is used to discriminate genuine NULLs and NULLS that // occur due to empty partitions. is_set: bool, + // Stores values of the ordering columns corresponding to the first value. + // These values are used during merging of multiple partitions. orderings: Vec, // Stores the applicable ordering requirement. ordering_req: LexOrdering, + // Stores whether incoming data already satisfies the ordering requirement. + requirement_satisfied: bool, // Ignore null values. ignore_nulls: bool, } @@ -1192,6 +1219,7 @@ impl LastValueAccumulator { is_set: false, orderings, ordering_req, + requirement_satisfied: false, ignore_nulls, }) } @@ -1207,6 +1235,20 @@ impl LastValueAccumulator { let [value, ordering_values @ ..] = values else { return internal_err!("Empty row in LAST_VALUE"); }; + if self.requirement_satisfied { + // Get last entry according to the order of data: + if self.ignore_nulls { + // If ignoring nulls, find the last non-null value. + for i in (0..value.len()).rev() { + if !value.is_null(i) { + return Ok(Some(i)); + } + } + return Ok(None); + } else { + return Ok((!value.is_empty()).then_some(value.len() - 1)); + } + } let sort_columns = ordering_values .iter() @@ -1228,6 +1270,11 @@ impl LastValueAccumulator { Ok(max_ind) } + + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { + self.requirement_satisfied = requirement_satisfied; + self + } } impl Accumulator for LastValueAccumulator { @@ -1244,6 +1291,7 @@ impl Accumulator for LastValueAccumulator { let orderings = &row[1..]; // Update when there is a more recent entry if !self.is_set + || self.requirement_satisfied || compare_rows( &self.orderings, orderings, @@ -1281,6 +1329,7 @@ impl Accumulator for LastValueAccumulator { // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set + || self.requirement_satisfied || compare_rows(&self.orderings, last_ordering, &sort_options)?.is_lt() { // Update with last value in the state. Note that we should exclude the diff --git a/datafusion/sqllogictest/test_files/my_group_by.slt b/datafusion/sqllogictest/test_files/my_group_by.slt new file mode 100644 index 0000000000000..29e1cd425cf22 --- /dev/null +++ b/datafusion/sqllogictest/test_files/my_group_by.slt @@ -0,0 +1,57 @@ +# 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. + +# create a table for testing +statement ok +CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT + ) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) + +query TT +EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, + FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, + LAST_VALUE(amount ORDER BY amount DESC) AS fv2 + FROM sales_global + GROUP BY country + ORDER BY country +---- +logical_plan +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +04)------TableScan: sales_global projection=[country, amount] +physical_plan +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +09)----------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] From 198ffae1abc63b0e6669ca84c2a5e7276b06e7a4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 2 Apr 2025 21:30:47 +0300 Subject: [PATCH 049/167] Replace AsRef with ordinary & for LexOrdering --- .../aggregation_fuzzer/data_generator.rs | 2 +- .../tests/fuzz_cases/equivalence/ordering.rs | 6 ++--- .../fuzz_cases/equivalence/projection.rs | 2 +- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../functions-aggregate/src/first_last.rs | 22 ++++++++----------- .../functions-aggregate/src/nth_value.rs | 2 +- .../physical-expr-common/src/sort_expr.rs | 13 ++--------- .../physical-expr/src/equivalence/ordering.rs | 4 ++-- .../src/equivalence/properties/dependency.rs | 2 +- .../src/equivalence/properties/union.rs | 4 ++-- .../src/enforce_sorting/sort_pushdown.rs | 10 +++------ .../physical-plan/src/aggregates/row_hash.rs | 6 ++--- .../physical-plan/src/sorts/partial_sort.rs | 4 ++-- datafusion/physical-plan/src/sorts/sort.rs | 6 ++--- .../src/sorts/sort_preserving_merge.rs | 6 ++--- 15 files changed, 37 insertions(+), 54 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index d61835a0804ed..196f922096a2b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -173,7 +173,7 @@ impl DatasetGenerator { Ok(PhysicalSortExpr::new_default(col_expr)) }) .collect::>()?; - let sorted_batch = sort_batch(&base_batch, sort_exprs.as_ref(), None)?; + let sorted_batch = sort_batch(&base_batch, &sort_exprs, None)?; let batches = stagger_batch(sorted_batch); let dataset = Dataset::new(batches, sort_keys); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index c1e7e4e553d5d..1ab9bd559fc6a 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -75,7 +75,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(requirement.as_ref()), + eq_properties.ordering_satisfy(&requirement), expected, "{}", err_msg @@ -148,7 +148,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(requirement.as_ref()), + eq_properties.ordering_satisfy(&requirement), (expected | false), "{}", err_msg @@ -322,7 +322,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expected ); assert_eq!( - eq_properties.ordering_satisfy(required.as_ref()), + eq_properties.ordering_satisfy(&required), expected, "{err_msg}" ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index b43e104b90246..7ea9f7e243c6c 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -184,7 +184,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - projected_eq.ordering_satisfy(requirement.as_ref()), + projected_eq.ordering_satisfy(&requirement), expected, "{}", err_msg diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 89b5a7d60a0ba..423fe9e8a9ecf 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -461,7 +461,7 @@ pub fn generate_table_for_orderings( let batch = RecordBatch::try_from_iter(arrays)?; // Sort batch according to first ordering expression - let sort_columns = get_sort_columns(&batch, orderings[0].as_ref())?; + let sort_columns = get_sort_columns(&batch, &orderings[0])?; let sort_indices = lexsort_to_indices(&sort_columns, None)?; let mut batch = take_record_batch(&batch, &sort_indices)?; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index a91bf071001c4..764647e4136ac 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -346,7 +346,7 @@ where .map(ScalarValue::try_from) .collect::>>()?; - let sort_options = get_sort_options(ordering_req.as_ref()); + let sort_options = get_sort_options(&ordering_req); Ok(Self { null_builder: BooleanBufferBuilder::new(0), @@ -928,7 +928,7 @@ impl Accumulator for FirstValueAccumulator { && compare_rows( &self.orderings, &row[1..], - &get_sort_options(self.ordering_req.as_ref()), + &get_sort_options(&self.ordering_req), )? .is_gt()) { @@ -946,10 +946,8 @@ impl Accumulator for FirstValueAccumulator { let filtered_states = filter_states_according_to_is_set(&states[0..is_set_idx], flags)?; // 1..is_set_idx range corresponds to ordering section - let sort_columns = convert_to_sort_cols( - &filtered_states[1..is_set_idx], - self.ordering_req.as_ref(), - ); + let sort_columns = + convert_to_sort_cols(&filtered_states[1..is_set_idx], &self.ordering_req); let comparator = LexicographicalComparator::try_new(&sort_columns)?; let min = (0..filtered_states[0].len()).min_by(|&a, &b| comparator.compare(a, b)); @@ -958,7 +956,7 @@ impl Accumulator for FirstValueAccumulator { let first_row = get_row_at_idx(&filtered_states, first_idx)?; // When collecting orderings, we exclude the is_set flag from the state. let first_ordering = &first_row[1..is_set_idx]; - let sort_options = get_sort_options(self.ordering_req.as_ref()); + let sort_options = get_sort_options(&self.ordering_req); // Either there is no existing value, or there is an earlier version in new data. if !self.is_set || compare_rows(&self.orderings, first_ordering, &sort_options)?.is_gt() @@ -1295,7 +1293,7 @@ impl Accumulator for LastValueAccumulator { || compare_rows( &self.orderings, orderings, - &get_sort_options(self.ordering_req.as_ref()), + &get_sort_options(&self.ordering_req), )? .is_lt() { @@ -1313,10 +1311,8 @@ impl Accumulator for LastValueAccumulator { let filtered_states = filter_states_according_to_is_set(&states[0..is_set_idx], flags)?; // 1..is_set_idx range corresponds to ordering section - let sort_columns = convert_to_sort_cols( - &filtered_states[1..is_set_idx], - self.ordering_req.as_ref(), - ); + let sort_columns = + convert_to_sort_cols(&filtered_states[1..is_set_idx], &self.ordering_req); let comparator = LexicographicalComparator::try_new(&sort_columns)?; let max = (0..filtered_states[0].len()).max_by(|&a, &b| comparator.compare(a, b)); @@ -1325,7 +1321,7 @@ impl Accumulator for LastValueAccumulator { let last_row = get_row_at_idx(&filtered_states, last_idx)?; // When collecting orderings, we exclude the is_set flag from the state. let last_ordering = &last_row[1..is_set_idx]; - let sort_options = get_sort_options(self.ordering_req.as_ref()); + let sort_options = get_sort_options(&self.ordering_req); // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index cffc4d10f338f..746d32a092b64 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -357,7 +357,7 @@ impl NthValueAccumulator { } fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); + let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); let mut column_wise_ordering_values = vec![]; diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 2087a0d91c752..144d1f19f6f4d 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -192,7 +192,7 @@ impl PhysicalSortExpr { } /// Checks whether this sort expression satisfies the given `sort_expr`. - pub fn satisfy_expr(&self, sort_expr: &PhysicalSortExpr, schema: &Schema) -> bool { + pub fn satisfy_expr(&self, sort_expr: &Self, schema: &Schema) -> bool { // If the column is not nullable, NULLS FIRST/LAST is not important. let nullable = self.expr.nullable(schema).unwrap_or(true); self.expr.eq(&sort_expr.expr) @@ -340,12 +340,6 @@ pub struct LexOrdering { inner: Vec, } -impl AsRef for LexOrdering { - fn as_ref(&self) -> &LexOrdering { - self - } -} - impl LexOrdering { /// Creates a new [`LexOrdering`] from a vector pub fn new(inner: Vec) -> Self { @@ -441,10 +435,7 @@ impl LexOrdering { /// Transforms each `PhysicalSortExpr` in the `LexOrdering` /// in place using the provided closure `f`. - pub fn transform(&mut self, f: F) - where - F: FnMut(&mut PhysicalSortExpr), - { + pub fn transform(&mut self, f: F) { self.inner.iter_mut().for_each(f); } } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 00d74f897bd7b..d82a1394f1544 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -388,14 +388,14 @@ mod tests { Arc::clone(&input_schema), &[finer.clone()], ); - assert!(eq_properties_finer.ordering_satisfy(crude.as_ref())); + assert!(eq_properties_finer.ordering_satisfy(&crude)); // Crude ordering doesn't satisfy finer ordering. should return false let eq_properties_crude = EquivalenceProperties::new_with_orderings( Arc::clone(&input_schema), &[crude.clone()], ); - assert!(!eq_properties_crude.ordering_satisfy(finer.as_ref())); + assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index dcf3378f2da8e..4b8b72fd1e6f8 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1199,7 +1199,7 @@ mod tests { .collect::>()?; assert_eq!( - properties.ordering_satisfy(sort.as_ref()), + properties.ordering_satisfy(&sort), case.should_satisfy_ordering, "failed test '{}'", case.name diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 78edf4af21b5a..f32e18ee56fd0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -186,7 +186,7 @@ impl UnionEquivalentOrderingBuilder { ) -> AddedOrdering { if ordering.is_empty() { AddedOrdering::Yes - } else if properties.ordering_satisfy(ordering.as_ref()) { + } else if properties.ordering_satisfy(&ordering) { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); @@ -227,7 +227,7 @@ impl UnionEquivalentOrderingBuilder { &properties.constants, ) { if !augmented_ordering.is_empty() { - assert!(properties.ordering_satisfy(augmented_ordering.as_ref())); + assert!(properties.ordering_satisfy(&augmented_ordering)); self.orderings.push(augmented_ordering); } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index ff906e999adb0..895b93a351a92 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -292,15 +292,11 @@ fn pushdown_requirement_to_children( let left_columns_len = smj.left().schema().fields().len(); let parent_required_expr = LexOrdering::from(parent_required.lex_requirement().clone()); - match expr_source_side( - parent_required_expr.as_ref(), - smj.join_type(), - left_columns_len, - ) { + match expr_source_side(&parent_required_expr, smj.join_type(), left_columns_len) { Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required.lex_requirement(), - parent_required_expr.as_ref(), + &parent_required_expr, JoinSide::Left, ), Some(JoinSide::Right) => { @@ -314,7 +310,7 @@ fn pushdown_requirement_to_children( try_pushdown_requirements_to_join( smj, parent_required.lex_requirement(), - new_right_required_expr.as_ref(), + &new_right_required_expr, JoinSide::Right, ) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 907f99aa21a18..cfe26a7a34801 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -977,7 +977,7 @@ impl GroupedHashAggregateStream { let Some(emit) = self.emit(EmitTo::All, true)? else { return Ok(()); }; - let sorted = sort_batch(&emit, self.spill_state.spill_expr.as_ref(), None)?; + let sorted = sort_batch(&emit, &self.spill_state.spill_expr, None)?; // Spill sorted state to disk let spillfile = self.spill_state.spill_manager.spill_record_batch_by_size( @@ -1044,7 +1044,7 @@ impl GroupedHashAggregateStream { streams.push(Box::pin(RecordBatchStreamAdapter::new( Arc::clone(&schema), futures::stream::once(futures::future::lazy(move |_| { - sort_batch(&batch, expr.as_ref(), None) + sort_batch(&batch, &expr, None) })), ))); for spill in self.spill_state.spills.drain(..) { @@ -1055,7 +1055,7 @@ impl GroupedHashAggregateStream { self.input = StreamingMergeBuilder::new() .with_streams(streams) .with_schema(schema) - .with_expressions(self.spill_state.spill_expr.as_ref()) + .with_expressions(&self.spill_state.spill_expr) .with_metrics(self.baseline_metrics.clone()) .with_batch_size(self.batch_size) .with_reservation(self.reservation.new_empty()) diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 320fa21c86656..58e210b53eaa1 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -159,7 +159,7 @@ impl PartialSortExec { /// Sort expressions pub fn expr(&self) -> &LexOrdering { - self.expr.as_ref() + &self.expr } /// If `Some(fetch)`, limits output to only the first "fetch" items @@ -420,7 +420,7 @@ impl PartialSortStream { fn sort_in_mem_batches(self: &mut Pin<&mut Self>) -> Result { let input_batch = concat_batches(&self.schema(), &self.in_mem_batches)?; self.in_mem_batches.clear(); - let result = sort_batch(&input_batch, self.expr.as_ref(), self.fetch)?; + let result = sort_batch(&input_batch, &self.expr, self.fetch)?; if let Some(remaining_fetch) = self.fetch { // remaining_fetch - result.num_rows() is always be >= 0 // because result length of sort_batch with limit cannot be diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 57dce2e1e8a67..b50ca990ab1a3 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -383,7 +383,7 @@ impl ExternalSorter { StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(expressions.as_ref()) + .with_expressions(&expressions) .with_metrics(self.metrics.baseline.clone()) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -737,7 +737,7 @@ impl ExternalSorter { StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(expressions.as_ref()) + .with_expressions(&expressions) .with_metrics(metrics) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -1993,7 +1993,7 @@ mod tests { options: SortOptions::default(), }]); - let result = sort_batch(&batch, expressions.as_ref(), None).unwrap(); + let result = sort_batch(&batch, &expressions, None).unwrap(); assert_eq!(result.num_rows(), 1); } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index c0e350d4f8418..b9834c8f63d1f 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -145,7 +145,7 @@ impl SortPreservingMergeExec { /// Sort expressions pub fn expr(&self) -> &LexOrdering { - self.expr.as_ref() + &self.expr } /// Fetch @@ -324,7 +324,7 @@ impl ExecutionPlan for SortPreservingMergeExec { let result = StreamingMergeBuilder::new() .with_streams(receivers) .with_schema(schema) - .with_expressions(self.expr.as_ref()) + .with_expressions(&self.expr) .with_metrics(BaselineMetrics::new(&self.metrics, partition)) .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) @@ -1098,7 +1098,7 @@ mod tests { let merge_stream = StreamingMergeBuilder::new() .with_streams(streams) .with_schema(batches.schema()) - .with_expressions(sort.as_ref()) + .with_expressions(&sort) .with_metrics(BaselineMetrics::new(&metrics, 0)) .with_batch_size(task_ctx.session_config().batch_size()) .with_fetch(fetch) From 241ac0c6d3e2cb0671a6c37a77288eb0642888c1 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 2 Apr 2025 23:44:48 +0300 Subject: [PATCH 050/167] Further cleanup --- .../physical-expr-common/src/sort_expr.rs | 175 ++++++------------ .../physical-expr/src/equivalence/class.rs | 43 +++-- .../physical-expr/src/equivalence/ordering.rs | 4 +- .../src/equivalence/properties/mod.rs | 23 ++- .../src/equivalence/properties/union.rs | 10 +- datafusion/physical-optimizer/src/utils.rs | 8 +- .../physical-plan/src/aggregates/mod.rs | 5 +- datafusion/physical-plan/src/windows/mod.rs | 15 +- 8 files changed, 114 insertions(+), 169 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 144d1f19f6f4d..ee252cd9f07b9 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -19,7 +19,7 @@ use std::fmt::{self, Display, Formatter}; use std::hash::{Hash, Hasher}; -use std::ops::{Deref, DerefMut, Index, Range, RangeFrom, RangeTo}; +use std::ops::{Deref, DerefMut}; use std::sync::Arc; use std::vec::IntoIter; @@ -161,8 +161,7 @@ impl Display for PhysicalSortExpr { impl PhysicalSortExpr { /// evaluate the sort expression into SortColumn that can be passed into arrow sort kernel pub fn evaluate_to_sort_column(&self, batch: &RecordBatch) -> Result { - let value_to_sort = self.expr.evaluate(batch)?; - let array_to_sort = match value_to_sort { + let array_to_sort = match self.expr.evaluate(batch)? { ColumnarValue::Array(array) => array, ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(batch.num_rows())?, }; @@ -180,25 +179,23 @@ impl PhysicalSortExpr { requirement: &PhysicalSortRequirement, schema: &Schema, ) -> bool { - // If the column is not nullable, NULLS FIRST/LAST is not important. - let nullable = self.expr.nullable(schema).unwrap_or(true); let opts = &requirement.options; self.expr.eq(&requirement.expr) - && if nullable { + && if self.expr.nullable(schema).unwrap_or(true) { opts.is_none_or(|opts| self.options == opts) } else { + // If the column is not nullable, NULLS FIRST/LAST is not important. opts.is_none_or(|opts| self.options.descending == opts.descending) } } /// Checks whether this sort expression satisfies the given `sort_expr`. pub fn satisfy_expr(&self, sort_expr: &Self, schema: &Schema) -> bool { - // If the column is not nullable, NULLS FIRST/LAST is not important. - let nullable = self.expr.nullable(schema).unwrap_or(true); self.expr.eq(&sort_expr.expr) - && if nullable { + && if self.expr.nullable(schema).unwrap_or(true) { self.options == sort_expr.options } else { + // If the column is not nullable, NULLS FIRST/LAST is not important. self.options.descending == sort_expr.options.descending } } @@ -232,26 +229,6 @@ pub struct PhysicalSortRequirement { pub options: Option, } -impl From for PhysicalSortExpr { - /// If options is `None`, the default sort options `ASC, NULLS LAST` is used. - /// - /// The default is picked to be consistent with - /// PostgreSQL: - fn from(value: PhysicalSortRequirement) -> Self { - let options = value.options.unwrap_or(SortOptions { - descending: false, - nulls_first: false, - }); - PhysicalSortExpr::new(value.expr, options) - } -} - -impl From for PhysicalSortRequirement { - fn from(value: PhysicalSortExpr) -> Self { - Self::new(value.expr, Some(value.options)) - } -} - impl PartialEq for PhysicalSortRequirement { fn eq(&self, other: &Self) -> bool { self.options == other.options && self.expr.eq(&other.expr) @@ -329,6 +306,27 @@ fn to_str(options: &SortOptions) -> &str { } } +// Cross-conversion utilities between `PhysicalSortExpr` and `PhysicalSortRequirement` +impl From for PhysicalSortRequirement { + fn from(value: PhysicalSortExpr) -> Self { + Self::new(value.expr, Some(value.options)) + } +} + +impl From for PhysicalSortExpr { + /// The default sort options `ASC, NULLS LAST` when the requirement does + /// not specify sort options. This default is consistent with PostgreSQL. + /// + /// Reference: + fn from(value: PhysicalSortRequirement) -> Self { + let options = value.options.unwrap_or(SortOptions { + descending: false, + nulls_first: false, + }); + Self::new(value.expr, options) + } +} + ///`LexOrdering` contains a `Vec`, which represents /// a lexicographical ordering. /// @@ -347,66 +345,38 @@ impl LexOrdering { Self { inner } } - /// Returns the number of elements that can be stored in the LexOrdering - /// without reallocating. - pub fn capacity(&self) -> usize { - self.inner.capacity() - } - - /// Clears the LexOrdering, removing all elements. - pub fn clear(&mut self) { - self.inner.clear() - } - - /// Takes ownership of the actual vector of `PhysicalSortExpr`s in the LexOrdering. - pub fn take_exprs(self) -> Vec { - self.inner - } - - /// Returns `true` if the LexOrdering contains `expr` - pub fn contains(&self, expr: &PhysicalSortExpr) -> bool { - self.inner.contains(expr) + /// Appends an element to the back of the `LexOrdering`. + pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { + self.inner.push(physical_sort_expr) } /// Add all elements from `iter` to the LexOrdering. - pub fn extend>(&mut self, iter: I) { + pub fn extend(&mut self, iter: impl IntoIterator) { self.inner.extend(iter) } - /// Remove all elements from the LexOrdering where `f` evaluates to `false`. - pub fn retain(&mut self, f: F) - where - F: FnMut(&PhysicalSortExpr) -> bool, - { - self.inner.retain(f) - } - - /// Returns `true` if the LexOrdering contains no elements. - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - /// Returns an iterator over each `&PhysicalSortExpr` in the LexOrdering. - pub fn iter(&self) -> core::slice::Iter { - self.inner.iter() - } - - /// Returns the number of elements in the LexOrdering. - pub fn len(&self) -> usize { - self.inner.len() + /// Returns the number of elements that can be stored in the `LexOrdering` + /// without reallocating. + pub fn capacity(&self) -> usize { + self.inner.capacity() } - /// Removes the last element from the LexOrdering and returns it, or `None` if it is empty. - pub fn pop(&mut self) -> Option { - self.inner.pop() + /// Remove all elements from the LexOrdering where `f` evaluates to `false`. + pub fn retain bool>(mut self, f: F) -> Option { + self.inner.retain(f); + (!self.inner.is_empty()).then_some(self) } - /// Appends an element to the back of the LexOrdering. - pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { - self.inner.push(physical_sort_expr) + /// Removes the last element from the `LexOrdering` and returns it along + /// with the resulting `LexOrdering`. If the `LexOrdering` becomes + /// degenerate, returns `None` as the first element of the tuple. + pub fn pop(mut self) -> (Option, PhysicalSortExpr) { + // The vector is always non-empty, so the `unwrap` call is safe. + let sort_expr = self.inner.pop().unwrap(); + ((!self.inner.is_empty()).then_some(self), sort_expr) } - /// Truncates the LexOrdering, keeping only the first `len` elements. + /// Truncates the `LexOrdering`, keeping only the first `len` elements. pub fn truncate(&mut self, len: usize) { self.inner.truncate(len) } @@ -488,38 +458,6 @@ impl FromIterator for LexOrdering { } } -impl Index for LexOrdering { - type Output = PhysicalSortExpr; - - fn index(&self, index: usize) -> &Self::Output { - &self.inner[index] - } -} - -impl Index> for LexOrdering { - type Output = [PhysicalSortExpr]; - - fn index(&self, range: Range) -> &Self::Output { - &self.inner[range] - } -} - -impl Index> for LexOrdering { - type Output = [PhysicalSortExpr]; - - fn index(&self, range_from: RangeFrom) -> &Self::Output { - &self.inner[range_from] - } -} - -impl Index> for LexOrdering { - type Output = [PhysicalSortExpr]; - - fn index(&self, range_to: RangeTo) -> &Self::Output { - &self.inner[range_to] - } -} - impl IntoIterator for LexOrdering { type Item = PhysicalSortExpr; type IntoIter = IntoIter; @@ -551,14 +489,6 @@ impl LexRequirement { Self { inner } } - pub fn is_empty(&self) -> bool { - self.inner.is_empty() - } - - pub fn iter(&self) -> impl Iterator { - self.inner.iter() - } - pub fn push(&mut self, physical_sort_requirement: PhysicalSortRequirement) { self.inner.push(physical_sort_requirement) } @@ -621,7 +551,16 @@ impl IntoIterator for LexRequirement { } } -// Cross-conversions utilities between `LexOrdering` and `LexRequirement` +impl<'a> IntoIterator for &'a LexRequirement { + type Item = &'a PhysicalSortRequirement; + type IntoIter = std::slice::Iter<'a, PhysicalSortRequirement>; + + fn into_iter(self) -> Self::IntoIter { + self.inner.iter() + } +} + +// Cross-conversion utilities between `LexOrdering` and `LexRequirement` impl From for LexRequirement { fn from(value: LexOrdering) -> Self { Self::new(value.into_iter().map(Into::into).collect()) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 4f022bf0a7e93..a2135fa7946a2 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::Display; +use std::sync::Arc; +use std::vec::IntoIter; + use super::{add_offset_to_expr, ProjectionMapping}; -use crate::{ - expressions::Column, LexOrdering, LexRequirement, PhysicalExpr, PhysicalExprRef, - PhysicalSortExpr, PhysicalSortRequirement, -}; +use crate::expressions::Column; +use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; + use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{JoinType, ScalarValue}; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; -use std::fmt::Display; -use std::sync::Arc; -use std::vec::IntoIter; use indexmap::{IndexMap, IndexSet}; @@ -512,28 +512,27 @@ impl EquivalenceGroup { /// This function applies the `normalize_sort_expr` function for all sort /// expressions in `sort_exprs` and returns the corresponding normalized /// sort expressions. - pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { - LexOrdering::new( - sort_exprs - .iter() - .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) - .collect(), - ) - .collapse() + pub fn normalize_sort_exprs<'a>( + &self, + sort_exprs: impl IntoIterator, + ) -> Vec { + sort_exprs + .into_iter() + .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) + .collect() } /// This function applies the `normalize_sort_requirement` function for all /// requirements in `sort_reqs` and returns the corresponding normalized /// sort requirements. - pub fn normalize_sort_requirements( + pub fn normalize_sort_requirements<'a>( &self, - sort_reqs: &LexRequirement, - ) -> Option { - let reqs = sort_reqs - .iter() + sort_reqs: impl IntoIterator, + ) -> Vec { + sort_reqs + .into_iter() .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect::>(); - (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) + .collect() } /// Projects `expr` according to the given projection mapping. diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d82a1394f1544..a6f22e46b9c14 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -682,7 +682,7 @@ mod tests { let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs.into()), + eq_properties.ordering_satisfy(&reqs), expected, "{}", err_msg @@ -742,7 +742,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs.into()), + eq_properties.ordering_satisfy(&reqs), expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 4629505d608a3..87c4746c44964 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -213,10 +213,9 @@ impl EquivalenceProperties { pub fn output_ordering(&self) -> Option { // Prune out constant expressions: let constants = self.constants(); - let mut output_ordering = self.oeq_class().output_ordering()?; - output_ordering - .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)); - (!output_ordering.is_empty()).then_some(output_ordering) + self.oeq_class() + .output_ordering()? + .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)) } /// Returns the normalized version of the ordering equivalence class within. @@ -497,7 +496,10 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> Option { + pub fn normalize_sort_exprs<'a>( + &self, + sort_exprs: impl IntoIterator, + ) -> Option { let normalized_sort_exprs = self.eq_group.normalize_sort_exprs(sort_exprs); let mut constant_exprs = vec![]; constant_exprs.extend( @@ -528,11 +530,11 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - fn normalize_sort_requirements( + fn normalize_sort_requirements<'a>( &self, - sort_reqs: &LexRequirement, + sort_reqs: impl IntoIterator, ) -> Option { - let normalized_reqs = self.eq_group.normalize_sort_requirements(sort_reqs)?; + let normalized_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); let mut constant_exprs = vec![]; constant_exprs.extend( self.constants @@ -550,7 +552,10 @@ impl EquivalenceProperties { /// Checks whether the given ordering is satisfied by any of the existing /// orderings. - pub fn ordering_satisfy(&self, given: &LexOrdering) -> bool { + pub fn ordering_satisfy<'a>( + &self, + given: impl IntoIterator, + ) -> bool { // First, standardize the given ordering: let Some(normalized_ordering) = self.normalize_sort_exprs(given) else { // If the requirement vanishes after normalization, it is satisfied diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index f32e18ee56fd0..6a0cafc20a08a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -158,11 +158,13 @@ impl UnionEquivalentOrderingBuilder { for mut ordering in orderings.into_iter() { // Progressively shorten the ordering to search for a satisfied prefix: loop { - match self.try_add_ordering(ordering, constants, properties) { + ordering = match self.try_add_ordering(ordering, constants, properties) { AddedOrdering::Yes => break, - AddedOrdering::No(o) => { - ordering = o; - ordering.pop(); + AddedOrdering::No(ordering) => { + let (Some(short), _) = ordering.pop() else { + break; + }; + short } } } diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 57a193315a5c3..688ed1aacf8ed 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -40,13 +40,15 @@ pub fn add_sort_above( sort_requirements: LexRequirement, fetch: Option, ) -> PlanContext { - let mut sort_expr = LexOrdering::from(sort_requirements); - sort_expr.retain(|sort_expr| { + let sort_expr = LexOrdering::from(sort_requirements); + let Some(sort_expr) = sort_expr.retain(|sort_expr| { !node .plan .equivalence_properties() .is_expr_constant(&sort_expr.expr) - }); + }) else { + return node; + }; let mut new_sort = SortExec::new(sort_expr, Arc::clone(&node.plan)).with_fetch(fetch); if node.plan.output_partitioning().partition_count() > 1 { new_sort = new_sort.with_preserve_partitioning(true); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1da75bc5037d5..51efd62269fe5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1116,7 +1116,8 @@ pub fn get_finer_aggregate_exprs_requirement( } else { requirement.clone() }; - if let Some(finer_ordering) = finer.take_if(|o| eq_properties.ordering_satisfy(o)) + if let Some(finer_ordering) = + finer.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) { // Requirement is satisfied by the existing ordering: requirement = Some(finer_ordering); @@ -1139,7 +1140,7 @@ pub fn get_finer_aggregate_exprs_requirement( requirement.clone() }; if let Some(finer_ordering) = - finer_rev.take_if(|o| eq_properties.ordering_satisfy(o)) + finer_rev.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) { // Reverse requirement is satisfied by the existing ordering. // Hence, we need to reverse the aggregate expression: diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 58cc9ae820ca2..09e70b069687a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -402,14 +402,13 @@ pub(crate) fn window_equivalence_properties( // Window function results in a partial constant value in // some ordering. Adjust the ordering equivalences accordingly: let new_lexs = all_satisfied_lexs.into_iter().flat_map(|lex| { - let orderings = lex.take_exprs(); let new_partial_consts = sort_options_resolving_constant(Arc::new(window_col.clone())); new_partial_consts.into_iter().map(move |partial| { - let mut existing = orderings.clone(); + let mut existing = lex.clone(); existing.push(partial); - LexOrdering::new(existing) + existing }) }); window_eq_properties.add_new_orderings(new_lexs); @@ -446,14 +445,12 @@ pub(crate) fn window_equivalence_properties( window_eq_properties.add_new_orderings(new_ordering); } else { // Reverse set-monotonic cases for all orderings: - for lex in all_satisfied_lexs.into_iter() { - let mut existing = lex.take_exprs(); - existing.push(PhysicalSortExpr::new( + for mut lex in all_satisfied_lexs.into_iter() { + lex.push(PhysicalSortExpr::new( Arc::new(window_col.clone()), SortOptions::new(increasing, true), )); - window_eq_properties - .add_new_ordering(LexOrdering::new(existing)); + window_eq_properties.add_new_ordering(lex); } } } @@ -476,7 +473,7 @@ pub(crate) fn window_equivalence_properties( if let Some(f) = order.first() { asc = !f.options.descending; } - window_eq_properties.ordering_satisfy(&LexOrdering::new(order)) + window_eq_properties.ordering_satisfy(&order) }) { let increasing = set_monotonicity.eq(&SetMonotonicity::Increasing); From 4c494d75aae533c83a0fa5f34e4975514dd02104 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 3 Apr 2025 10:33:54 +0300 Subject: [PATCH 051/167] add OutputRequirementExec fetches to sort adding --- .../physical-optimizer/src/enforce_distribution.rs | 5 ++++- .../physical-optimizer/src/enforce_sorting/mod.rs | 10 +++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index a66a77b3b24e3..acbe8a4ff6849 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1294,7 +1294,10 @@ pub fn ensure_distribution( child = add_sort_above_with_check( child, required_input_ordering.lex_requirement().clone(), - None, + plan.as_any() + .downcast_ref::() + .map(|output| output.fetch()) + .unwrap_or(None), ); } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 32dea9e18967b..b87d1b8bffba3 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -46,6 +46,7 @@ use crate::enforce_sorting::replace_with_order_preserving_variants::{ use crate::enforce_sorting::sort_pushdown::{ assign_initial_requirements, pushdown_sorts, SortPushDown, }; +use crate::output_requirements::OutputRequirementExec; use crate::utils::{ add_sort_above, add_sort_above_with_check, is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, @@ -491,7 +492,14 @@ pub fn ensure_sorting( if physical_ordering.is_some() { child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; } - child = add_sort_above(child, required.lex_requirement().clone(), None); + child = add_sort_above( + child, + required.lex_requirement().clone(), + plan.as_any() + .downcast_ref::() + .map(|output| output.fetch()) + .unwrap_or(None), + ); child = update_sort_ctx_children_data(child, true)?; } } else if physical_ordering.is_none() From 99755c745a13be6065793547ecb524ecdd71c2c3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 3 Apr 2025 10:26:19 +0300 Subject: [PATCH 052/167] Simplify remove_redundant_entries --- .../functions-aggregate/src/first_last.rs | 38 +----- .../functions-aggregate/src/nth_value.rs | 127 +++++++----------- .../physical-expr-common/src/sort_expr.rs | 10 +- .../physical-expr/src/equivalence/ordering.rs | 43 +++--- .../src/equivalence/properties/union.rs | 10 +- .../src/sorts/streaming_merge.rs | 2 +- datafusion/proto/src/logical_plan/to_proto.rs | 20 +-- 7 files changed, 96 insertions(+), 154 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 764647e4136ac..82720ba6425c3 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -320,8 +320,6 @@ where ordering_req: LexOrdering, // derived from `ordering_req`. sort_options: Vec, - // Stores whether incoming data already satisfies the ordering requirement. - input_requirement_satisfied: bool, // Ignore null values. ignore_nulls: bool, /// The output type @@ -339,12 +337,10 @@ where data_type: &DataType, ordering_dtypes: &[DataType], ) -> Result { - let requirement_satisfied = ordering_req.is_empty(); - let default_orderings = ordering_dtypes .iter() .map(ScalarValue::try_from) - .collect::>>()?; + .collect::>()?; let sort_options = get_sort_options(&ordering_req); @@ -352,7 +348,6 @@ where null_builder: BooleanBufferBuilder::new(0), ordering_req, sort_options, - input_requirement_satisfied: requirement_satisfied, ignore_nulls, default_orderings, data_type: data_type.clone(), @@ -364,18 +359,6 @@ where }) } - fn need_update(&self, group_idx: usize) -> bool { - if !self.is_sets.get_bit(group_idx) { - return true; - } - - if self.ignore_nulls && !self.null_builder.get_bit(group_idx) { - return true; - } - - !self.input_requirement_satisfied - } - fn should_update_state( &self, group_idx: usize, @@ -536,31 +519,24 @@ where let group_idx = *group_idx; let passed_filter = opt_filter.is_none_or(|x| x.value(idx_in_val)); - let is_set = is_set_arr.is_none_or(|x| x.value(idx_in_val)); if !passed_filter || !is_set { continue; } - if !self.need_update(group_idx) { - continue; - } - if self.ignore_nulls && vals.is_null(idx_in_val) { continue; } - let is_valid = self.min_of_each_group_buf.1.get_bit(group_idx); - if is_valid - && comparator - .compare(self.min_of_each_group_buf.0[group_idx], idx_in_val) - .is_gt() - { - self.min_of_each_group_buf.0[group_idx] = idx_in_val; - } else if !is_valid { + if !self.min_of_each_group_buf.1.get_bit(group_idx) { self.min_of_each_group_buf.1.set_bit(group_idx, true); self.min_of_each_group_buf.0[group_idx] = idx_in_val; + } else if comparator + .compare(self.min_of_each_group_buf.0[group_idx], idx_in_val) + .is_gt() + { + self.min_of_each_group_buf.0[group_idx] = idx_in_val; } } diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 746d32a092b64..e353a6b622913 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -446,91 +446,60 @@ impl Accumulator for NthValueAccumulator { if states.is_empty() { return Ok(()); } - // First entry in the state is the aggregation result. - let array_agg_values = &states[0]; - let n_required = self.n.unsigned_abs() as usize; - if self.ordering_req.is_empty() { - let array_agg_res = - ScalarValue::convert_array_to_scalar_vec(array_agg_values)?; - for v in array_agg_res.into_iter() { - self.values.extend(v); - if self.values.len() > n_required { - // There is enough data collected can stop merging - break; - } - } - } else if let Some(agg_orderings) = states[1].as_list_opt::() { - // 2nd entry stores values received for ordering requirement columns, for each aggregation value inside NTH_VALUE list. - // For each `StructArray` inside NTH_VALUE list, we will receive an `Array` that stores - // values received from its ordering requirement expression. (This information is necessary for during merging). - - // Stores NTH_VALUE results coming from each partition - let mut partition_values: Vec> = vec![]; - // Stores ordering requirement expression results coming from each partition - let mut partition_ordering_values: Vec>> = vec![]; - - // Existing values should be merged also. - partition_values.push(self.values.clone()); - - partition_ordering_values.push(self.ordering_values.clone()); - - let array_agg_res = - ScalarValue::convert_array_to_scalar_vec(array_agg_values)?; - - for v in array_agg_res.into_iter() { - partition_values.push(v.into()); - } - - let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; - - let ordering_values = orderings.into_iter().map(|partition_ordering_rows| { - // Extract value from struct to ordering_rows for each group/partition - partition_ordering_rows.into_iter().map(|ordering_row| { - if let ScalarValue::Struct(s) = ordering_row { - let mut ordering_columns_per_row = vec![]; - - for column in s.columns() { - let sv = ScalarValue::try_from_array(column, 0)?; - ordering_columns_per_row.push(sv); - } - - Ok(ordering_columns_per_row) - } else { - exec_err!( - "Expects to receive ScalarValue::Struct(Some(..), _) but got: {:?}", - ordering_row.data_type() - ) - } - }).collect::>>() - }).collect::>>()?; - for ordering_values in ordering_values.into_iter() { - partition_ordering_values.push(ordering_values.into()); - } - - let sort_options = self - .ordering_req - .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(); - let (new_values, new_orderings) = merge_ordered_arrays( - &mut partition_values, - &mut partition_ordering_values, - &sort_options, - )?; - self.values = new_values.into(); - self.ordering_values = new_orderings.into(); - } else { + // Second entry stores values received for ordering requirement columns + // for each aggregation value inside NTH_VALUE list. For each `StructArray` + // inside this list, we will receive an `Array` that stores values received + // from its ordering requirement expression. This information is necessary + // during merging. + let Some(agg_orderings) = states[1].as_list_opt::() else { return exec_err!("Expects to receive a list array"); + }; + + // Stores NTH_VALUE results coming from each partition + let mut partition_values = vec![self.values.clone()]; + // First entry in the state is the aggregation result. + let array_agg_res = ScalarValue::convert_array_to_scalar_vec(&states[0])?; + for v in array_agg_res.into_iter() { + partition_values.push(v.into()); + } + // Stores ordering requirement expression results coming from each partition: + let mut partition_ordering_values = vec![self.ordering_values.clone()]; + let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; + // Extract value from struct to ordering_rows for each group/partition: + for partition_ordering_rows in orderings.into_iter() { + let ordering_values = partition_ordering_rows.into_iter().map(|ordering_row| { + let ScalarValue::Struct(s_array) = ordering_row else { + return exec_err!( + "Expects to receive ScalarValue::Struct(Some(..), _) but got: {:?}", + ordering_row.data_type() + ); + }; + s_array + .columns() + .iter() + .map(|column| ScalarValue::try_from_array(column, 0)) + .collect() + }).collect::>>()?; + partition_ordering_values.push(ordering_values); } + + let sort_options = self + .ordering_req + .iter() + .map(|sort_expr| sort_expr.options) + .collect::>(); + let (new_values, new_orderings) = merge_ordered_arrays( + &mut partition_values, + &mut partition_ordering_values, + &sort_options, + )?; + self.values = new_values.into(); + self.ordering_values = new_orderings.into(); Ok(()) } fn state(&mut self) -> Result> { - let mut result = vec![self.evaluate_values()]; - if !self.ordering_req.is_empty() { - result.push(self.evaluate_orderings()?); - } - Ok(result) + Ok(vec![self.evaluate_values(), self.evaluate_orderings()?]) } fn evaluate(&mut self) -> Result { diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index ee252cd9f07b9..58e93a8babbb4 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -28,7 +28,7 @@ use crate::physical_expr::{fmt_sql, PhysicalExpr}; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::Result; +use datafusion_common::{exec_err, Result}; use datafusion_expr_common::columnar_value::ColumnarValue; use itertools::Itertools; @@ -377,8 +377,12 @@ impl LexOrdering { } /// Truncates the `LexOrdering`, keeping only the first `len` elements. - pub fn truncate(&mut self, len: usize) { - self.inner.truncate(len) + pub fn truncate(&mut self, len: usize) -> Result<()> { + if len == 0 { + return exec_err!("Degenerate LexOrdering instances are not allowed"); + } + self.inner.truncate(len); + Ok(()) } /// Merge the contents of `other` into `self`, removing duplicates. diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index a6f22e46b9c14..5104f55516b36 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -133,27 +133,26 @@ impl OrderingEquivalenceClass { while work { work = false; let mut idx = 0; - while idx < self.orderings.len() { + 'outer: while idx < self.orderings.len() { let mut ordering_idx = idx + 1; - let mut removal = self.orderings[idx].is_empty(); while ordering_idx < self.orderings.len() { - work |= self.resolve_overlap(idx, ordering_idx); - if self.orderings[idx].is_empty() { - removal = true; - break; + if let Some(remove) = self.resolve_overlap(idx, ordering_idx) { + work = true; + if remove { + self.orderings.swap_remove(idx); + continue 'outer; + } } - work |= self.resolve_overlap(ordering_idx, idx); - if self.orderings[ordering_idx].is_empty() { - self.orderings.swap_remove(ordering_idx); - } else { - ordering_idx += 1; + if let Some(remove) = self.resolve_overlap(ordering_idx, idx) { + work = true; + if remove { + self.orderings.swap_remove(ordering_idx); + continue; + } } + ordering_idx += 1; } - if removal { - self.orderings.swap_remove(idx); - } else { - idx += 1; - } + idx += 1; } } } @@ -164,18 +163,22 @@ impl OrderingEquivalenceClass { /// For example, if `orderings[idx]` is `[a ASC, b ASC, c DESC]` and /// `orderings[pre_idx]` is `[b ASC, c DESC]`, then the function will trim /// `orderings[idx]` to `[a ASC]`. - fn resolve_overlap(&mut self, idx: usize, pre_idx: usize) -> bool { + fn resolve_overlap(&mut self, idx: usize, pre_idx: usize) -> Option { let length = self.orderings[idx].len(); let other_length = self.orderings[pre_idx].len(); for overlap in 1..=length.min(other_length) { if self.orderings[idx][length - overlap..] == self.orderings[pre_idx][..overlap] { - self.orderings[idx].truncate(length - overlap); - return true; + return if length == overlap { + Some(true) + } else { + self.orderings[idx].truncate(length - overlap).unwrap(); + Some(false) + }; } } - false + None } /// Returns the concatenation of all the orderings. This enables merge diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 6a0cafc20a08a..73523a5044361 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -186,9 +186,7 @@ impl UnionEquivalentOrderingBuilder { constants: &[ConstExpr], properties: &EquivalenceProperties, ) -> AddedOrdering { - if ordering.is_empty() { - AddedOrdering::Yes - } else if properties.ordering_satisfy(&ordering) { + if properties.ordering_satisfy(&ordering) { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); @@ -228,10 +226,8 @@ impl UnionEquivalentOrderingBuilder { existing_ordering, &properties.constants, ) { - if !augmented_ordering.is_empty() { - assert!(properties.ordering_satisfy(&augmented_ordering)); - self.orderings.push(augmented_ordering); - } + assert!(properties.ordering_satisfy(&augmented_ordering)); + self.orderings.push(augmented_ordering); } } diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 05dca0b291303..b74954eb9688d 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -83,7 +83,7 @@ impl<'a> StreamingMergeBuilder<'a> { } pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { - self.expressions = (!expressions.is_empty()).then_some(expressions); + self.expressions = Some(expressions); self } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 841c31fa035f4..bbe5889c0c9fa 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -314,28 +314,22 @@ pub fn serialize_expr( null_treatment: _, }, }) => { - let (window_function, fun_definition) = match fun { + let mut buf = Vec::new(); + let window_function = match fun { WindowFunctionDefinition::AggregateUDF(aggr_udf) => { - let mut buf = Vec::new(); let _ = codec.try_encode_udaf(aggr_udf, &mut buf); - ( - protobuf::window_expr_node::WindowFunction::Udaf( - aggr_udf.name().to_string(), - ), - (!buf.is_empty()).then_some(buf), + protobuf::window_expr_node::WindowFunction::Udaf( + aggr_udf.name().to_string(), ) } WindowFunctionDefinition::WindowUDF(window_udf) => { - let mut buf = Vec::new(); let _ = codec.try_encode_udwf(window_udf, &mut buf); - ( - protobuf::window_expr_node::WindowFunction::Udwf( - window_udf.name().to_string(), - ), - (!buf.is_empty()).then_some(buf), + protobuf::window_expr_node::WindowFunction::Udwf( + window_udf.name().to_string(), ) } }; + let fun_definition = (!buf.is_empty()).then_some(buf); let partition_by = serialize_exprs(partition_by, codec)?; let order_by = serialize_sorts(order_by, codec)?; From 99223ae4952f9bec58d9b3c21aa3c33f0cb0b289 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 3 Apr 2025 11:34:52 +0300 Subject: [PATCH 053/167] Work with iterators in ordering_satisfy_requirement --- .../src/equivalence/properties/mod.rs | 5 +++- .../src/enforce_sorting/mod.rs | 6 ++--- .../src/update_aggr_exprs.rs | 27 ++++++++++--------- datafusion/physical-plan/src/windows/mod.rs | 9 ++----- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 87c4746c44964..a246b38e8d9c4 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -608,7 +608,10 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - pub fn ordering_satisfy_requirement(&self, given: &LexRequirement) -> bool { + pub fn ordering_satisfy_requirement<'a>( + &self, + given: impl IntoIterator, + ) -> bool { // First, standardize the given requirement: let Some(normalized_reqs) = self.normalize_sort_requirements(given) else { // If the requirement vanishes after normalization, it is satisfied diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index b87d1b8bffba3..6b09e5274f1c9 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -282,9 +282,9 @@ fn replace_with_partial_sort( let sort_req = LexRequirement::from(sort_plan.expr().clone()); let mut common_prefix_length = 0; - while child_eq_properties.ordering_satisfy_requirement(&LexRequirement::new( - sort_req[0..common_prefix_length + 1].to_vec(), - )) { + while child_eq_properties + .ordering_satisfy_requirement(&sort_req[0..common_prefix_length + 1]) + { common_prefix_length += 1; } if common_prefix_length > 0 { diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index eeb202b8a410b..9209c2121a6ce 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -24,7 +24,6 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; -use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr::{ reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; @@ -167,21 +166,23 @@ fn try_convert_aggregate_if_better( if !aggr_expr.order_sensitivity().is_beneficial() { Ok(aggr_expr) } else if let Some(order_bys) = order_bys { - let aggr_sort_reqs = LexRequirement::from(order_bys.clone()); - let reqs = LexRequirement::new(concat_slices( + if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, - &aggr_sort_reqs, - )); - - if eq_properties.ordering_satisfy_requirement(&reqs) { + &order_bys + .clone() + .into_iter() + .map(Into::into) + .collect::>(), + )) { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) - } else if eq_properties.ordering_satisfy_requirement( - &LexRequirement::new(concat_slices( - prefix_requirement, - &LexRequirement::from(reverse_order_bys(order_bys)), - )), - ) { + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_order_bys(order_bys) + .into_iter() + .map(Into::into) + .collect::>(), + )) { // Converting to reverse enables more efficient execution // given the existing ordering (if possible): aggr_expr diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 09e70b069687a..ae17c9709147e 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -48,7 +48,6 @@ use datafusion_physical_expr::{ window::{SlidingAggregateWindowExpr, StandardWindowFunctionExpr}, ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use itertools::Itertools; @@ -374,9 +373,8 @@ pub(crate) fn window_equivalence_properties( .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))); let all_satisfied_lexs = partition_by_orders .multi_cartesian_product() - .filter(|lex| !lex.is_empty()) + .filter(|lex| !lex.is_empty() && window_eq_properties.ordering_satisfy(lex)) .map(LexOrdering::new) - .filter(|lex| window_eq_properties.ordering_satisfy(lex)) .collect::>(); // If there is a partitioning, and no possible ordering cannot satisfy // the input plan's orderings, then we cannot further introduce any @@ -614,10 +612,7 @@ pub fn get_window_mode( if let Some(ob) = orderbys { req.extend(ob.iter().cloned().map(Into::into)); } - if req.is_empty() - || partition_by_eqs - .ordering_satisfy_requirement(&LexRequirement::new(req).collapse()) - { + if req.is_empty() || partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { InputOrderMode::Sorted From cef12cad2badae14e57944d1458a9a0551a0c226 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 3 Apr 2025 13:13:06 +0300 Subject: [PATCH 054/167] Fix doctests --- datafusion/physical-expr/src/aggregate.rs | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index b60e1fc5678d6..489c1c12d0558 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -128,25 +128,28 @@ impl AggregateExprBuilder { /// # impl AggregateUDFImpl for FirstValueUdf { /// # fn as_any(&self) -> &dyn Any { /// # unimplemented!() - /// # } + /// # } + /// # /// # fn name(&self) -> &str { /// # unimplemented!() - /// } + /// # } + /// # /// # fn signature(&self) -> &Signature { /// # unimplemented!() - /// # } + /// # } + /// # /// # fn return_type(&self, args: &[DataType]) -> Result { /// # unimplemented!() /// # } - /// # + /// # /// # fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { /// # unimplemented!() - /// # } - /// # + /// # } + /// # /// # fn state_fields(&self, args: StateFieldsArgs) -> Result> { /// # unimplemented!() /// # } - /// # + /// # /// # fn documentation(&self) -> Option<&Documentation> { /// # unimplemented!() /// # } @@ -169,16 +172,16 @@ impl AggregateExprBuilder { /// }]; /// /// let first_value = AggregateUDF::from(FirstValueUdf::new()); - /// + /// /// let aggregate_expr = AggregateExprBuilder::new( /// Arc::new(first_value), /// args /// ) - /// .order_by(order_by.into()) + /// .order_by(Some(order_by.into())) /// .alias("first_a_by_x") /// .ignore_nulls() /// .build()?; - /// + /// /// Ok(()) /// } /// ``` From 4ae0807652618acccc26c0f2c9ddd0cb0ed2e452 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 3 Apr 2025 19:47:44 +0300 Subject: [PATCH 055/167] Cleanup LexOrdering APIs --- .../physical-expr-common/src/sort_expr.rs | 62 ++++----- .../src/equivalence/properties/mod.rs | 6 +- .../src/equivalence/properties/union.rs | 123 ++++++++---------- datafusion/physical-optimizer/src/utils.rs | 13 +- 4 files changed, 100 insertions(+), 104 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 58e93a8babbb4..d9422fbcaf66c 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -339,7 +339,8 @@ pub struct LexOrdering { } impl LexOrdering { - /// Creates a new [`LexOrdering`] from a vector + /// Creates a new [`LexOrdering`] from the given vector of sort expressions. + /// The vector must not be empty. pub fn new(inner: Vec) -> Self { debug_assert!(!inner.is_empty()); Self { inner } @@ -350,7 +351,7 @@ impl LexOrdering { self.inner.push(physical_sort_expr) } - /// Add all elements from `iter` to the LexOrdering. + /// Add all elements from `iter` to the `LexOrdering`. pub fn extend(&mut self, iter: impl IntoIterator) { self.inner.extend(iter) } @@ -361,19 +362,10 @@ impl LexOrdering { self.inner.capacity() } - /// Remove all elements from the LexOrdering where `f` evaluates to `false`. - pub fn retain bool>(mut self, f: F) -> Option { - self.inner.retain(f); - (!self.inner.is_empty()).then_some(self) - } - - /// Removes the last element from the `LexOrdering` and returns it along - /// with the resulting `LexOrdering`. If the `LexOrdering` becomes - /// degenerate, returns `None` as the first element of the tuple. - pub fn pop(mut self) -> (Option, PhysicalSortExpr) { - // The vector is always non-empty, so the `unwrap` call is safe. - let sort_expr = self.inner.pop().unwrap(); - ((!self.inner.is_empty()).then_some(self), sort_expr) + /// Takes ownership of the underlying vector of sort expressions and + /// returns it. + pub fn take(self) -> Vec { + self.inner } /// Truncates the `LexOrdering`, keeping only the first `len` elements. @@ -391,12 +383,11 @@ impl LexOrdering { self } - /// Collapse a `LexOrdering` into a new duplicate-free `LexOrdering` based on expression. + /// Constructs a duplicate-free `LexOrdering` by filtering out duplicate + /// entries that have same physical expression inside. /// - /// This function filters duplicate entries that have same physical - /// expression inside, ignoring [`SortOptions`]. For example: - /// - /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. + /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a + /// Some(ASC)]`. pub fn collapse(self) -> Self { let mut orderings = Vec::::new(); for element in self { @@ -488,24 +479,37 @@ pub struct LexRequirement { } impl LexRequirement { + /// Creates a new [`LexOrdering`] from the given vector of sort requirements. + /// The vector must not be empty. pub fn new(inner: Vec) -> Self { debug_assert!(!inner.is_empty()); Self { inner } } - pub fn push(&mut self, physical_sort_requirement: PhysicalSortRequirement) { - self.inner.push(physical_sort_requirement) + /// Appends an element to the back of the `LexRequirement`. + pub fn push(&mut self, requirement: PhysicalSortRequirement) { + self.inner.push(requirement) + } + + /// Add all elements from `iter` to the `LexRequirement`. + pub fn extend(&mut self, iter: impl IntoIterator) { + self.inner.extend(iter) + } + + /// Returns the number of elements that can be stored in the `LexRequirement` + /// without reallocating. + pub fn capacity(&self) -> usize { + self.inner.capacity() } - pub fn extend( - &mut self, - requirements: impl IntoIterator, - ) { - self.inner.extend(requirements) + /// Takes ownership of the underlying vector of sort requirements and + /// returns it. + pub fn take(self) -> Vec { + self.inner } - /// Constructs a duplicate-free `LexOrderingReq` by filtering out - /// duplicate entries that have same physical expression inside. + /// Constructs a duplicate-free `LexRequirement` by filtering out duplicate + /// entries that have same physical expression inside. /// /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a /// Some(ASC)]`. diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index a246b38e8d9c4..6626ca4131145 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -213,9 +213,9 @@ impl EquivalenceProperties { pub fn output_ordering(&self) -> Option { // Prune out constant expressions: let constants = self.constants(); - self.oeq_class() - .output_ordering()? - .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)) + let mut sort_exprs = self.oeq_class().output_ordering()?.take(); + sort_exprs.retain(|item| !const_exprs_contains(constants, &item.expr)); + (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs)) } /// Returns the normalized version of the ordering equivalence class within. diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 73523a5044361..d996c359fe730 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -79,8 +79,8 @@ fn calculate_union_binary( // Next, calculate valid orderings for the union by searching for prefixes // in both sides. let mut orderings = UnionEquivalentOrderingBuilder::new(); - orderings.add_satisfied_orderings(lhs.normalized_oeq_class(), lhs.constants(), &rhs); - orderings.add_satisfied_orderings(rhs.normalized_oeq_class(), rhs.constants(), &lhs); + orderings.add_satisfied_orderings(&lhs, &rhs); + orderings.add_satisfied_orderings(&rhs, &lhs); let orderings = orderings.build(); let mut eq_properties = @@ -137,49 +137,51 @@ impl UnionEquivalentOrderingBuilder { Self { orderings: vec![] } } - /// Add all orderings from `orderings` that satisfy `properties`, - /// potentially augmented with`constants`. + /// Add all orderings from `source` that satisfy `properties`, + /// potentially augmented with the constants in `source`. /// - /// Note: any column that is known to be constant can be inserted into the - /// ordering without changing its meaning + /// Note: Any column that is known to be constant can be inserted into the + /// ordering without changing its meaning. /// /// For example: - /// * `orderings` contains `[a ASC, c ASC]` and `constants` contains `b` - /// * `properties` has required ordering `[a ASC, b ASC]` + /// * Orderings in `source` contains `[a ASC, c ASC]` and constants contains + /// `b`, + /// * `properties` has the ordering `[a ASC, b ASC]`. /// /// Then this will add `[a ASC, b ASC]` to the `orderings` list (as `a` was /// in the sort order and `b` was a constant). fn add_satisfied_orderings( &mut self, - orderings: impl IntoIterator, - constants: &[ConstExpr], + source: &EquivalenceProperties, properties: &EquivalenceProperties, ) { - for mut ordering in orderings.into_iter() { + let constants = source.constants(); + for mut ordering in source.normalized_oeq_class() { // Progressively shorten the ordering to search for a satisfied prefix: loop { ordering = match self.try_add_ordering(ordering, constants, properties) { AddedOrdering::Yes => break, AddedOrdering::No(ordering) => { - let (Some(short), _) = ordering.pop() else { + let mut sort_exprs = ordering.take(); + sort_exprs.pop(); + if sort_exprs.is_empty() { break; - }; - short + } + LexOrdering::new(sort_exprs) } } } } } - /// Adds `ordering`, potentially augmented with constants, if it satisfies - /// the target `properties` properties. + /// Adds `ordering`, potentially augmented with `constants`, if it satisfies + /// the given `properties`. /// - /// Returns + /// # Returns /// - /// * [`AddedOrdering::Yes`] if the ordering was added (either directly or - /// augmented), or was empty. - /// - /// * [`AddedOrdering::No`] if the ordering was not added + /// An [`AddedOrdering::Yes`] instance if the ordering was added (either + /// directly or augmented), or was empty. An [`AddedOrdering::No`] instance + /// otherwise. fn try_add_ordering( &mut self, ordering: LexOrdering, @@ -191,79 +193,69 @@ impl UnionEquivalentOrderingBuilder { // augment it with constants. self.orderings.push(ordering); AddedOrdering::Yes + } else if self.try_find_augmented_ordering(&ordering, constants, properties) { + // Augmented with constants to match the properties. + AddedOrdering::Yes } else { - // Did not satisfy target properties, try and augment with constants - // to match the properties - if self.try_find_augmented_ordering(&ordering, constants, properties) { - AddedOrdering::Yes - } else { - AddedOrdering::No(ordering) - } + AddedOrdering::No(ordering) } } /// Attempts to add `constants` to `ordering` to satisfy the properties. - /// - /// returns true if any orderings were added, false otherwise + /// Returns `true` if augmentation took place, `false` otherwise. fn try_find_augmented_ordering( &mut self, ordering: &LexOrdering, constants: &[ConstExpr], properties: &EquivalenceProperties, ) -> bool { - // can't augment if there is nothing to augment with - if constants.is_empty() { - return false; - } - let start_num_orderings = self.orderings.len(); - - // for each equivalent ordering in properties, try and augment - // `ordering` it with the constants to match - for existing_ordering in properties.oeq_class.iter() { - if let Some(augmented_ordering) = self.augment_ordering( - ordering, - constants, - existing_ordering, - &properties.constants, - ) { - assert!(properties.ordering_satisfy(&augmented_ordering)); - self.orderings.push(augmented_ordering); + let mut result = false; + // Can only augment if there are constants. + if !constants.is_empty() { + // For each equivalent ordering in properties, try and augment + // `ordering` with the constants to match `existing_ordering`: + for existing_ordering in properties.oeq_class.iter() { + if let Some(augmented_ordering) = Self::augment_ordering( + ordering, + constants, + existing_ordering, + &properties.constants, + ) { + debug_assert!(properties.ordering_satisfy(&augmented_ordering)); + self.orderings.push(augmented_ordering); + result = true; + } } } - - self.orderings.len() > start_num_orderings + result } - /// Attempts to augment the ordering with constants to match the - /// `existing_ordering` - /// - /// Returns Some(ordering) if an augmented ordering was found, None otherwise + /// Attempts to augment the ordering with constants to match `existing_ordering`. + /// Returns `Some(ordering)` if an augmented ordering was found, `None` otherwise. fn augment_ordering( - &mut self, ordering: &LexOrdering, constants: &[ConstExpr], existing_ordering: &LexOrdering, existing_constants: &[ConstExpr], ) -> Option { let mut augmented_ordering = vec![]; - let mut sort_expr_iter = ordering.iter().peekable(); - let mut existing_sort_expr_iter = existing_ordering.iter().peekable(); - - // walk in parallel down the two orderings, trying to match them up - while sort_expr_iter.peek().is_some() || existing_sort_expr_iter.peek().is_some() - { - // If the next expressions are equal, add the next match - // otherwise try and match with a constant + let mut sort_exprs = ordering.iter().peekable(); + let mut existing_sort_exprs = existing_ordering.iter().peekable(); + + // Walk in parallel down the two orderings, trying to match them up: + while sort_exprs.peek().is_some() || existing_sort_exprs.peek().is_some() { + // If the next expressions are equal, add the next match. Otherwise, + // try and match with a constant. if let Some(expr) = - advance_if_match(&mut sort_expr_iter, &mut existing_sort_expr_iter) + advance_if_match(&mut sort_exprs, &mut existing_sort_exprs) { augmented_ordering.push(expr); } else if let Some(expr) = - advance_if_matches_constant(&mut sort_expr_iter, existing_constants) + advance_if_matches_constant(&mut sort_exprs, existing_constants) { augmented_ordering.push(expr); } else if let Some(expr) = - advance_if_matches_constant(&mut existing_sort_expr_iter, constants) + advance_if_matches_constant(&mut existing_sort_exprs, constants) { augmented_ordering.push(expr); } else { @@ -318,7 +310,6 @@ fn advance_if_matches_constant( #[cfg(test)] mod tests { - use super::*; use crate::equivalence::class::const_exprs_contains; use crate::equivalence::tests::{create_test_schema, parse_sort_expr}; diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 688ed1aacf8ed..46ad84c64a455 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -18,7 +18,6 @@ use std::sync::Arc; use datafusion_physical_expr::LexRequirement; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -40,16 +39,18 @@ pub fn add_sort_above( sort_requirements: LexRequirement, fetch: Option, ) -> PlanContext { - let sort_expr = LexOrdering::from(sort_requirements); - let Some(sort_expr) = sort_expr.retain(|sort_expr| { + let mut sort_reqs = sort_requirements.take(); + sort_reqs.retain(|sort_expr| { !node .plan .equivalence_properties() .is_expr_constant(&sort_expr.expr) - }) else { + }); + if sort_reqs.is_empty() { return node; - }; - let mut new_sort = SortExec::new(sort_expr, Arc::clone(&node.plan)).with_fetch(fetch); + } + let mut new_sort = + SortExec::new(sort_reqs.into(), Arc::clone(&node.plan)).with_fetch(fetch); if node.plan.output_partitioning().partition_count() > 1 { new_sort = new_sort.with_preserve_partitioning(true); } From 851779d334255cd76344204d7878e113ef0a7c75 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 4 Apr 2025 12:33:57 +0300 Subject: [PATCH 056/167] Cleanup LexOrdering APIs 2 --- .../physical-expr-common/src/sort_expr.rs | 19 +------------------ .../src/equivalence/properties/union.rs | 2 +- .../src/enforce_sorting/sort_pushdown.rs | 6 +++--- 3 files changed, 5 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index d9422fbcaf66c..e73c639d4a21a 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -31,8 +31,6 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{exec_err, Result}; use datafusion_expr_common::columnar_value::ColumnarValue; -use itertools::Itertools; - /// Represents Sort operation for a column in a RecordBatch /// /// Example: @@ -76,7 +74,7 @@ use itertools::Itertools; /// .nulls_last(); /// assert_eq!(sort_expr.to_string(), "a DESC NULLS LAST"); /// ``` -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Eq)] pub struct PhysicalSortExpr { /// Physical expression representing the column to sort pub expr: Arc, @@ -130,21 +128,12 @@ impl PhysicalSortExpr { } } -/// Access the PhysicalSortExpr as a PhysicalExpr -impl AsRef for PhysicalSortExpr { - fn as_ref(&self) -> &(dyn PhysicalExpr + 'static) { - self.expr.as_ref() - } -} - impl PartialEq for PhysicalSortExpr { fn eq(&self, other: &Self) -> bool { self.options == other.options && self.expr.eq(&other.expr) } } -impl Eq for PhysicalSortExpr {} - impl Hash for PhysicalSortExpr { fn hash(&self, state: &mut H) { self.expr.hash(state); @@ -377,12 +366,6 @@ impl LexOrdering { Ok(()) } - /// Merge the contents of `other` into `self`, removing duplicates. - pub fn merge(mut self, other: Self) -> Self { - self.inner = self.inner.into_iter().chain(other).unique().collect(); - self - } - /// Constructs a duplicate-free `LexOrdering` by filtering out duplicate /// entries that have same physical expression inside. /// diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index d996c359fe730..0a388e63828d7 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -302,7 +302,7 @@ fn advance_if_matches_constant( constants: &[ConstExpr], ) -> Option { let expr = iter.peek()?; - let const_expr = constants.iter().find(|c| c.eq_expr(expr))?; + let const_expr = constants.iter().find(|c| c.eq_expr(&expr.expr))?; let found_expr = PhysicalSortExpr::new(Arc::clone(const_expr.expr()), expr.options); iter.next(); Some(found_expr) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 895b93a351a92..4c7ae78d91f99 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -364,11 +364,11 @@ fn pushdown_would_violate_requirements( .iter() .any(|child_required| { let Some(child_required) = child_required.as_ref() else { - // no requirements, so pushing down would not violate anything + // No requirements, so pushing down would not violate anything. return false; }; - // check if the plan's requirements would still e satisfied if we pushed - // down the parent requirements + // Check if the plan's requirements would still be satisfied if we + // pushed down the parent requirements: child_required .lex_requirement() .iter() From cebb0d04e125ec2ad666fe2341bee60fa0d5566e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 4 Apr 2025 14:18:37 +0300 Subject: [PATCH 057/167] Add reverse_each to LexOrdering --- .../physical-expr-common/src/sort_expr.rs | 43 +++++++++++++------ datafusion/physical-expr-common/src/utils.rs | 18 ++------ datafusion/physical-expr/src/aggregate.rs | 3 +- datafusion/physical-expr/src/lib.rs | 2 - .../physical-expr/src/window/aggregate.rs | 6 +-- .../src/window/sliding_aggregate.rs | 6 +-- .../physical-expr/src/window/standard.rs | 4 +- .../src/update_aggr_exprs.rs | 7 ++- datafusion/physical-plan/src/windows/mod.rs | 7 +-- 9 files changed, 49 insertions(+), 47 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index e73c639d4a21a..0772057bde80e 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -28,7 +28,7 @@ use crate::physical_expr::{fmt_sql, PhysicalExpr}; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_err, Result}; +use datafusion_common::{plan_err, Result}; use datafusion_expr_common::columnar_value::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch @@ -357,15 +357,6 @@ impl LexOrdering { self.inner } - /// Truncates the `LexOrdering`, keeping only the first `len` elements. - pub fn truncate(&mut self, len: usize) -> Result<()> { - if len == 0 { - return exec_err!("Degenerate LexOrdering instances are not allowed"); - } - self.inner.truncate(len); - Ok(()) - } - /// Constructs a duplicate-free `LexOrdering` by filtering out duplicate /// entries that have same physical expression inside. /// @@ -378,7 +369,27 @@ impl LexOrdering { orderings.push(element); } } - Self::new(orderings) + Self { inner: orderings } + } + + /// Reverses each element in the ordering. For instance, `[a ASC NULLS LAST]` + /// turns into `[a DESC NULLS FIRST]`. Such reversals are useful in planning, + /// e.g. when constructing equivalent window expressions. + pub fn reverse_each(&self) -> Self { + let mut result = self.clone(); + for sort_expr in result.iter_mut() { + sort_expr.options = !sort_expr.options; + } + result + } + + /// Truncates the `LexOrdering`, keeping only the first `len` elements. + pub fn truncate(&mut self, len: usize) -> Result<()> { + if len == 0 { + return plan_err!("Degenerate LexOrdering instances are not allowed"); + } + self.inner.truncate(len); + Ok(()) } /// Transforms each `PhysicalSortExpr` in the `LexOrdering` @@ -554,7 +565,10 @@ impl<'a> IntoIterator for &'a LexRequirement { // Cross-conversion utilities between `LexOrdering` and `LexRequirement` impl From for LexRequirement { fn from(value: LexOrdering) -> Self { - Self::new(value.into_iter().map(Into::into).collect()) + // Can construct directly as `value` is non-degenerate: + Self { + inner: value.into_iter().map(Into::into).collect(), + } } } @@ -566,7 +580,10 @@ impl From> for LexRequirement { impl From for LexOrdering { fn from(value: LexRequirement) -> Self { - value.into_iter().map(Into::into).collect() + // Can construct directly as `value` is non-degenerate: + Self { + inner: value.into_iter().map(Into::into).collect(), + } } } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 114007bfa6afb..05b216ab75ebc 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -17,16 +17,14 @@ use std::sync::Arc; +use crate::physical_expr::PhysicalExpr; +use crate::tree_node::ExprContext; + use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; - use datafusion_common::Result; use datafusion_expr_common::sort_properties::ExprProperties; -use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; -use crate::tree_node::ExprContext; - /// Represents a [`PhysicalExpr`] node with associated properties (order and /// range) in a context where properties are tracked. pub type ExprPropertiesNode = ExprContext; @@ -93,16 +91,6 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { Ok(make_array(data)) } -/// Reverses the ORDER BY expression, which is useful during equivalent window -/// expression construction. For instance, 'ORDER BY a ASC, NULLS LAST' turns into -/// 'ORDER BY a DESC, NULLS FIRST'. -pub fn reverse_order_bys(order_bys: &LexOrdering) -> LexOrdering { - order_bys - .iter() - .map(|e| PhysicalSortExpr::new(Arc::clone(&e.expr), !e.options)) - .collect() -} - #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 489c1c12d0558..9a73a4a536a37 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -53,7 +53,6 @@ use datafusion_functions_aggregate_common::accumulator::{ use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr_common::utils::reverse_order_bys; /// Builder for physical [`AggregateFunctionExpr`] /// @@ -587,7 +586,7 @@ impl AggregateFunctionExpr { replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) - .order_by(self.ordering_req.as_ref().map(reverse_order_bys)) + .order_by(self.ordering_req.as_ref().map(|o| o.reverse_each())) .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 93ced2eb628d8..f0d1d57845b78 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -66,8 +66,6 @@ pub use datafusion_physical_expr_common::sort_expr::{ pub use planner::{create_physical_expr, create_physical_exprs}; pub use scalar_function::ScalarFunctionExpr; - -pub use datafusion_physical_expr_common::utils::reverse_order_bys; pub use utils::split_conjunction; // For backwards compatibility diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 4a0218e6e2a3e..c0a46b2aa87f1 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -27,7 +27,7 @@ use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; +use crate::{EquivalenceProperties, PhysicalExpr}; use arrow::array::Array; use arrow::record_batch::RecordBatch; @@ -156,14 +156,14 @@ impl WindowExpr for PlainAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(reverse_order_bys), + self.order_by.as_ref().map(|o| o.reverse_each()), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(reverse_order_bys), + self.order_by.as_ref().map(|o| o.reverse_each()), Arc::new(self.window_frame.reverse()), )) as _ } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index b775d4094bb7e..c430d3e3fdfc7 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -26,7 +26,7 @@ use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; -use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +use crate::{expressions::PhysicalSortExpr, PhysicalExpr}; use arrow::array::{Array, ArrayRef}; use arrow::datatypes::Field; @@ -123,14 +123,14 @@ impl WindowExpr for SlidingAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(reverse_order_bys), + self.order_by.as_ref().map(|o| o.reverse_each()), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(reverse_order_bys), + self.order_by.as_ref().map(|o| o.reverse_each()), Arc::new(self.window_frame.reverse()), )) as _ } diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 1430ff676644f..94c91619c994b 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use super::{StandardWindowFunctionExpr, WindowExpr}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; -use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; +use crate::{EquivalenceProperties, PhysicalExpr}; use arrow::array::{new_empty_array, ArrayRef}; use arrow::datatypes::Field; @@ -260,7 +260,7 @@ impl WindowExpr for StandardWindowExpr { Arc::new(StandardWindowExpr::new( reverse_expr, &self.partition_by.clone(), - self.order_by.as_ref().map(reverse_order_bys), + self.order_by.as_ref().map(|o| o.reverse_each()), Arc::new(self.window_frame.reverse()), )) as _ }) diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 9209c2121a6ce..b99efb8c08f79 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -24,9 +24,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; -use datafusion_physical_expr::{ - reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; use datafusion_physical_plan::{ @@ -178,7 +176,8 @@ fn try_convert_aggregate_if_better( aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, - &reverse_order_bys(order_bys) + &order_bys + .reverse_each() .into_iter() .map(Into::into) .collect::>(), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ae17c9709147e..0277358bb4ab7 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -43,9 +43,10 @@ use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::window::{ + SlidingAggregateWindowExpr, StandardWindowFunctionExpr, +}; use datafusion_physical_expr::{ - reverse_order_bys, - window::{SlidingAggregateWindowExpr, StandardWindowFunctionExpr}, ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; @@ -604,7 +605,7 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); - let reverse_orderby_keys = orderby_keys.map(reverse_order_bys); + let reverse_orderby_keys = orderby_keys.map(|o| o.reverse_each()); for (should_swap, orderbys) in [(false, orderby_keys), (true, reverse_orderby_keys.as_ref())] { From 44e6b953a239867548efddc06cadd4ed7ac15614 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 4 Apr 2025 14:27:21 +0300 Subject: [PATCH 058/167] Use LexOrdering instead of Arc<[PhysicalSortExpr]> --- datafusion/physical-expr-common/src/sort_expr.rs | 7 ------- datafusion/physical-plan/src/joins/utils.rs | 3 +-- datafusion/physical-plan/src/sorts/sort.rs | 4 ++-- datafusion/physical-plan/src/topk/mod.rs | 15 +++++---------- 4 files changed, 8 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 0772057bde80e..2afc3ec1b014b 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -405,13 +405,6 @@ impl From> for LexOrdering { } } -/// Convert a `LexOrdering` into a `Arc[]` for fast copies -impl From for Arc<[PhysicalSortExpr]> { - fn from(value: LexOrdering) -> Self { - value.inner.into() - } -} - impl Deref for LexOrdering { type Target = [PhysicalSortExpr]; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 6a563186d06f9..7d8d8e7d7f255 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -139,7 +139,7 @@ fn replace_on_columns_of_right_ordering( ) { for (left_col, right_col) in on_columns { right_ordering.transform(|item| { - let new_expr = Arc::clone(&item.expr) + item.expr = Arc::clone(&item.expr) .transform(|e| { if e.eq(right_col) { Ok(Transformed::yes(Arc::clone(left_col))) @@ -149,7 +149,6 @@ fn replace_on_columns_of_right_ordering( }) .data() .expect("closure is infallible"); - item.expr = new_expr; }); } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 1eea86aa7f01a..a40e988874d52 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -204,7 +204,7 @@ struct ExternalSorter { /// Schema of the output (and the input) schema: SchemaRef, /// Sort expressions - expr: Arc<[PhysicalSortExpr]>, + expr: LexOrdering, /// RowConverter corresponding to the sort expressions sort_keys_row_converter: Arc, /// The target number of rows for output batches @@ -301,7 +301,7 @@ impl ExternalSorter { in_mem_batches: vec![], in_progress_spill_file: None, finished_spill_files: vec![], - expr: expr.into(), + expr, sort_keys_row_converter: Arc::new(converter), metrics, reservation, diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 85de1eefce2e4..907049033eacc 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -17,25 +17,22 @@ //! TopK: Combination of Sort / LIMIT -use arrow::{ - compute::interleave, - row::{RowConverter, Rows, SortField}, -}; use std::mem::size_of; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; use crate::spill::get_record_batch_memory_size; use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; + use arrow::array::{Array, ArrayRef, RecordBatch}; +use arrow::compute::interleave; use arrow::datatypes::SchemaRef; -use datafusion_common::HashMap; -use datafusion_common::Result; +use arrow::row::{RowConverter, Rows, SortField}; +use datafusion_common::{HashMap, Result}; use datafusion_execution::{ memory_pool::{MemoryConsumer, MemoryReservation}, runtime_env::RuntimeEnv, }; -use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Global TopK @@ -83,7 +80,7 @@ pub struct TopK { /// The target number of rows for output batches batch_size: usize, /// sort expressions - expr: Arc<[PhysicalSortExpr]>, + expr: LexOrdering, /// row converter, for sort keys row_converter: RowConverter, /// scratch space for converting rows @@ -108,8 +105,6 @@ impl TopK { let reservation = MemoryConsumer::new(format!("TopK[{partition_id}]")) .register(&runtime.memory_pool); - let expr: Arc<[PhysicalSortExpr]> = expr.into(); - let sort_fields: Vec<_> = expr .iter() .map(|e| { From 804bec5e2811e42b7754b0226400e6fe9af0aa5b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 4 Apr 2025 17:29:00 +0300 Subject: [PATCH 059/167] Use PhysicalSortExpr slices in contexts where we simply list sort expressions --- datafusion/core/src/physical_planner.rs | 2 +- .../core/tests/fuzz_cases/window_fuzz.rs | 7 +- .../physical_optimizer/enforce_sorting.rs | 2 +- .../tests/physical_optimizer/test_utils.rs | 2 +- .../physical-expr-common/src/sort_expr.rs | 23 +++- .../physical-expr/src/equivalence/ordering.rs | 7 +- .../src/equivalence/properties/mod.rs | 127 ++++++++---------- .../physical-expr/src/window/aggregate.rs | 24 ++-- .../src/window/sliding_aggregate.rs | 49 +++---- .../physical-expr/src/window/standard.rs | 28 ++-- .../physical-expr/src/window/window_expr.rs | 38 ++---- .../src/windows/bounded_window_agg_exec.rs | 21 ++- datafusion/physical-plan/src/windows/mod.rs | 24 ++-- .../src/windows/window_agg_exec.rs | 10 +- .../proto/src/physical_plan/from_proto.rs | 2 +- .../proto/src/physical_plan/to_proto.rs | 5 +- .../tests/cases/roundtrip_physical_plan.rs | 22 +-- 17 files changed, 181 insertions(+), 212 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 3b0a950ec7968..5f5ff115460f9 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1545,7 +1545,7 @@ pub fn create_window_expr_with_name( name, &physical_args, &partition_by, - (!order_by.is_empty()).then(|| order_by.into()), + &order_by, window_frame, physical_schema, ignore_nulls, diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 897fd964be06a..7765185edeb29 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -284,7 +284,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { fn_name.to_string(), &args, &partitionby_exprs, - None, + &[], Arc::new(window_frame), &extended_schema, false, @@ -647,14 +647,13 @@ async fn run_window_test( let extended_schema = schema_add_window_field(&args, &schema, &window_fn, &fn_name)?; - let orderby = (!orderby_exprs.is_empty()).then(|| orderby_exprs.into()); let usual_window_exec = Arc::new(WindowAggExec::try_new( vec![create_window_expr( &window_fn, fn_name.clone(), &args, &partitionby_exprs, - orderby.clone(), + &orderby_exprs.clone(), Arc::new(window_frame.clone()), &extended_schema, false, @@ -672,7 +671,7 @@ async fn run_window_test( fn_name, &args, &partitionby_exprs, - orderby, + &orderby_exprs, Arc::new(window_frame.clone()), &extended_schema, false, diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 8247336d29459..ae49ad71a3970 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -3702,7 +3702,7 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { case.func.1, &case.func.2, &partition_by, - None, + &[], case.window_frame, input_schema.as_ref(), false, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 0e2d50e8bf06f..87839d5ce224b 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -216,7 +216,7 @@ pub fn bounded_window_exec_with_partition( "count".to_owned(), &[col(col_name, &schema).unwrap()], partition_by, - (!sort_exprs.is_empty()).then(|| sort_exprs.into()), + &sort_exprs, Arc::new(WindowFrame::new(Some(false))), schema.as_ref(), false, diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 2afc3ec1b014b..b2f13ff991a1e 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -28,7 +28,7 @@ use crate::physical_expr::{fmt_sql, PhysicalExpr}; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::{plan_err, Result}; +use datafusion_common::Result; use datafusion_expr_common::columnar_value::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch @@ -93,6 +93,15 @@ impl PhysicalSortExpr { Self::new(expr, SortOptions::default()) } + /// Reverses the sort expression. For instance, `[a ASC NULLS LAST]` turns + /// into `[a DESC NULLS FIRST]`. Such reversals are useful in planning, e.g. + /// when constructing equivalent window expressions. + pub fn reverse(&self) -> Self { + let mut result = self.clone(); + result.options = !result.options; + result + } + /// Set the sort sort options to ASC pub fn asc(mut self) -> Self { self.options.descending = false; @@ -384,12 +393,16 @@ impl LexOrdering { } /// Truncates the `LexOrdering`, keeping only the first `len` elements. - pub fn truncate(&mut self, len: usize) -> Result<()> { - if len == 0 { - return plan_err!("Degenerate LexOrdering instances are not allowed"); + /// Returns `true` if truncation made a change, `false` otherwise. Negative + /// cases happen in two scenarios: (1) When `len` is greater than or equal + /// to the number of expressions inside this `LexOrdering`, making truncation + /// a no-op, or (2) when `len` is `0`, making truncation impossible. + pub fn truncate(&mut self, len: usize) -> bool { + if len == 0 || len >= self.inner.len() { + return false; } self.inner.truncate(len); - Ok(()) + true } /// Transforms each `PhysicalSortExpr` in the `LexOrdering` diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 5104f55516b36..2d64fe1cc4b17 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -170,12 +170,7 @@ impl OrderingEquivalenceClass { if self.orderings[idx][length - overlap..] == self.orderings[pre_idx][..overlap] { - return if length == overlap { - Some(true) - } else { - self.orderings[idx].truncate(length - overlap).unwrap(); - Some(false) - }; + return Some(!self.orderings[idx].truncate(length - overlap)); } } None diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 6626ca4131145..18d1aa3c6677f 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -662,6 +662,11 @@ impl EquivalenceProperties { true } + /// Checks if the sort expressions are satisfied by any of the table + /// constraints (primary key or unique). Returns true if any constraint + /// fully satisfies the expressions (i.e. constraint indices form a valid + /// prefix of an existing ordering that matches the expressions). For + /// unique constraints, also verifies nullable columns. fn satisfied_by_constraints_ordering( &self, normalized_exprs: &[PhysicalSortExpr], @@ -700,90 +705,68 @@ impl EquivalenceProperties { // Check if this ordering matches the prefix: let ordering_len = ordering.len(); normalized_exprs.len() >= ordering_len - && normalized_exprs[..ordering_len].iter().zip(ordering).all( - |(req, existing)| { - req.expr.eq(&existing.expr) - && req.options == existing.options - }, - ) + && normalized_exprs[..ordering_len] + .iter() + .zip(ordering) + .all(|(req, existing)| req == existing) }) } }) } - /// Checks if the sort requirements are satisfied by any of the table constraints (primary key or unique). - /// Returns true if any constraint fully satisfies the requirements. + /// Checks if the sort requirements are satisfied by any of the table + /// constraints (primary key or unique). Returns true if any constraint + /// fully satisfies the requirements (i.e. constraint indices form a valid + /// prefix of an existing ordering that matches the requirements). For + /// unique constraints, also verifies nullable columns. fn satisfied_by_constraints( &self, normalized_reqs: &[PhysicalSortRequirement], ) -> bool { self.constraints.iter().any(|constraint| match constraint { - Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => self - .satisfied_by_constraint( - normalized_reqs, - indices, - matches!(constraint, Constraint::Unique(_)), - ), - }) - } - - /// Checks if sort requirements are satisfied by a constraint (primary key or unique). - /// Returns true if the constraint indices form a valid prefix of an existing ordering - /// that matches the requirements. For unique constraints, also verifies nullable columns. - fn satisfied_by_constraint( - &self, - normalized_reqs: &[PhysicalSortRequirement], - indices: &[usize], - check_null: bool, - ) -> bool { - // Requirements must contain indices - if indices.len() > normalized_reqs.len() { - return false; - } - - // Iterate over all orderings - self.oeq_class.iter().any(|ordering| { - if indices.len() > ordering.len() { - return false; - } - - // Build a map of column positions in the requirement - let mut col_positions = HashMap::with_capacity(ordering.len()); - for (pos, req) in ordering.iter().enumerate() { - if let Some(col) = req.expr.as_any().downcast_ref::() { - col_positions.insert( - col.index(), - (pos, col.nullable(&self.schema).unwrap_or(true)), - ); - } - } - - // Check if all constraint indices appear in valid positions - if !indices.iter().all(|&idx| { - col_positions - .get(&idx) - .map(|&(pos, nullable)| { - // For unique constraints, verify column is not nullable if it's first/last - !check_null - || (pos != 0 && pos != ordering.len() - 1) - || !nullable + Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { + let check_null = matches!(constraint, Constraint::Unique(_)); + indices.len() <= normalized_reqs.len() + && self.oeq_class.iter().any(|ordering| { + if indices.len() > ordering.len() { + return false; + } + // Build a map of column positions in the ordering: + let mut col_positions = HashMap::with_capacity(ordering.len()); + for (pos, req) in ordering.iter().enumerate() { + if let Some(col) = req.expr.as_any().downcast_ref::() + { + let nullable = col.nullable(&self.schema).unwrap_or(true); + col_positions.insert(col.index(), (pos, nullable)); + } + } + // Check if all constraint indices appear in valid positions: + if !indices.iter().all(|&idx| { + col_positions + .get(&idx) + .map(|&(pos, nullable)| { + // For unique constraints, verify column is not nullable if it's first/last: + !check_null + || !nullable + || (pos != 0 && pos != ordering.len() - 1) + }) + .unwrap_or(false) + }) { + return false; + } + // Check if this ordering matches the prefix: + let ordering_len = ordering.len(); + normalized_reqs.len() >= ordering_len + && normalized_reqs[..ordering_len].iter().zip(ordering).all( + |(req, existing)| { + req.expr.eq(&existing.expr) + && req.options.is_none_or(|req_opts| { + req_opts == existing.options + }) + }, + ) }) - .unwrap_or(false) - }) { - return false; } - - // Check if this ordering matches the prefix - let ordering_len = ordering.len(); - normalized_reqs.len() >= ordering_len - && normalized_reqs[..ordering_len].iter().zip(ordering).all( - |(req, existing)| { - req.expr.eq(&existing.expr) - && req - .options - .is_none_or(|req_opts| req_opts == existing.options) - }, - ) }) } diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index c0a46b2aa87f1..07d392666f128 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -34,7 +34,7 @@ use arrow::record_batch::RecordBatch; use arrow::{array::ArrayRef, datatypes::Field}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// A window expr that takes the form of an aggregate function. /// @@ -43,7 +43,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; pub struct PlainAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Option, + order_by: Vec, window_frame: Arc, } @@ -52,13 +52,13 @@ impl PlainAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: Option, + order_by: &[PhysicalSortExpr], window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by, + order_by: order_by.to_vec(), window_frame, } } @@ -141,8 +141,8 @@ impl WindowExpr for PlainAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> Option<&LexOrdering> { - self.order_by.as_ref() + fn order_by(&self) -> &[PhysicalSortExpr] { + &self.order_by } fn get_window_frame(&self) -> &Arc { @@ -156,14 +156,22 @@ impl WindowExpr for PlainAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(|o| o.reverse_each()), + &self + .order_by + .iter() + .map(|e| e.reverse()) + .collect::>(), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(|o| o.reverse_each()), + &self + .order_by + .iter() + .map(|e| e.reverse()) + .collect::>(), Arc::new(self.window_frame.reverse()), )) as _ } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index c430d3e3fdfc7..2a95449286a40 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -33,7 +33,6 @@ use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A window expr that takes the form of an aggregate function that /// can be incrementally computed over sliding windows. @@ -43,7 +42,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; pub struct SlidingAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Option, + order_by: Vec, window_frame: Arc, } @@ -52,13 +51,13 @@ impl SlidingAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: Option, + order_by: &[PhysicalSortExpr], window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.clone(), + order_by: order_by.to_vec(), window_frame, } } @@ -108,8 +107,8 @@ impl WindowExpr for SlidingAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> Option<&LexOrdering> { - self.order_by.as_ref() + fn order_by(&self) -> &[PhysicalSortExpr] { + &self.order_by } fn get_window_frame(&self) -> &Arc { @@ -123,14 +122,22 @@ impl WindowExpr for SlidingAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(|o| o.reverse_each()), + &self + .order_by + .iter() + .map(|e| e.reverse()) + .collect::>(), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - self.order_by.as_ref().map(|o| o.reverse_each()), + &self + .order_by + .iter() + .map(|e| e.reverse()) + .collect::>(), Arc::new(self.window_frame.reverse()), )) as _ } @@ -147,21 +154,17 @@ impl WindowExpr for SlidingAggregateWindowExpr { partition_bys: Vec>, order_by_exprs: Vec>, ) -> Option> { - debug_assert!(self.order_by.as_ref().map_or_else( - || order_by_exprs.is_empty(), - |old_exprs| old_exprs.len() == order_by_exprs.len() - )); - - let new_order_by = self.order_by().map(|ob_exprs| { - ob_exprs - .iter() - .zip(order_by_exprs) - .map(|(sort_expr, new_expr)| PhysicalSortExpr { - expr: new_expr, - options: sort_expr.options, - }) - .collect::() - }); + debug_assert_eq!(self.order_by.len(), order_by_exprs.len()); + + let new_order_by: Vec = self + .order_by + .iter() + .zip(order_by_exprs) + .map(|(req, new_expr)| PhysicalSortExpr { + expr: new_expr, + options: req.options, + }) + .collect(); Some(Arc::new(SlidingAggregateWindowExpr { aggregate: self .aggregate diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 94c91619c994b..5ef22c18f4586 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -40,7 +40,7 @@ use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; pub struct StandardWindowExpr { expr: Arc, partition_by: Vec>, - order_by: Option, + order_by: Vec, window_frame: Arc, } @@ -49,13 +49,13 @@ impl StandardWindowExpr { pub fn new( expr: Arc, partition_by: &[Arc], - order_by: Option, + order_by: &[PhysicalSortExpr], window_frame: Arc, ) -> Self { Self { expr, partition_by: partition_by.to_vec(), - order_by, + order_by: order_by.to_vec(), window_frame, } } @@ -104,19 +104,15 @@ impl WindowExpr for StandardWindowExpr { &self.partition_by } - fn order_by(&self) -> Option<&LexOrdering> { - self.order_by.as_ref() + fn order_by(&self) -> &[PhysicalSortExpr] { + &self.order_by } fn evaluate(&self, batch: &RecordBatch) -> Result { let mut evaluator = self.expr.create_evaluator()?; let num_rows = batch.num_rows(); if evaluator.uses_window_frame() { - let sort_options = if let Some(ob) = &self.order_by { - ob.iter().map(|o| o.options).collect() - } else { - vec![] - }; + let sort_options = self.order_by.iter().map(|o| o.options).collect(); let mut row_wise_results = vec![]; let mut values = self.evaluate_args(batch)?; @@ -160,11 +156,7 @@ impl WindowExpr for StandardWindowExpr { ) -> Result<()> { let field = self.expr.field()?; let out_type = field.data_type(); - let sort_options = if let Some(ob) = &self.order_by { - ob.iter().map(|o| o.options).collect() - } else { - vec![] - }; + let sort_options = self.order_by.iter().map(|o| o.options).collect::>(); for (partition_row, partition_batch_state) in partition_batches.iter() { let window_state = if let Some(window_state) = window_agg_state.get_mut(partition_row) { @@ -260,7 +252,11 @@ impl WindowExpr for StandardWindowExpr { Arc::new(StandardWindowExpr::new( reverse_expr, &self.partition_by.clone(), - self.order_by.as_ref().map(|o| o.reverse_each()), + &self + .order_by + .iter() + .map(|e| e.reverse()) + .collect::>(), Arc::new(self.window_frame.reverse()), )) as _ }) diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index a92dea6ee7297..b4864c01f27ef 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; -use crate::{LexOrdering, PhysicalExpr}; +use crate::PhysicalExpr; use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; @@ -33,6 +33,7 @@ use datafusion_expr::window_state::{ PartitionBatchState, WindowAggState, WindowFrameContext, WindowFrameStateGroups, }; use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame, WindowFrameBound}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use indexmap::IndexMap; @@ -109,14 +110,11 @@ pub trait WindowExpr: Send + Sync + Debug { fn partition_by(&self) -> &[Arc]; /// Expressions that's from the window function's order by clause, empty if absent - fn order_by(&self) -> Option<&LexOrdering>; + fn order_by(&self) -> &[PhysicalSortExpr]; /// Get order by columns, empty if absent fn order_by_columns(&self, batch: &RecordBatch) -> Result> { - let Some(order_bys) = self.order_by() else { - return Ok(vec![]); - }; - order_bys + self.order_by() .iter() .map(|e| e.evaluate_to_sort_column(batch)) .collect() @@ -137,13 +135,11 @@ pub trait WindowExpr: Send + Sync + Debug { fn all_expressions(&self) -> WindowPhysicalExpressions { let args = self.expressions(); let partition_by_exprs = self.partition_by().to_vec(); - let order_by_exprs = if let Some(ob) = self.order_by() { - ob.iter() - .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect() - } else { - vec![] - }; + let order_by_exprs = self + .order_by() + .iter() + .map(|sort_expr| Arc::clone(&sort_expr.expr)) + .collect(); WindowPhysicalExpressions { args, partition_by_exprs, @@ -195,11 +191,7 @@ pub trait AggregateWindowExpr: WindowExpr { fn aggregate_evaluate(&self, batch: &RecordBatch) -> Result { let mut accumulator = self.get_accumulator()?; let mut last_range = Range { start: 0, end: 0 }; - let sort_options = if let Some(ob) = self.order_by() { - ob.iter().map(|sort_expr| sort_expr.options).collect() - } else { - vec![] - }; + let sort_options = self.order_by().iter().map(|o| o.options).collect(); let mut window_frame_ctx = WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options); self.get_result_column( @@ -247,11 +239,7 @@ pub trait AggregateWindowExpr: WindowExpr { // If there is no window state context, initialize it. let window_frame_ctx = state.window_frame_ctx.get_or_insert_with(|| { - let sort_options = if let Some(ob) = self.order_by() { - ob.iter().map(|sort_expr| sort_expr.options).collect() - } else { - vec![] - }; + let sort_options = self.order_by().iter().map(|o| o.options).collect(); WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options) }); let out_col = self.get_result_column( @@ -355,10 +343,10 @@ pub(crate) fn is_end_bound_safe( window_frame_ctx: &WindowFrameContext, order_bys: &[ArrayRef], most_recent_order_bys: Option<&[ArrayRef]>, - sort_exprs: Option<&LexOrdering>, + sort_exprs: &[PhysicalSortExpr], idx: usize, ) -> Result { - let Some(sort_exprs) = sort_exprs else { + if sort_exprs.is_empty() { // Early return if no sort expressions are present: return Ok(false); }; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index c8eea9cbfd5b8..1461a937a2b38 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -294,10 +294,7 @@ impl ExecutionPlan for BoundedWindowAggExec { .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements( - partition_bys, - order_keys.map_or_else(Vec::new, |o| o.to_vec()), - )] + vec![calc_requirements(partition_bys, order_keys)] } fn required_input_distribution(&self) -> Vec { @@ -1343,10 +1340,10 @@ mod tests { Arc::new(Column::new(schema.fields[0].name(), 0)) as Arc; let args = vec![col_expr]; let partitionby_exprs = vec![col(hash, &schema)?]; - let orderby_exprs = LexOrdering::new(vec![PhysicalSortExpr { + let orderby_exprs = vec![PhysicalSortExpr { expr: col(order_by, &schema)?, options: SortOptions::default(), - }]); + }]; let window_frame = WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::CurrentRow, @@ -1363,7 +1360,7 @@ mod tests { fn_name, &args, &partitionby_exprs, - Some(orderby_exprs), + &orderby_exprs, Arc::new(window_frame), &input.schema(), false, @@ -1617,7 +1614,7 @@ mod tests { Arc::new(StandardWindowExpr::new( last_value_func, &[], - None, + &[], Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1628,7 +1625,7 @@ mod tests { Arc::new(StandardWindowExpr::new( nth_value_func1, &[], - None, + &[], Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1639,7 +1636,7 @@ mod tests { Arc::new(StandardWindowExpr::new( nth_value_func2, &[], - None, + &[], Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1780,8 +1777,8 @@ mod tests { let plan = projection_exec(window)?; let expected_plan = vec![ - "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]@2 as col_2]", - " BoundedWindowAggExec: wdw=[count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]: Ok(Field { name: \"count([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", + "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]@2 as col_2]", + " BoundedWindowAggExec: wdw=[count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]: Ok(Field { name: \"count([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", " StreamingTableExec: partition_sizes=1, projection=[sn, hash], infinite_source=true, output_ordering=[sn@0 ASC NULLS LAST]", ]; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0277358bb4ab7..1c907a69d998b 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -101,7 +101,7 @@ pub fn create_window_expr( name: String, args: &[Arc], partition_by: &[Arc], - order_by: Option, + order_by: &[PhysicalSortExpr], window_frame: Arc, input_schema: &Schema, ignore_nulls: bool, @@ -133,7 +133,7 @@ pub fn create_window_expr( /// Creates an appropriate [`WindowExpr`] based on the window frame and fn window_expr_from_aggregate_expr( partition_by: &[Arc], - order_by: Option, + order_by: &[PhysicalSortExpr], window_frame: Arc, aggregate: Arc, ) -> Arc { @@ -590,7 +590,7 @@ pub fn get_best_fitting_window( /// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], - orderby_keys: Option<&LexOrdering>, + orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); @@ -605,14 +605,13 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); - let reverse_orderby_keys = orderby_keys.map(|o| o.reverse_each()); + let reverse_orderby_keys = + orderby_keys.iter().map(|e| e.reverse()).collect::>(); for (should_swap, orderbys) in [(false, orderby_keys), (true, reverse_orderby_keys.as_ref())] { let mut req = partition_by_reqs.clone(); - if let Some(ob) = orderbys { - req.extend(ob.iter().cloned().map(Into::into)); - } + req.extend(orderbys.iter().cloned().map(Into::into)); if req.is_empty() || partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { @@ -814,7 +813,7 @@ mod tests { "count".to_owned(), &[col("a", &schema)?], &[], - None, + &[], Arc::new(WindowFrame::new(None)), schema.as_ref(), false, @@ -1019,11 +1018,8 @@ mod tests { let options = SortOptions::default(); order_by_exprs.push(PhysicalSortExpr { expr, options }); } - let res = get_window_mode( - &partition_by_exprs, - Some(&order_by_exprs.into()), - &exec_unbounded, - ); + let res = + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded); // Since reversibility is not important in this test. Convert Option<(bool, InputOrderMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -1187,7 +1183,7 @@ mod tests { } assert_eq!( - get_window_mode(&partition_by_exprs, Some(&order_by_exprs.into()), &exec_unbounded), + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded), *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 14b7ad543b85a..c2765319483a9 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -220,19 +220,13 @@ impl ExecutionPlan for WindowAggExec { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { - vec![calc_requirements( - partition_bys, - order_keys.map_or_else(Vec::new, |o| o.to_vec()), - )] + vec![calc_requirements(partition_bys, order_keys)] } else { let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements( - partition_bys, - order_keys.map_or_else(Vec::new, |o| o.to_vec()), - )] + vec![calc_requirements(partition_bys, order_keys)] } } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index aa8401730b965..aad988d88bc0a 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -174,7 +174,7 @@ pub fn parse_physical_window_expr( name, &window_node_expr, &partition_by, - (!order_by.is_empty()).then(|| order_by.into()), + &order_by, Arc::new(window_frame), &extended_schema, false, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 27e230128ee0c..52677ca8153b7 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -146,10 +146,7 @@ pub fn serialize_physical_window_expr( let args = serialize_physical_exprs(&args, codec)?; let partition_by = serialize_physical_exprs(window_expr.partition_by(), codec)?; - let order_by = serialize_physical_sort_exprs( - window_expr.order_by().map_or_else(Vec::new, |o| o.to_vec()), - codec, - )?; + let order_by = serialize_physical_sort_exprs(window_expr.order_by().to_vec(), codec)?; let window_frame: protobuf::WindowFrame = window_frame .as_ref() .try_into() diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index fa47acc10f003..156d06d7aef69 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -320,9 +320,9 @@ fn roundtrip_udwf() -> Result<()> { &[ col("a", &schema)? ], - Some(LexOrdering::new(vec![ - PhysicalSortExpr::new(col("b", &schema)?, SortOptions::new(true, true)), - ])), + &[ + PhysicalSortExpr::new(col("b", &schema)?, SortOptions::new(true, true)) + ], Arc::new(WindowFrame::new(None)), )); @@ -359,13 +359,13 @@ fn roundtrip_window() -> Result<()> { let udwf_expr = Arc::new(StandardWindowExpr::new( nth_value_window, &[col("b", &schema)?], - Some(LexOrdering::new(vec![PhysicalSortExpr { + &[PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }])), + }], Arc::new(window_frame), )); @@ -379,7 +379,7 @@ fn roundtrip_window() -> Result<()> { .build() .map(Arc::new)?, &[], - None, + &[], Arc::new(WindowFrame::new(None)), )); @@ -399,7 +399,7 @@ fn roundtrip_window() -> Result<()> { let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( sum_expr, &[], - None, + &[], Arc::new(window_frame), )); @@ -1118,7 +1118,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), &[col("author", &schema)?], - None, + &[], Arc::new(WindowFrame::new(None)), ))], filter, @@ -1163,13 +1163,13 @@ fn roundtrip_udwf_extension_codec() -> Result<()> { let udwf_expr = Arc::new(StandardWindowExpr::new( udwf, &[col("b", &schema)?], - Some(LexOrdering::new(vec![PhysicalSortExpr { + &[PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }])), + }], Arc::new(window_frame), )); @@ -1226,7 +1226,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr, &[col("author", &schema)?], - None, + &[], Arc::new(WindowFrame::new(None)), ))], filter, From b2c2c75f9ab5c5559b0c8d22a19c7a2de127bc5b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 5 Apr 2025 11:22:17 +0300 Subject: [PATCH 060/167] Generalize add_new_ordering APIs --- .../tests/fuzz_cases/equivalence/utils.rs | 18 +-- .../sort_preserving_repartition_fuzz.rs | 18 +-- .../physical-expr/src/equivalence/ordering.rs | 23 ++-- .../src/equivalence/properties/dependency.rs | 104 ++++++++---------- .../src/equivalence/properties/mod.rs | 13 ++- .../src/window/sliding_aggregate.rs | 2 +- .../physical-expr/src/window/standard.rs | 6 +- datafusion/physical-plan/src/sorts/sort.rs | 6 +- .../src/sorts/sort_preserving_merge.rs | 11 +- datafusion/physical-plan/src/windows/mod.rs | 26 ++--- 10 files changed, 109 insertions(+), 118 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 423fe9e8a9ecf..037e30e2d1ce4 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -117,15 +117,15 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti let n_sort_expr = rng.gen_range(1..remaining_exprs.len() + 1); remaining_exprs.shuffle(&mut rng); - let ordering = remaining_exprs - .drain(0..n_sort_expr) - .map(|expr| PhysicalSortExpr { - expr: Arc::clone(expr), - options: options_asc, - }) - .collect(); - - eq_properties.add_new_orderings([ordering]); + let ordering = + remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: Arc::clone(expr), + options: options_asc, + }); + + eq_properties.add_new_ordering(ordering); } Ok((test_schema, eq_properties)) diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 647f498ba4803..3f8a5f03a28f5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -94,15 +94,15 @@ mod sp_repartition_fuzz_tests { let n_sort_expr = rng.gen_range(1..remaining_exprs.len() + 1); remaining_exprs.shuffle(&mut rng); - let ordering = remaining_exprs - .drain(0..n_sort_expr) - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: options_asc, - }) - .collect(); - - eq_properties.add_new_orderings([ordering]); + let ordering = + remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: options_asc, + }); + + eq_properties.add_new_ordering(ordering); } Ok((test_schema, eq_properties)) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 2d64fe1cc4b17..8bddc2381ac02 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -25,6 +25,7 @@ use crate::{LexOrdering, PhysicalExpr}; use arrow::compute::SortOptions; use datafusion_common::HashSet; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// An `OrderingEquivalenceClass` object keeps track of different alternative /// orderings than can describe a schema. For example, consider the following table: @@ -74,16 +75,6 @@ impl OrderingEquivalenceClass { self.orderings.contains(ordering) } - /// Adds `ordering` to this equivalence class. - #[allow(dead_code)] - #[deprecated( - since = "45.0.0", - note = "use OrderingEquivalenceClass::add_new_ordering instead" - )] - fn push(&mut self, ordering: LexOrdering) { - self.add_new_ordering(ordering) - } - /// Checks whether this ordering equivalence class is empty. pub fn is_empty(&self) -> bool { self.len() == 0 @@ -112,15 +103,21 @@ impl OrderingEquivalenceClass { /// Adds new orderings into this ordering equivalence class pub fn add_new_orderings( &mut self, - orderings: impl IntoIterator, + orderings: impl IntoIterator>, ) { - self.orderings.extend(orderings); + self.orderings.extend(orderings.into_iter().filter_map(|o| { + let sort_exprs = o.into_iter().collect::>(); + (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs)) + })); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); } /// Adds a single ordering to the existing ordering equivalence class. - pub fn add_new_ordering(&mut self, ordering: LexOrdering) { + pub fn add_new_ordering( + &mut self, + ordering: impl IntoIterator, + ) { self.add_new_orderings([ordering]); } diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 4b8b72fd1e6f8..51f3c682e4e0a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -508,20 +508,20 @@ mod tests { let mut input_properties = EquivalenceProperties::new(Arc::clone(&input_schema)); // add equivalent ordering [a, b, c, d] - input_properties.add_new_ordering(LexOrdering::new(vec![ + input_properties.add_new_ordering([ parse_sort_expr("a", &input_schema), parse_sort_expr("b", &input_schema), parse_sort_expr("c", &input_schema), parse_sort_expr("d", &input_schema), - ])); + ]); // add equivalent ordering [a, c, b, d] - input_properties.add_new_ordering(LexOrdering::new(vec![ + input_properties.add_new_ordering([ parse_sort_expr("a", &input_schema), parse_sort_expr("c", &input_schema), parse_sort_expr("b", &input_schema), // NB b and c are swapped parse_sort_expr("d", &input_schema), - ])); + ]); // simply project all the columns in order let proj_exprs = vec![ @@ -556,28 +556,27 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr)?; - let others = vec![ - LexOrdering::new(vec![PhysicalSortExpr { + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }]), - LexOrdering::new(vec![PhysicalSortExpr { + }], + vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), options: sort_options, - }]), - ]; - eq_properties.add_new_orderings(others); + }], + ]); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_new_orderings([ - LexOrdering::new(vec![PhysicalSortExpr { + vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }]), - LexOrdering::new(vec![PhysicalSortExpr { + }], + vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), options: sort_options, - }]), + }], ]); let oeq_class = eq_properties.oeq_class().clone(); @@ -600,7 +599,7 @@ mod tests { let col_b = &col("b", &schema)?; let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -609,7 +608,7 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ])]); + ]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0, 1]); assert_eq!( @@ -636,11 +635,11 @@ mod tests { let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_new_orderings([ - LexOrdering::new(vec![PhysicalSortExpr { + vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, - }]), - LexOrdering::new(vec![ + }], + vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -649,7 +648,7 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]), + ], ]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0, 1]); @@ -679,7 +678,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); // not satisfied orders - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -692,7 +691,7 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ])]); + ]); let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0]); @@ -720,15 +719,15 @@ mod tests { // b=a (e.g they are aliases) eq_properties.add_equal_conditions(col_b, col_a)?; // [b ASC], [d ASC] - eq_properties.add_new_orderings(vec![ - LexOrdering::new(vec![PhysicalSortExpr { + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { expr: Arc::clone(col_b), options: option_asc, - }]), - LexOrdering::new(vec![PhysicalSortExpr { + }], + vec![PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, - }]), + }], ]); let test_cases = vec![ @@ -738,7 +737,7 @@ mod tests { Arc::clone(col_d), Operator::Plus, Arc::clone(col_b), - )) as Arc, + )) as _, SortProperties::Ordered(option_asc), ), // b @@ -803,7 +802,7 @@ mod tests { nulls_first: true, }; // [d ASC, h DESC] also satisfies schema. - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, @@ -812,7 +811,7 @@ mod tests { expr: Arc::clone(col_h), options: option_desc, }, - ])]); + ]); let test_cases = vec![ // TEST CASE 1 (vec![col_a], vec![(col_a, option_asc)]), @@ -1232,11 +1231,11 @@ mod tests { // Assume existing ordering is [c ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - eq_properties.add_new_ordering(LexOrdering::from(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), - ])); + ]); // Add equality condition c = concat(a, b) eq_properties.add_equal_conditions(&col_c, &a_concat_b)?; @@ -1323,11 +1322,11 @@ mod tests { // Assume existing ordering is [concat(a, b) ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - eq_properties.add_new_ordering(LexOrdering::from(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr::new_default(Arc::clone(&a_concat_b)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), - ])); + ]); // Add equality condition c = concat(a, b) eq_properties.add_equal_conditions(&col_c, &a_concat_b)?; @@ -1445,7 +1444,7 @@ mod tests { }; // Initial ordering: [a ASC, b DESC, c ASC] - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, @@ -1458,7 +1457,7 @@ mod tests { expr: Arc::clone(&col_c), options: asc, }, - ])]); + ]); // New ordering: [a ASC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { @@ -1497,7 +1496,7 @@ mod tests { let asc = SortOptions::default(); // Initial ordering: [a ASC, c ASC] - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, @@ -1506,7 +1505,7 @@ mod tests { expr: Arc::clone(&col_c), options: asc, }, - ])]); + ]); // New ordering: [b ASC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { @@ -1545,7 +1544,7 @@ mod tests { }; // Initial ordering: [a ASC, b DESC] - eq_properties.add_new_orderings([LexOrdering::new(vec![ + eq_properties.add_new_ordering([ PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, @@ -1554,7 +1553,7 @@ mod tests { expr: Arc::clone(&col_b), options: desc, }, - ])]); + ]); // New ordering: [a DESC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { @@ -1593,7 +1592,7 @@ mod tests { // Orderings: [d ASC, a ASC], [e ASC] eq_properties.add_new_orderings([ - LexOrdering::new(vec![ + vec![ PhysicalSortExpr { expr: Arc::clone(&col_d), options: asc, @@ -1602,11 +1601,11 @@ mod tests { expr: Arc::clone(&col_a), options: asc, }, - ]), - LexOrdering::new(vec![PhysicalSortExpr { + ], + vec![PhysicalSortExpr { expr: Arc::clone(&col_e), options: asc, - }]), + }], ]); // Initial ordering: [b ASC, c ASC] @@ -1739,17 +1738,6 @@ mod tests { { let mut eq_properties = EquivalenceProperties::new(Arc::clone(schema)); - // Convert base ordering - let base_ordering = LexOrdering::new( - base_order - .iter() - .map(|col_name| PhysicalSortExpr { - expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), - }) - .collect(), - ); - // Convert string column names to orderings let satisfied_orderings: Vec = satisfied_orders .iter() @@ -1790,6 +1778,10 @@ mod tests { } // Add base ordering + let base_ordering = base_order.iter().map(|col_name| PhysicalSortExpr { + expr: col(col_name, schema).unwrap(), + options: SortOptions::default(), + }); eq_properties.add_new_ordering(base_ordering); // Add constraints diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 18d1aa3c6677f..82cc2786ae7f8 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -127,10 +127,10 @@ use itertools::Itertools; /// // with a single constant value of b /// let mut eq_properties = EquivalenceProperties::new(schema) /// .with_constants(vec![ConstExpr::from(col_b)]); -/// eq_properties.add_new_ordering(LexOrdering::new(vec![ +/// eq_properties.add_new_ordering([ /// PhysicalSortExpr::new_default(col_a).asc(), /// PhysicalSortExpr::new_default(col_c).desc(), -/// ])); +/// ]); /// /// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], const: [b@1(heterogeneous)]") /// ``` @@ -260,13 +260,16 @@ impl EquivalenceProperties { /// Adds new orderings into the existing ordering equivalence class. pub fn add_new_orderings( &mut self, - orderings: impl IntoIterator, + orderings: impl IntoIterator>, ) { self.oeq_class.add_new_orderings(orderings); } /// Adds a single ordering to the existing ordering equivalence class. - pub fn add_new_ordering(&mut self, ordering: LexOrdering) { + pub fn add_new_ordering( + &mut self, + ordering: impl IntoIterator, + ) { self.add_new_orderings([ordering]); } @@ -1484,7 +1487,7 @@ impl EquivalenceProperties { sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; Ok(sort_expr) }) - .collect::>()?; + .collect::>>()?; new_orderings.push(new_ordering); } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 2a95449286a40..981c709cf9b31 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -156,7 +156,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { ) -> Option> { debug_assert_eq!(self.order_by.len(), order_by_exprs.len()); - let new_order_by: Vec = self + let new_order_by = self .order_by .iter() .zip(order_by_exprs) diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 5ef22c18f4586..009c55c793498 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -33,7 +33,7 @@ use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// A window expr that takes the form of a [`StandardWindowFunctionExpr`]. #[derive(Debug)] @@ -282,7 +282,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( ) { if partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eqp.add_new_orderings([LexOrdering::new(vec![expr])]); + eqp.add_new_ordering([expr]); } else { // If we have a PARTITION BY, standard functions can not introduce // a global ordering unless the existing ordering is compatible @@ -293,7 +293,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( let (mut ordering, _) = eqp.find_longest_permutation(partition_by); if ordering.len() == partition_by.len() { ordering.push(expr); - eqp.add_new_orderings([ordering.into()]); + eqp.add_new_ordering(ordering); } } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a40e988874d52..e578194bd6c15 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1337,9 +1337,9 @@ mod tests { impl SortedUnboundedExec { fn compute_properties(schema: SchemaRef) -> PlanProperties { let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_orderings(vec![LexOrdering::new(vec![ - PhysicalSortExpr::new_default(Arc::new(Column::new("c1", 0))), - ])]); + eq_properties.add_new_ordering([PhysicalSortExpr::new_default(Arc::new( + Column::new("c1", 0), + ))]); PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index b9834c8f63d1f..62077ae2ca29f 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -161,7 +161,7 @@ impl SortPreservingMergeExec { ) -> PlanProperties { let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_per_partition_constants(); - eq_properties.add_new_orderings(vec![ordering]); + eq_properties.add_new_ordering(ordering); PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning @@ -1308,10 +1308,11 @@ mod tests { .map(|(i, f)| Arc::new(Column::new(f.name(), i)) as Arc) .collect::>(); let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_orderings(vec![columns - .iter() - .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))) - .collect::()]); + eq_properties.add_new_ordering( + columns + .iter() + .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))), + ); PlanProperties::new( eq_properties, Partitioning::Hash(columns, 3), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1c907a69d998b..965984af20926 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -436,12 +436,10 @@ pub(crate) fn window_equivalence_properties( let window_col = Column::new(expr.name(), i + input_schema_len); if no_partitioning { // Reverse set-monotonic cases with no partitioning: - let new_ordering = - vec![LexOrdering::new(vec![PhysicalSortExpr::new( - Arc::new(window_col), - SortOptions::new(increasing, true), - )])]; - window_eq_properties.add_new_orderings(new_ordering); + window_eq_properties.add_new_ordering([PhysicalSortExpr::new( + Arc::new(window_col), + SortOptions::new(increasing, true), + )]); } else { // Reverse set-monotonic cases for all orderings: for mut lex in all_satisfied_lexs.into_iter() { @@ -478,19 +476,19 @@ pub(crate) fn window_equivalence_properties( set_monotonicity.eq(&SetMonotonicity::Increasing); let window_col = Column::new(expr.name(), i + input_schema_len); if increasing && (asc || no_partitioning) { - let new_ordering = - LexOrdering::new(vec![PhysicalSortExpr::new( + window_eq_properties.add_new_ordering([ + PhysicalSortExpr::new( Arc::new(window_col), SortOptions::new(false, false), - )]); - window_eq_properties.add_new_ordering(new_ordering); + ), + ]); } else if !increasing && (!asc || no_partitioning) { - let new_ordering = - LexOrdering::new(vec![PhysicalSortExpr::new( + window_eq_properties.add_new_ordering([ + PhysicalSortExpr::new( Arc::new(window_col), SortOptions::new(true, false), - )]); - window_eq_properties.add_new_ordering(new_ordering); + ), + ]); }; } } From 166f9a465551011560b280a35ae7331407eea92e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 5 Apr 2025 15:31:39 +0300 Subject: [PATCH 061/167] Simplifications --- .../src/equivalence/properties/mod.rs | 41 +++++++++---------- 1 file changed, 19 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 82cc2786ae7f8..1f322af183139 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -503,19 +503,17 @@ impl EquivalenceProperties { &self, sort_exprs: impl IntoIterator, ) -> Option { - let normalized_sort_exprs = self.eq_group.normalize_sort_exprs(sort_exprs); - let mut constant_exprs = vec![]; - constant_exprs.extend( - self.constants - .iter() - .map(|const_expr| Arc::clone(const_expr.expr())), - ); - let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); - // Prune redundant sections in the ordering: - let sort_exprs = normalized_sort_exprs + let constant_exprs = self + .constants .iter() - .filter(|&order| !physical_exprs_contains(&constants_normalized, &order.expr)) - .cloned() + .map(|const_expr| Arc::clone(const_expr.expr())); + let normalized_constants = self.eq_group.normalize_exprs(constant_exprs); + // Prune redundant sections in the ordering: + let sort_exprs = self + .eq_group + .normalize_sort_exprs(sort_exprs) + .into_iter() + .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) .collect::>(); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) } @@ -537,18 +535,17 @@ impl EquivalenceProperties { &self, sort_reqs: impl IntoIterator, ) -> Option { - let normalized_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); - let mut constant_exprs = vec![]; - constant_exprs.extend( - self.constants - .iter() - .map(|const_expr| Arc::clone(const_expr.expr())), - ); - let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); + let constant_exprs = self + .constants + .iter() + .map(|const_expr| Arc::clone(const_expr.expr())); + let normalized_constants = self.eq_group.normalize_exprs(constant_exprs); // Prune redundant sections in the requirement: - let reqs = normalized_reqs + let reqs = self + .eq_group + .normalize_sort_requirements(sort_reqs) .into_iter() - .filter(|order| !physical_exprs_contains(&constants_normalized, &order.expr)) + .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) .collect::>(); (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) } From 96b73b363d270b6ef2f21ac1ecad43129db2b5fc Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 5 Apr 2025 17:44:03 +0300 Subject: [PATCH 062/167] More cleanups --- .../physical-expr/src/equivalence/class.rs | 17 +-- .../src/equivalence/properties/mod.rs | 103 +++++++----------- .../src/equivalence/properties/union.rs | 2 +- 3 files changed, 42 insertions(+), 80 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index a2135fa7946a2..f455f197df5c4 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -69,7 +69,7 @@ use indexmap::{IndexMap, IndexSet}; // addressed in a separate issue/PR. #[derive(Debug, Clone)] pub struct ConstExpr { - /// The expression that is known to be constant (e.g. a `Column`) + /// The expression that is known to be constant (e.g. a `Column`) expr: Arc, /// Does the constant have the same value across all partitions? See /// struct docs for more details @@ -106,7 +106,7 @@ impl ConstExpr { /// Create a new constant expression from a physical expression. /// /// Note you can also use `ConstExpr::from` to create a constant expression - /// from a reference as well + /// from a reference as well. pub fn new(expr: Arc) -> Self { Self { expr, @@ -115,17 +115,13 @@ impl ConstExpr { } } - /// Set the `across_partitions` flag - /// - /// See struct docs for more details + /// Sets the `across_partitions` flag. pub fn with_across_partitions(mut self, across_partitions: AcrossPartitions) -> Self { self.across_partitions = across_partitions; self } - /// Is the expression the same across all partitions? - /// - /// See struct docs for more details + /// Indicates whther the expression the same across all partitions. pub fn across_partitions(&self) -> AcrossPartitions { self.across_partitions.clone() } @@ -149,11 +145,6 @@ impl ConstExpr { }) } - /// Returns true if this constant expression is equal to the given expression - pub fn eq_expr(&self, other: impl AsRef) -> bool { - self.expr.as_ref() == other.as_ref() - } - /// Returns a [`Display`]able list of `ConstExpr`. pub fn format_list(input: &[ConstExpr]) -> impl Display + '_ { struct DisplayableList<'a>(&'a [ConstExpr]); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 1f322af183139..3aab4bcafe0c5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -312,12 +312,6 @@ impl EquivalenceProperties { Ok(()) } - /// Track/register physical expressions with constant values. - #[deprecated(since = "43.0.0", note = "Use [`with_constants`] instead")] - pub fn add_constants(self, constants: impl IntoIterator) -> Self { - self.with_constants(constants) - } - /// Remove the specified constant pub fn remove_constant(mut self, c: &ConstExpr) -> Self { self.constants.retain(|existing| existing != c); @@ -335,15 +329,10 @@ impl EquivalenceProperties { let across_partitions = c.across_partitions(); let expr = c.owned_expr(); let normalized_expr = self.eq_group.normalize_expr(expr); - - if const_exprs_contains(&self.constants, &normalized_expr) { - return None; - } - - let const_expr = ConstExpr::from(normalized_expr) - .with_across_partitions(across_partitions); - - Some(const_expr) + (!const_exprs_contains(&self.constants, &normalized_expr)).then(|| { + let const_expr = ConstExpr::from(normalized_expr); + const_expr.with_across_partitions(across_partitions) + }) }) .collect::>(); @@ -352,9 +341,7 @@ impl EquivalenceProperties { // Discover any new orderings based on the constants for ordering in self.normalized_oeq_class().iter() { - if let Err(e) = self.discover_new_orderings(&ordering[0].expr) { - log::debug!("error discovering new orderings: {e}"); - } + self.discover_new_orderings(&ordering[0].expr).unwrap(); } self @@ -371,69 +358,53 @@ impl EquivalenceProperties { let eq_class = self .eq_group .iter() - .find_map(|class| { - class - .contains(&normalized_expr) - .then(|| class.clone().into_vec()) - }) + .find(|class| class.contains(&normalized_expr)) + .map(|class| class.clone().into_vec()) .unwrap_or_else(|| vec![Arc::clone(&normalized_expr)]); - let mut new_orderings: Vec = vec![]; - for ordering in self.normalized_oeq_class().iter() { + let mut new_orderings = vec![]; + for ordering in self.normalized_oeq_class() { if !ordering[0].expr.eq(&normalized_expr) { continue; } let leading_ordering_options = ordering[0].options; - for equivalent_expr in &eq_class { + 'exprs: for equivalent_expr in &eq_class { let children = equivalent_expr.children(); if children.is_empty() { continue; } - - // Check if all children match the next expressions in the ordering - let mut all_children_match = true; + // Check if all children match the next expressions in the ordering: let mut child_properties = vec![]; - - // Build properties for each child based on the next expressions - for (i, child) in children.iter().enumerate() { - if let Some(next) = ordering.get(i + 1) { - if !child.as_ref().eq(next.expr.as_ref()) { - all_children_match = false; - break; - } - child_properties.push(ExprProperties { - sort_properties: SortProperties::Ordered(next.options), - range: Interval::make_unbounded( - &child.data_type(&self.schema)?, - )?, - preserves_lex_ordering: true, - }); - } else { - all_children_match = false; - break; + // Build properties for each child based on the next expression: + for (i, child) in children.into_iter().enumerate() { + let Some(next) = ordering.get(i + 1) else { + break 'exprs; + }; + if !next.expr.eq(child) { + break 'exprs; } + let data_type = child.data_type(&self.schema)?; + child_properties.push(ExprProperties { + sort_properties: SortProperties::Ordered(next.options), + range: Interval::make_unbounded(&data_type)?, + preserves_lex_ordering: true, + }); } - - if all_children_match { - // Check if the expression is monotonic in all arguments - if let Ok(expr_properties) = - equivalent_expr.get_properties(&child_properties) - { - if expr_properties.preserves_lex_ordering - && SortProperties::Ordered(leading_ordering_options) - == expr_properties.sort_properties - { - // Assume existing ordering is [c ASC, a ASC, b ASC] - // When equality c = f(a,b) is given, if we know that given ordering `[a ASC, b ASC]`, - // ordering `[f(a,b) ASC]` is valid, then we can deduce that ordering `[a ASC, b ASC]` is also valid. - // Hence, ordering `[a ASC, b ASC]` can be added to the state as a valid ordering. - // (e.g. existing ordering where leading ordering is removed) - new_orderings.push(LexOrdering::new(ordering[1..].to_vec())); - break; - } - } + // Check if the expression is monotonic in all arguments: + let expr_properties = + equivalent_expr.get_properties(&child_properties)?; + if expr_properties.preserves_lex_ordering + && SortProperties::Ordered(leading_ordering_options) + == expr_properties.sort_properties + { + // Assume existing ordering is `[c ASC, a ASC, b ASC]`. When + // equality `c = f(a, b)` is given, the ordering `[a ASC, b ASC]`, + // implies the ordering `[f(a, b) ASC]`. Thus, we can deduce that + // ordering `[a ASC, b ASC]` is also valid. + new_orderings.push(ordering[1..].to_vec()); + break; } } } diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 0a388e63828d7..dcb264d26349d 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -302,7 +302,7 @@ fn advance_if_matches_constant( constants: &[ConstExpr], ) -> Option { let expr = iter.peek()?; - let const_expr = constants.iter().find(|c| c.eq_expr(&expr.expr))?; + let const_expr = constants.iter().find(|c| expr.expr.eq(c.expr()))?; let found_expr = PhysicalSortExpr::new(Arc::clone(const_expr.expr()), expr.options); iter.next(); Some(found_expr) From 159139a031e2ade25f27da3b9cba7c6ce655f425 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 6 Apr 2025 16:14:04 +0300 Subject: [PATCH 063/167] API Simplifications --- .../physical-expr/src/equivalence/class.rs | 38 ----------- .../src/equivalence/properties/mod.rs | 65 +++++-------------- 2 files changed, 18 insertions(+), 85 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index f455f197df5c4..7d3e022212851 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -488,44 +488,6 @@ impl EquivalenceGroup { sort_requirement } - /// This function applies the `normalize_expr` function for all expressions - /// in `exprs` and returns the corresponding normalized physical expressions. - pub fn normalize_exprs( - &self, - exprs: impl IntoIterator>, - ) -> Vec> { - exprs - .into_iter() - .map(|expr| self.normalize_expr(expr)) - .collect() - } - - /// This function applies the `normalize_sort_expr` function for all sort - /// expressions in `sort_exprs` and returns the corresponding normalized - /// sort expressions. - pub fn normalize_sort_exprs<'a>( - &self, - sort_exprs: impl IntoIterator, - ) -> Vec { - sort_exprs - .into_iter() - .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) - .collect() - } - - /// This function applies the `normalize_sort_requirement` function for all - /// requirements in `sort_reqs` and returns the corresponding normalized - /// sort requirements. - pub fn normalize_sort_requirements<'a>( - &self, - sort_reqs: impl IntoIterator, - ) -> Vec { - sort_reqs - .into_iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect() - } - /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. pub fn project_expr( diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 3aab4bcafe0c5..645787d7d1a4c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -474,16 +474,15 @@ impl EquivalenceProperties { &self, sort_exprs: impl IntoIterator, ) -> Option { - let constant_exprs = self + let normalized_constants = self .constants .iter() - .map(|const_expr| Arc::clone(const_expr.expr())); - let normalized_constants = self.eq_group.normalize_exprs(constant_exprs); + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .collect::>(); // Prune redundant sections in the ordering: - let sort_exprs = self - .eq_group - .normalize_sort_exprs(sort_exprs) + let sort_exprs = sort_exprs .into_iter() + .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr.clone())) .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) .collect::>(); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) @@ -506,16 +505,15 @@ impl EquivalenceProperties { &self, sort_reqs: impl IntoIterator, ) -> Option { - let constant_exprs = self + let normalized_constants = self .constants .iter() - .map(|const_expr| Arc::clone(const_expr.expr())); - let normalized_constants = self.eq_group.normalize_exprs(constant_exprs); + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .collect::>(); // Prune redundant sections in the requirement: - let reqs = self - .eq_group - .normalize_sort_requirements(sort_reqs) + let reqs = sort_reqs .into_iter() + .map(|req| self.eq_group.normalize_sort_requirement(req.clone())) .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) .collect::>(); (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) @@ -1286,38 +1284,15 @@ impl EquivalenceProperties { // As an example, assume that we know columns `a` and `b` are constant. // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will // return `false`. - let const_exprs = self + let normalized_constants = self .constants .iter() - .map(|const_expr| Arc::clone(const_expr.expr())); - let normalized_constants = self.eq_group.normalize_exprs(const_exprs); + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .collect::>(); let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); is_constant_recurse(&normalized_constants, &normalized_expr) } - /// This function determines whether the provided expression is constant - /// across partitions based on the known constants. - /// - /// # Parameters - /// - /// - `expr`: A reference to a `Arc` representing the - /// expression to be checked. - /// - /// # Returns - /// - /// Returns `true` if the expression is constant across all partitions according - /// to equivalence group, `false` otherwise - #[deprecated( - since = "45.0.0", - note = "Use [`is_expr_constant_across_partitions`] instead" - )] - pub fn is_expr_constant_accross_partitions( - &self, - expr: &Arc, - ) -> bool { - self.is_expr_constant_across_partitions(expr) - } - /// This function determines whether the provided expression is constant /// across partitions based on the known constants. /// @@ -1337,21 +1312,17 @@ impl EquivalenceProperties { // As an example, assume that we know columns `a` and `b` are constant. // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will // return `false`. - let const_exprs = self + let normalized_constants = self .constants .iter() - .filter_map(|const_expr| { - if matches!( + .filter(|const_expr| { + matches!( const_expr.across_partitions(), AcrossPartitions::Uniform { .. } - ) { - Some(Arc::clone(const_expr.expr())) - } else { - None - } + ) }) + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) .collect::>(); - let normalized_constants = self.eq_group.normalize_exprs(const_exprs); let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); is_constant_recurse(&normalized_constants, &normalized_expr) } From a3eb637044706048588e081aa65f04e03e7e2929 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 6 Apr 2025 16:55:50 +0300 Subject: [PATCH 064/167] Improve comments --- .../src/equivalence/properties/mod.rs | 67 ++++++++++--------- 1 file changed, 35 insertions(+), 32 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 645787d7d1a4c..4a0a03385ee1b 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -461,24 +461,24 @@ impl EquivalenceProperties { } /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the - /// equivalence group and the ordering equivalence class within. + /// equivalence group and the ordering equivalence class within. Returns + /// a `LexOrdering` instance if the expressions define a proper lexicographical + /// ordering. It works by: + /// - Removing expressions that have a constant value from the given expressions. + /// - Replacing sections that belong to some equivalence class in the equivalence + /// group with the first entry in the matching equivalence class. /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. - /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` - /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this - /// function would return `vec![a ASC, c ASC]`. Internally, it would first - /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result - /// after deduplication. + /// Assume that `self.eq_group` states column `a` and `b` are aliases. Also + /// assume that `self.oeq_class` contains equivalent orderings `d ASC` and + /// `a ASC, c ASC` (in the sense that both describe the ordering of the + /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function + /// would return `[a ASC, c ASC]`. Internally, it would first normalize to + /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. pub fn normalize_sort_exprs<'a>( &self, sort_exprs: impl IntoIterator, ) -> Option { - let normalized_constants = self - .constants - .iter() - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) - .collect::>(); + let normalized_constants = self.normalized_constant_exprs(); // Prune redundant sections in the ordering: let sort_exprs = sort_exprs .into_iter() @@ -489,27 +489,24 @@ impl EquivalenceProperties { } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the - /// equivalence group and the ordering equivalence class within. It works by: - /// - Removing expressions that have a constant value from the given requirement. + /// equivalence group and the ordering equivalence class within. Returns + /// a `LexRequirement` instance if the expressions define a proper lexicographical + /// ordering requirement. It works by: + /// - Removing expressions that have a constant value from the given requirements. /// - Replacing sections that belong to some equivalence class in the equivalence /// group with the first entry in the matching equivalence class. /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. - /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` - /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this - /// function would return `vec![a ASC, c ASC]`. Internally, it would first - /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result - /// after deduplication. + /// Assume that `self.eq_group` states column `a` and `b` are aliases. Also + /// assume that `self.oeq_class` contains equivalent orderings `d ASC` and + /// `a ASC, c ASC` (in the sense that both describe the ordering of the + /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function + /// would return `[a ASC, c ASC]`. Internally, it would first normalize to + /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. fn normalize_sort_requirements<'a>( &self, sort_reqs: impl IntoIterator, ) -> Option { - let normalized_constants = self - .constants - .iter() - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) - .collect::>(); + let normalized_constants = self.normalized_constant_exprs(); // Prune redundant sections in the requirement: let reqs = sort_reqs .into_iter() @@ -1171,6 +1168,16 @@ impl EquivalenceProperties { self.constraints.project(&indices) } + /// Returns normalized versions of the expressions defining constants. + /// Normalization removes duplicates and standardizes expressions + /// according to the equivalence group within. + fn normalized_constant_exprs(&self) -> Vec> { + self.constants + .iter() + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .collect() + } + /// Projects the equivalences within according to `mapping` /// and `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { @@ -1284,11 +1291,7 @@ impl EquivalenceProperties { // As an example, assume that we know columns `a` and `b` are constant. // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will // return `false`. - let normalized_constants = self - .constants - .iter() - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) - .collect::>(); + let normalized_constants = self.normalized_constant_exprs(); let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); is_constant_recurse(&normalized_constants, &normalized_expr) } From e56306c3816cc4579996df36642da6ad4fccfbd3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 6 Apr 2025 19:47:44 +0300 Subject: [PATCH 065/167] Use vector in Expr structs --- datafusion/core/src/physical_planner.rs | 31 ++++----- .../src/accumulator.rs | 4 +- .../functions-aggregate-common/src/utils.rs | 8 +-- .../functions-aggregate/benches/count.rs | 9 ++- datafusion/functions-aggregate/benches/sum.rs | 9 ++- .../functions-aggregate/src/array_agg.rs | 36 +++++----- .../functions-aggregate/src/first_last.rs | 69 +++++++++++-------- .../functions-aggregate/src/nth_value.rs | 7 +- datafusion/functions-aggregate/src/stddev.rs | 4 +- datafusion/physical-expr/src/aggregate.rs | 67 ++++++++---------- .../src/update_aggr_exprs.rs | 12 ++-- .../physical-plan/src/aggregates/mod.rs | 32 ++++----- datafusion/proto/src/physical_plan/mod.rs | 8 +-- .../proto/src/physical_plan/to_proto.rs | 9 +-- .../tests/cases/roundtrip_physical_plan.rs | 6 +- 15 files changed, 154 insertions(+), 157 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5f5ff115460f9..7892581caa87c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -82,7 +82,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::{create_physical_sort_exprs, LexOrdering}; +use datafusion_physical_expr::{create_physical_sort_exprs, PhysicalSortExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; @@ -1573,8 +1573,8 @@ type AggregateExprWithOptionalArgs = ( Arc, // The filter clause, if any Option>, - // Ordering requirements, if any - Option, + // Expressions in the ORDER BY clause + Vec, ); /// Create an aggregate expression with a name from a logical expression @@ -1618,22 +1618,19 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( let ignore_nulls = null_treatment.unwrap_or(NullTreatment::RespectNulls) == NullTreatment::IgnoreNulls; - let (agg_expr, filter, order_by) = { - let ordering = match order_by { - Some(exprs) => Some( - create_physical_sort_exprs( - exprs, - logical_input_schema, - execution_props, - )? - .into(), - ), - None => None, + let (agg_expr, filter, order_bys) = { + let order_bys = match order_by { + Some(exprs) => create_physical_sort_exprs( + exprs, + logical_input_schema, + execution_props, + )?, + None => vec![], }; let agg_expr = AggregateExprBuilder::new(func.to_owned(), physical_args.to_vec()) - .order_by(ordering.clone()) + .order_by(order_bys.clone()) .schema(Arc::new(physical_input_schema.to_owned())) .alias(name) .human_display(human_displan) @@ -1642,10 +1639,10 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( .build() .map(Arc::new)?; - (agg_expr, filter, ordering) + (agg_expr, filter, order_bys) }; - Ok((agg_expr, filter, order_by)) + Ok((agg_expr, filter, order_bys)) } other => internal_err!("Invalid aggregate expression '{other:?}'"), } diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index b9e4451f05002..786706488657c 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use std::sync::Arc; /// [`AccumulatorArgs`] contains information about how an aggregate @@ -50,7 +50,7 @@ pub struct AccumulatorArgs<'a> { /// ```sql /// SELECT FIRST_VALUE(column1 ORDER BY column2) FROM t; /// ``` - pub ordering_req: Option<&'a LexOrdering>, + pub order_bys: &'a [PhysicalSortExpr], /// Whether the aggregation is running in reverse order pub is_reversed: bool, diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index 083dac615b5d1..8235a2b584113 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -30,7 +30,7 @@ use arrow::{ }; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( @@ -87,13 +87,13 @@ pub fn adjust_output_array(data_type: &DataType, array: ArrayRef) -> Result Vec { - ordering_req + order_bys .iter() .zip(data_types.iter()) .map(|(sort_expr, dtype)| { diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 44f362444bd21..a9a67ded94bb5 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -15,14 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::{DataType, Field, Int32Type, Schema}; use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; + use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; -use std::sync::Arc; + +use criterion::{black_box, criterion_group, criterion_main, Criterion}; fn prepare_accumulator() -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Int32, true)])); @@ -30,7 +33,7 @@ fn prepare_accumulator() -> Box { return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, - ordering_req: None, + order_bys: &[], is_reversed: false, name: "COUNT(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index 44aa932b2f843..1486b740d18d8 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -15,14 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use arrow::array::{ArrayRef, BooleanArray}; use arrow::datatypes::{DataType, Field, Int64Type, Schema}; use arrow::util::bench_util::{create_boolean_array, create_primitive_array}; -use criterion::{black_box, criterion_group, criterion_main, Criterion}; + use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; use datafusion_physical_expr::expressions::col; -use std::sync::Arc; + +use criterion::{black_box, criterion_group, criterion_main, Criterion}; fn prepare_accumulator(data_type: &DataType) -> Box { let schema = Arc::new(Schema::new(vec![Field::new("f", data_type.clone(), true)])); @@ -30,7 +33,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box { return_type: data_type, schema: &schema, ignore_nulls: false, - ordering_req: None, + order_bys: &[], is_reversed: false, name: "SUM(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index d23f8c6385d50..90b8f765e908b 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -153,25 +153,27 @@ impl AggregateUDFImpl for ArrayAgg { // ARRAY_AGG(DISTINCT concat(col, '') ORDER BY concat(col, '')) <- Valid // ARRAY_AGG(DISTINCT col ORDER BY other_col) <- Invalid // ARRAY_AGG(DISTINCT col ORDER BY concat(col, '')) <- Invalid - let mut sort_option: Option = None; - if let Some(ordering_req) = acc_args.ordering_req { - if ordering_req.len() > 1 || !ordering_req[0].expr.eq(&acc_args.exprs[0]) - { - return exec_err!("In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list"); + let sort_option = match acc_args.order_bys { + [single] if single.expr.eq(&acc_args.exprs[0]) => Some(single.options), + [] => None, + _ => { + return exec_err!( + "In an aggregate with DISTINCT, ORDER BY expressions must appear in argument list" + ); } - sort_option = Some(ordering_req[0].options) - } + }; return Ok(Box::new(DistinctArrayAggAccumulator::try_new( &data_type, sort_option, )?)); } - let Some(ordering_req) = acc_args.ordering_req else { + let order_bys = acc_args.order_bys; + if order_bys.is_empty() { return Ok(Box::new(ArrayAggAccumulator::try_new(&data_type)?)); }; - let ordering_dtypes = ordering_req + let ordering_dtypes = order_bys .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -179,7 +181,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - ordering_req.clone(), + LexOrdering::new(order_bys.to_vec()), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) @@ -661,7 +663,7 @@ mod tests { use datafusion_common::cast::as_generic_string_array; use datafusion_common::internal_err; use datafusion_physical_expr::expressions::Column; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use std::sync::Arc; #[test] @@ -926,7 +928,7 @@ mod tests { struct ArrayAggAccumulatorBuilder { data_type: DataType, distinct: bool, - ordering: Option, + order_bys: Vec, schema: Schema, } @@ -939,7 +941,7 @@ mod tests { Self { data_type: data_type.clone(), distinct: Default::default(), - ordering: None, + order_bys: vec![], schema: Schema { fields: Fields::from(vec![Field::new( "col", @@ -966,11 +968,7 @@ mod tests { ), sort_options, ); - if let Some(existing_ordering) = self.ordering.as_mut() { - existing_ordering.extend([new_order]); - } else { - self.ordering = Some(LexOrdering::from(vec![new_order])); - } + self.order_bys.push(new_order); self } @@ -979,7 +977,7 @@ mod tests { return_type: &self.data_type, schema: &self.schema, ignore_nulls: false, - ordering_req: self.ordering.as_ref(), + order_bys: &self.order_bys, is_reversed: false, name: "", is_distinct: self.distinct, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 82720ba6425c3..ee7b406127ce7 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -134,23 +134,27 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if let Some(ordering_req) = acc_args.ordering_req { - let ordering_dtypes = ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - return FirstValueAccumulator::try_new( + if acc_args.order_bys.is_empty() { + return TrivialFirstValueAccumulator::try_new( acc_args.return_type, - &ordering_dtypes, - ordering_req.clone(), acc_args.ignore_nulls, ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }); + .map(|acc| Box::new(acc) as _); } - TrivialFirstValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) - .map(|acc| Box::new(acc) as _) + let ordering_dtypes = acc_args + .order_bys + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + FirstValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + acc_args.order_bys.iter().cloned().collect(), + acc_args.ignore_nulls, + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { @@ -166,7 +170,7 @@ impl AggregateUDFImpl for FirstValue { fn groups_accumulator_supported(&self, args: AccumulatorArgs) -> bool { use DataType::*; - args.ordering_req.is_some() + !args.order_bys.is_empty() && matches!( args.return_type, Int8 | Int16 @@ -196,17 +200,18 @@ impl AggregateUDFImpl for FirstValue { fn create_accumulator( args: AccumulatorArgs, ) -> Result> { - let Some(ordering_req) = args.ordering_req else { + if args.order_bys.is_empty() { return internal_err!("Groups accumulator must have an ordering."); }; - let ordering_dtypes = ordering_req + let ordering_dtypes = args + .order_bys .iter() .map(|e| e.expr.data_type(args.schema)) .collect::>>()?; Ok(Box::new(FirstPrimitiveGroupsAccumulator::::try_new( - ordering_req.clone(), + args.order_bys.iter().cloned().collect(), args.ignore_nulls, args.return_type, &ordering_dtypes, @@ -1032,23 +1037,27 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if let Some(ordering_req) = acc_args.ordering_req { - let ordering_dtypes = ordering_req - .iter() - .map(|e| e.expr.data_type(acc_args.schema)) - .collect::>>()?; - return LastValueAccumulator::try_new( + if acc_args.order_bys.is_empty() { + return TrivialLastValueAccumulator::try_new( acc_args.return_type, - &ordering_dtypes, - ordering_req.clone(), acc_args.ignore_nulls, ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }); + .map(|acc| Box::new(acc) as _); } - TrivialLastValueAccumulator::try_new(acc_args.return_type, acc_args.ignore_nulls) - .map(|acc| Box::new(acc) as _) + let ordering_dtypes = acc_args + .order_bys + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + LastValueAccumulator::try_new( + acc_args.return_type, + &ordering_dtypes, + acc_args.order_bys.iter().cloned().collect(), + acc_args.ignore_nulls, + ) + .map(|acc| { + Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ + }) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index e353a6b622913..6b8a1e3575a92 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -148,11 +148,12 @@ impl AggregateUDFImpl for NthValueAgg { } }; - let Some(ordering_req) = acc_args.ordering_req else { + if acc_args.order_bys.is_empty() { return TrivialNthValueAccumulator::try_new(n, acc_args.return_type) .map(|acc| Box::new(acc) as _); }; - let ordering_dtypes = ordering_req + let ordering_dtypes = acc_args + .order_bys .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -162,7 +163,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &data_type, &ordering_dtypes, - ordering_req.clone(), + acc_args.order_bys.iter().cloned().collect(), ) .map(|acc| Box::new(acc) as _) } diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index b26572c69c225..9f60ec393c356 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -438,7 +438,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: None, + order_bys: &[], name: "a", is_distinct: false, is_reversed: false, @@ -449,7 +449,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: None, + order_bys: &[], name: "a", is_distinct: false, is_reversed: false, diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 9a73a4a536a37..7cdc8e5970867 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -52,7 +52,7 @@ use datafusion_functions_aggregate_common::accumulator::{ }; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// Builder for physical [`AggregateFunctionExpr`] /// @@ -69,7 +69,7 @@ pub struct AggregateExprBuilder { /// Arrow Schema for the aggregate function schema: SchemaRef, /// The physical order by expressions - ordering_req: Option, + order_bys: Vec, /// Whether to ignore null values ignore_nulls: bool, /// Whether is distinct aggregate function @@ -86,7 +86,7 @@ impl AggregateExprBuilder { alias: None, human_display: String::default(), schema: Arc::new(Schema::empty()), - ordering_req: None, + order_bys: vec![], ignore_nulls: false, is_distinct: false, is_reversed: false, @@ -194,7 +194,7 @@ impl AggregateExprBuilder { alias, human_display, schema, - ordering_req, + order_bys, ignore_nulls, is_distinct, is_reversed, @@ -203,16 +203,12 @@ impl AggregateExprBuilder { return internal_err!("args should not be empty"); } - let mut ordering_fields = vec![]; + let ordering_types = order_bys + .iter() + .map(|e| e.expr.data_type(&schema)) + .collect::>>()?; - if let Some(ordering_req) = &ordering_req { - let ordering_types = ordering_req - .iter() - .map(|e| e.expr.data_type(&schema)) - .collect::>>()?; - - ordering_fields = utils::ordering_fields(ordering_req, &ordering_types); - } + let ordering_fields = utils::ordering_fields(&order_bys, &ordering_types); let input_exprs_types = args .iter() @@ -243,7 +239,7 @@ impl AggregateExprBuilder { name, human_display, schema: Arc::unwrap_or_clone(schema), - ordering_req, + order_bys, ignore_nulls, ordering_fields, is_distinct, @@ -268,8 +264,8 @@ impl AggregateExprBuilder { self } - pub fn order_by(mut self, order_by: Option) -> Self { - self.ordering_req = order_by; + pub fn order_by(mut self, order_bys: Vec) -> Self { + self.order_bys = order_bys; self } @@ -319,7 +315,7 @@ pub struct AggregateFunctionExpr { human_display: String, schema: Schema, // The physical order by expressions - ordering_req: Option, + order_bys: Vec, // Whether to ignore null values ignore_nulls: bool, // fields used for order sensitive aggregation functions @@ -385,7 +381,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: self.ordering_req.as_ref(), + order_bys: self.order_bys.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -408,26 +404,24 @@ impl AggregateFunctionExpr { self.fun.state_fields(args) } - /// Order by requirements for the aggregate function - /// By default it is `None` (there is no requirement) - /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - pub fn order_bys(&self) -> Option<&LexOrdering> { - let Some(ordering_req) = &self.ordering_req else { - return None; - }; - (!self.order_sensitivity().is_insensitive()).then_some(ordering_req) + /// Returns the ORDER BY expressions for the aggregate function. + pub fn order_bys(&self) -> &[PhysicalSortExpr] { + if self.order_sensitivity().is_insensitive() { + &[] + } else { + &self.order_bys + } } /// Indicates whether aggregator can produce the correct result with any /// arbitrary input ordering. By default, we assume that aggregate expressions /// are order insensitive. pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { - if self.ordering_req.is_some() { - // If there is requirement, use the sensitivity of the implementation - self.fun.order_sensitivity() - } else { - // If no requirement, aggregator is order insensitive + if self.order_bys.is_empty() { AggregateOrderSensitivity::Insensitive + } else { + // If there is an ORDER BY clause, use the sensitivity of the implementation: + self.fun.order_sensitivity() } } @@ -455,7 +449,7 @@ impl AggregateFunctionExpr { }; AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) - .order_by(self.ordering_req.clone()) + .order_by(self.order_bys.clone()) .schema(Arc::new(self.schema.clone())) .alias(self.name().to_string()) .with_ignore_nulls(self.ignore_nulls) @@ -471,7 +465,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: self.ordering_req.as_ref(), + order_bys: self.order_bys.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -540,7 +534,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: self.ordering_req.as_ref(), + order_bys: self.order_bys.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -559,7 +553,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: self.ordering_req.as_ref(), + order_bys: self.order_bys.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -586,7 +580,7 @@ impl AggregateFunctionExpr { replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) - .order_by(self.ordering_req.as_ref().map(|o| o.reverse_each())) + .order_by(self.order_bys.iter().map(|e| e.reverse()).collect()) .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) @@ -604,7 +598,6 @@ impl AggregateFunctionExpr { let args = self.expressions(); let order_by_exprs = self .order_bys() - .map_or_else(Vec::new, |ordering| ordering.iter().collect()) .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) .collect(); diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index b99efb8c08f79..7a08be6a9f55e 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -163,13 +163,12 @@ fn try_convert_aggregate_if_better( // Otherwise, leave it as is. if !aggr_expr.order_sensitivity().is_beneficial() { Ok(aggr_expr) - } else if let Some(order_bys) = order_bys { + } else if !order_bys.is_empty() { if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &order_bys - .clone() - .into_iter() - .map(Into::into) + .iter() + .map(|e| e.clone().into()) .collect::>(), )) { // Existing ordering satisfies the aggregator requirements: @@ -177,9 +176,8 @@ fn try_convert_aggregate_if_better( } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &order_bys - .reverse_each() - .into_iter() - .map(Into::into) + .iter() + .map(|e| e.reverse().into()) .collect::>(), )) { // Converting to reverse enables more efficient execution diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 51efd62269fe5..78aab5b27da90 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -648,7 +648,7 @@ impl AggregateExec { return false; } // ensure there are no order by expressions - if self.aggr_expr().iter().any(|e| e.order_bys().is_some()) { + if !self.aggr_expr().iter().all(|e| e.order_bys().is_empty()) { return false; } // ensure there is no output ordering; can this rule be relaxed? @@ -1052,7 +1052,7 @@ fn get_aggregate_expr_req( if !aggr_expr.order_sensitivity().hard_requires() || !agg_mode.is_first_stage() { return None; } - let mut req = aggr_expr.order_bys().cloned()?.to_vec(); + let mut req = aggr_expr.order_bys().to_vec(); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we @@ -1197,9 +1197,7 @@ pub fn aggregate_expressions( // Append ordering requirements to expressions' results. This // way order sensitive aggregators can satisfy requirement // themselves. - if let Some(ordering_req) = agg.order_bys() { - result.extend(ordering_req.iter().map(|item| Arc::clone(&item.expr))); - } + result.extend(agg.order_bys().iter().map(|item| Arc::clone(&item.expr))); result }) .collect()), @@ -2199,14 +2197,14 @@ mod tests { schema: &Schema, sort_options: SortOptions, ) -> Result> { - let ordering_req = [PhysicalSortExpr { + let order_bys = vec![PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; let args = [col("b", schema)?]; AggregateExprBuilder::new(first_value_udaf(), args.to_vec()) - .order_by(Some(LexOrdering::from_iter(ordering_req))) + .order_by(order_bys) .schema(Arc::new(schema.clone())) .alias(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2218,13 +2216,13 @@ mod tests { schema: &Schema, sort_options: SortOptions, ) -> Result> { - let ordering_req = [PhysicalSortExpr { + let order_bys = vec![PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, }]; let args = [col("b", schema)?]; AggregateExprBuilder::new(last_value_udaf(), args.to_vec()) - .order_by(Some(LexOrdering::from_iter(ordering_req))) + .order_by(order_bys) .schema(Arc::new(schema.clone())) .alias(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2361,12 +2359,12 @@ mod tests { // Aggregate requirements are // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let order_by_exprs = vec![ - None, - Some(vec![PhysicalSortExpr { + vec![], + vec![PhysicalSortExpr { expr: Arc::clone(col_a), options: options1, - }]), - Some(vec![ + }], + vec![ PhysicalSortExpr { expr: Arc::clone(col_a), options: options1, @@ -2379,8 +2377,8 @@ mod tests { expr: Arc::clone(col_c), options: options1, }, - ]), - Some(vec![ + ], + vec![ PhysicalSortExpr { expr: Arc::clone(col_a), options: options1, @@ -2389,7 +2387,7 @@ mod tests { expr: Arc::clone(col_b), options: options1, }, - ]), + ], ]; let common_requirement = LexOrdering::new(vec![ @@ -2407,7 +2405,7 @@ mod tests { .map(|order_by_expr| { AggregateExprBuilder::new(array_agg_udaf(), vec![Arc::clone(col_a)]) .alias("a") - .order_by(order_by_expr.map(LexOrdering::new)) + .order_by(order_by_expr) .schema(Arc::clone(&test_schema)) .build() .map(Arc::new) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index d8e687cfb79b2..dfc7b44d4cc3a 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -540,10 +540,10 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { match expr_type { ExprType::AggregateExpr(agg_node) => { let input_phy_expr = agg_node.expr.iter() - .map(|e| parse_physical_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; - let ordering_req = agg_node.ordering_req.iter() + .map(|e| parse_physical_expr(e, registry, &physical_schema, extension_codec)).collect::>()?; + let order_bys = agg_node.ordering_req.iter() .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)) - .collect::>>()?; + .collect::>()?; agg_node.aggregate_function.as_ref().map(|func| { match func { AggregateFunction::UserDefinedAggrFunction(udaf_name) => { @@ -557,7 +557,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .alias(name) .with_ignore_nulls(agg_node.ignore_nulls) .with_distinct(agg_node.distinct) - .order_by((!ordering_req.is_empty()).then(|| ordering_req.into())) + .order_by(order_bys) .build() .map(Arc::new) } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 52677ca8153b7..b3ae24685e8a8 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -52,11 +52,8 @@ pub fn serialize_physical_aggr_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; - let ordering_req = match aggr_expr.order_bys() { - Some(order) => order.to_vec(), - None => vec![], - }; - let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; + let order_bys = + serialize_physical_sort_exprs(aggr_expr.order_bys().iter().cloned(), codec)?; let name = aggr_expr.fun().name().to_string(); let mut buf = Vec::new(); @@ -66,7 +63,7 @@ pub fn serialize_physical_aggr_expr( protobuf::PhysicalAggregateExprNode { aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), expr: expressions, - ordering_req, + ordering_req: order_bys, distinct: aggr_expr.is_distinct(), ignore_nulls: aggr_expr.ignore_nulls(), fun_definition: (!buf.is_empty()).then_some(buf), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 156d06d7aef69..d6ebb6aaa799b 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -527,20 +527,20 @@ fn rountrip_aggregate_with_sort() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let sort_exprs = LexOrdering::new(vec![PhysicalSortExpr { + let sort_exprs = vec![PhysicalSortExpr { expr: col("b", &schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); + }]; let aggregates = vec![ AggregateExprBuilder::new(array_agg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) .alias("ARRAY_AGG(b)") - .order_by(Some(sort_exprs)) + .order_by(sort_exprs) .build() .map(Arc::new)?, ]; From a82fa38487056aa5fc42e9a317a204f94c9a909d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 6 Apr 2025 20:38:10 +0300 Subject: [PATCH 066/167] Fix doctests --- datafusion/physical-expr/src/aggregate.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 7cdc8e5970867..26b310fa450fe 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -176,7 +176,7 @@ impl AggregateExprBuilder { /// Arc::new(first_value), /// args /// ) - /// .order_by(Some(order_by.into())) + /// .order_by(order_by) /// .alias("first_a_by_x") /// .ignore_nulls() /// .build()?; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 965984af20926..303b2b1fa4e01 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -369,10 +369,9 @@ pub(crate) fn window_equivalence_properties( // Collect columns defining partitioning, and construct all `SortOptions` // variations for them. Then, we will check each one whether it satisfies // the existing ordering provided by the input plan. - let partition_by_orders = partitioning_exprs + let all_satisfied_lexs = partitioning_exprs .iter() - .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))); - let all_satisfied_lexs = partition_by_orders + .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))) .multi_cartesian_product() .filter(|lex| !lex.is_empty() && window_eq_properties.ordering_satisfy(lex)) .map(LexOrdering::new) From 6f86ebd4fc49ed537f5ce3ed4219fb032f71ef41 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 6 Apr 2025 21:04:42 +0300 Subject: [PATCH 067/167] Simplify sort --- datafusion/physical-plan/src/sorts/sort.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e578194bd6c15..c8ca61987c65f 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -58,7 +58,6 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -1034,10 +1033,7 @@ impl SortExec { preserve_partitioning: bool, ) -> PlanProperties { // Determine execution mode: - let requirement = LexRequirement::from(sort_exprs); - let sort_satisfied = input - .equivalence_properties() - .ordering_satisfy_requirement(&requirement); + let sort_satisfied = input.equivalence_properties().ordering_satisfy(&sort_exprs); // The emission type depends on whether the input is already sorted: // - If already sorted, we can emit results in the same way as the input @@ -1066,7 +1062,6 @@ impl SortExec { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: - let sort_exprs = LexOrdering::from(requirement); let eq_properties = input .equivalence_properties() .clone() @@ -1168,7 +1163,7 @@ impl ExecutionPlan for SortExec { let sort_satisfied = self .input .equivalence_properties() - .ordering_satisfy_requirement(&LexRequirement::from(self.expr.clone())); + .ordering_satisfy(&self.expr); match (sort_satisfied, self.fetch.as_ref()) { (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( From 1a1a5567b26149c0dca72167079b8f564b5e6d70 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 00:23:50 +0300 Subject: [PATCH 068/167] Simplify the get_finer_aggregate_exprs_requirement function --- .../physical-plan/src/aggregates/mod.rs | 51 +++++++------------ 1 file changed, 18 insertions(+), 33 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 78aab5b27da90..debeacc8b86fd 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1102,19 +1102,13 @@ pub fn get_finer_aggregate_exprs_requirement( ) -> Result> { let mut requirement = None; for aggr_expr in aggr_exprs.iter_mut() { - let (mut conflict, mut conflict_rev) = (false, false); - let mut finer = if let Some(aggr_req) = - get_aggregate_expr_req(aggr_expr, group_by, agg_mode) - { - if let Some(req) = requirement.as_ref() { - let finer = eq_properties.get_finer_ordering(req, &aggr_req); - conflict = finer.is_none(); - finer - } else { - eq_properties.normalize_sort_exprs(&aggr_req) - } + let Some(aggr_req) = get_aggregate_expr_req(aggr_expr, group_by, agg_mode) else { + continue; + }; + let mut finer = if let Some(req) = requirement.as_ref() { + eq_properties.get_finer_ordering(req, &aggr_req) } else { - requirement.clone() + eq_properties.normalize_sort_exprs(&aggr_req) }; if let Some(finer_ordering) = finer.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) @@ -1126,18 +1120,13 @@ pub fn get_finer_aggregate_exprs_requirement( let mut finer_rev = None; let mut rev_expr = aggr_expr.reverse_expr(); if let Some(reverse_aggr_expr) = rev_expr.take() { - finer_rev = if let Some(aggr_req) = - get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) - { - if let Some(req) = requirement.as_ref() { - let finer = eq_properties.get_finer_ordering(req, &aggr_req); - conflict_rev = finer.is_none(); - finer - } else { - eq_properties.normalize_sort_exprs(&aggr_req) - } + let Some(aggr_req) = get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) else { + continue; + }; + finer_rev = if let Some(req) = requirement.as_ref() { + eq_properties.get_finer_ordering(req, &aggr_req) } else { - requirement.clone() + eq_properties.normalize_sort_exprs(&aggr_req) }; if let Some(finer_ordering) = finer_rev.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) @@ -1164,14 +1153,12 @@ pub fn get_finer_aggregate_exprs_requirement( continue; } } - if conflict && conflict_rev { - // Neither the existing requirement nor the aggregate requirement - // satisfy the other, this means requirements are conflicting. - // Currently, we do not support conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); - } + // Neither the existing requirement nor the aggregate requirement + // satisfy the other, this means requirements are conflicting. + // Currently, we do not support conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); } Ok(requirement.map_or_else(Vec::new, |o| o.into_iter().map(Into::into).collect())) @@ -2344,8 +2331,6 @@ mod tests { async fn test_get_finest_requirements() -> Result<()> { let test_schema = create_test_schema()?; - // Assume column a and b are aliases - // Assume also that a ASC and c DESC describe the same global ordering for the table. (Since they are ordering equivalent). let options1 = SortOptions { descending: false, nulls_first: false, From 5b121d01ee9202aee03cec3abdae36764119afaf Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 01:12:54 +0300 Subject: [PATCH 069/167] Avoid hidden clones --- .../tests/fuzz_cases/equivalence/ordering.rs | 6 +-- .../fuzz_cases/equivalence/projection.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 10 ++--- .../src/equivalence/properties/dependency.rs | 29 ++++++------ .../src/equivalence/properties/mod.rs | 15 ++++--- .../src/equivalence/properties/union.rs | 3 +- .../src/enforce_sorting/mod.rs | 2 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/enforce_sorting/sort_pushdown.rs | 3 +- .../physical-plan/src/aggregates/mod.rs | 44 ++++++++++--------- datafusion/physical-plan/src/sorts/sort.rs | 6 ++- datafusion/physical-plan/src/windows/mod.rs | 6 ++- 12 files changed, 67 insertions(+), 61 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 1ab9bd559fc6a..f54b1cb2ecce1 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -75,7 +75,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(&requirement), + eq_properties.ordering_satisfy(requirement), expected, "{}", err_msg @@ -148,7 +148,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(&requirement), + eq_properties.ordering_satisfy(requirement), (expected | false), "{}", err_msg @@ -322,7 +322,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expected ); assert_eq!( - eq_properties.ordering_satisfy(&required), + eq_properties.ordering_satisfy(required), expected, "{err_msg}" ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index 7ea9f7e243c6c..c25d366ac9549 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -184,7 +184,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - projected_eq.ordering_satisfy(&requirement), + projected_eq.ordering_satisfy(requirement), expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 8bddc2381ac02..8e7c6be905aea 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -383,14 +383,14 @@ mod tests { Arc::clone(&input_schema), &[finer.clone()], ); - assert!(eq_properties_finer.ordering_satisfy(&crude)); + assert!(eq_properties_finer.ordering_satisfy(crude.clone())); // Crude ordering doesn't satisfy finer ordering. should return false let eq_properties_crude = EquivalenceProperties::new_with_orderings( Arc::clone(&input_schema), - &[crude.clone()], + &[crude], ); - assert!(!eq_properties_crude.ordering_satisfy(&finer)); + assert!(!eq_properties_crude.ordering_satisfy(finer)); Ok(()) } @@ -677,7 +677,7 @@ mod tests { let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs), + eq_properties.ordering_satisfy(reqs), expected, "{}", err_msg @@ -737,7 +737,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs), + eq_properties.ordering_satisfy(reqs), expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 51f3c682e4e0a..2ea09fa7a08ac 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1198,7 +1198,7 @@ mod tests { .collect::>()?; assert_eq!( - properties.ordering_satisfy(&sort), + properties.ordering_satisfy(sort), case.should_satisfy_ordering, "failed test '{}'", case.name @@ -1768,13 +1768,12 @@ mod tests { .collect(); // Test that orderings are not satisfied before adding constraints - for ordering in &satisfied_orderings { - assert!( - !eq_properties.ordering_satisfy(ordering), + for ordering in satisfied_orderings.clone() { + let err_msg = format!( "{}: ordering {:?} should not be satisfied before adding constraints", - name, - ordering + name, ordering ); + assert!(!eq_properties.ordering_satisfy(ordering), "{err_msg}"); } // Add base ordering @@ -1789,23 +1788,21 @@ mod tests { eq_properties.with_constraints(Constraints::new_unverified(constraints)); // Test that expected orderings are now satisfied - for ordering in &satisfied_orderings { - assert!( - eq_properties.ordering_satisfy(ordering), + for ordering in satisfied_orderings { + let err_msg = format!( "{}: ordering {:?} should be satisfied after adding constraints", - name, - ordering + name, ordering ); + assert!(eq_properties.ordering_satisfy(ordering), "{err_msg}"); } // Test that unsatisfied orderings remain unsatisfied - for ordering in &unsatisfied_orderings { - assert!( - !eq_properties.ordering_satisfy(ordering), + for ordering in unsatisfied_orderings { + let err_msg = format!( "{}: ordering {:?} should not be satisfied after adding constraints", - name, - ordering + name, ordering ); + assert!(!eq_properties.ordering_satisfy(ordering), "{err_msg}"); } } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 4a0a03385ee1b..963dea409aeed 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -225,6 +225,7 @@ impl EquivalenceProperties { OrderingEquivalenceClass::new( self.oeq_class .iter() + .cloned() .filter_map(|ordering| self.normalize_sort_exprs(ordering)) .collect(), ) @@ -474,15 +475,15 @@ impl EquivalenceProperties { /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function /// would return `[a ASC, c ASC]`. Internally, it would first normalize to /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. - pub fn normalize_sort_exprs<'a>( + pub fn normalize_sort_exprs( &self, - sort_exprs: impl IntoIterator, + sort_exprs: impl IntoIterator, ) -> Option { let normalized_constants = self.normalized_constant_exprs(); // Prune redundant sections in the ordering: let sort_exprs = sort_exprs .into_iter() - .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr.clone())) + .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) .collect::>(); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) @@ -518,9 +519,9 @@ impl EquivalenceProperties { /// Checks whether the given ordering is satisfied by any of the existing /// orderings. - pub fn ordering_satisfy<'a>( + pub fn ordering_satisfy( &self, - given: impl IntoIterator, + given: impl IntoIterator, ) -> bool { // First, standardize the given ordering: let Some(normalized_ordering) = self.normalize_sort_exprs(given) else { @@ -768,8 +769,8 @@ impl EquivalenceProperties { /// the latter. pub fn get_finer_ordering( &self, - lhs: &LexOrdering, - rhs: &LexOrdering, + lhs: LexOrdering, + rhs: LexOrdering, ) -> Option { let Some(mut rhs) = self.normalize_sort_exprs(rhs) else { return self.normalize_sort_exprs(lhs); diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index dcb264d26349d..b2327044a6454 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -188,7 +188,7 @@ impl UnionEquivalentOrderingBuilder { constants: &[ConstExpr], properties: &EquivalenceProperties, ) -> AddedOrdering { - if properties.ordering_satisfy(&ordering) { + if properties.ordering_satisfy(ordering.clone()) { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); @@ -221,7 +221,6 @@ impl UnionEquivalentOrderingBuilder { existing_ordering, &properties.constants, ) { - debug_assert!(properties.ordering_satisfy(&augmented_ordering)); self.orderings.push(augmented_ordering); result = true; } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 6b09e5274f1c9..4a5d0de2ed647 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -554,7 +554,7 @@ fn analyze_immediate_sort_removal( .is_none_or(|ordering| { sort_input .equivalence_properties() - .ordering_satisfy(ordering) + .ordering_satisfy(ordering.clone()) }) { return Transformed::no(node); diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 02ba93711fe63..36ab0785e8f31 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -267,7 +267,7 @@ pub fn replace_with_order_preserving_variants( alternate_plan .plan .equivalence_properties() - .ordering_satisfy(ordering) + .ordering_satisfy(ordering.clone()) }) { for child in alternate_plan.children.iter_mut() { child.data = false; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 4c7ae78d91f99..622b93cee1a2c 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -334,9 +334,10 @@ fn pushdown_requirement_to_children( } else if is_sort_preserving_merge(plan) { let new_ordering = LexOrdering::from(parent_required.lex_requirement().clone()); let mut spm_eqs = plan.equivalence_properties().clone(); + let old_ordering = spm_eqs.output_ordering().unwrap(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); - if spm_eqs.ordering_satisfy(plan.output_ordering().unwrap()) { + if spm_eqs.ordering_satisfy(old_ordering) { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. Ok(Some(vec![Some(parent_required.clone())])) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index debeacc8b86fd..f817e861c36d9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1100,42 +1100,46 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - let mut requirement = None; + let mut requirement: Option = None; for aggr_expr in aggr_exprs.iter_mut() { let Some(aggr_req) = get_aggregate_expr_req(aggr_expr, group_by, agg_mode) else { continue; }; - let mut finer = if let Some(req) = requirement.as_ref() { - eq_properties.get_finer_ordering(req, &aggr_req) + let mut finer = if let Some(req) = requirement.take() { + eq_properties.get_finer_ordering(req, aggr_req) } else { - eq_properties.normalize_sort_exprs(&aggr_req) + eq_properties.normalize_sort_exprs(aggr_req) }; - if let Some(finer_ordering) = - finer.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) - { + if let Some(finer_ordering) = finer.take() { // Requirement is satisfied by the existing ordering: - requirement = Some(finer_ordering); - continue; + if eq_properties.ordering_satisfy(finer_ordering.clone()) { + requirement = Some(finer_ordering); + continue; + } + let _ = finer.insert(finer_ordering); } let mut finer_rev = None; let mut rev_expr = aggr_expr.reverse_expr(); if let Some(reverse_aggr_expr) = rev_expr.take() { - let Some(aggr_req) = get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) else { - continue; + let Some(aggr_req) = + get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) + else { + unreachable!("Reverse aggregate expressions have ordering requirements if forward ones do"); }; - finer_rev = if let Some(req) = requirement.as_ref() { - eq_properties.get_finer_ordering(req, &aggr_req) + finer_rev = if let Some(req) = requirement.take() { + eq_properties.get_finer_ordering(req, aggr_req) } else { - eq_properties.normalize_sort_exprs(&aggr_req) + eq_properties.normalize_sort_exprs(aggr_req) }; - if let Some(finer_ordering) = - finer_rev.take_if(|o| eq_properties.ordering_satisfy(o.as_ref())) - { + if let Some(finer_ordering) = finer_rev.take() { // Reverse requirement is satisfied by the existing ordering. // Hence, we need to reverse the aggregate expression: - requirement = Some(finer_ordering); - *aggr_expr = Arc::new(reverse_aggr_expr); - continue; + if eq_properties.ordering_satisfy(finer_ordering.clone()) { + requirement = Some(finer_ordering); + *aggr_expr = Arc::new(reverse_aggr_expr); + continue; + } + let _ = finer_rev.insert(finer_ordering); } let _ = rev_expr.insert(reverse_aggr_expr); } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index c8ca61987c65f..5f0296122a60f 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1033,7 +1033,9 @@ impl SortExec { preserve_partitioning: bool, ) -> PlanProperties { // Determine execution mode: - let sort_satisfied = input.equivalence_properties().ordering_satisfy(&sort_exprs); + let sort_satisfied = input + .equivalence_properties() + .ordering_satisfy(sort_exprs.clone()); // The emission type depends on whether the input is already sorted: // - If already sorted, we can emit results in the same way as the input @@ -1163,7 +1165,7 @@ impl ExecutionPlan for SortExec { let sort_satisfied = self .input .equivalence_properties() - .ordering_satisfy(&self.expr); + .ordering_satisfy(self.expr.clone()); match (sort_satisfied, self.fetch.as_ref()) { (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 303b2b1fa4e01..0b52c36d76e26 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -373,7 +373,9 @@ pub(crate) fn window_equivalence_properties( .iter() .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))) .multi_cartesian_product() - .filter(|lex| !lex.is_empty() && window_eq_properties.ordering_satisfy(lex)) + .filter(|lex| { + !lex.is_empty() && window_eq_properties.ordering_satisfy(lex.clone()) + }) .map(LexOrdering::new) .collect::>(); // If there is a partitioning, and no possible ordering cannot satisfy @@ -469,7 +471,7 @@ pub(crate) fn window_equivalence_properties( if let Some(f) = order.first() { asc = !f.options.descending; } - window_eq_properties.ordering_satisfy(&order) + window_eq_properties.ordering_satisfy(order) }) { let increasing = set_monotonicity.eq(&SetMonotonicity::Increasing); From aba45beab97341b27741263373cf48879249324c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 01:22:35 +0300 Subject: [PATCH 070/167] bugfix --- datafusion/physical-plan/src/aggregates/mod.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f817e861c36d9..f8ea5a01d7f2a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1105,8 +1105,8 @@ pub fn get_finer_aggregate_exprs_requirement( let Some(aggr_req) = get_aggregate_expr_req(aggr_expr, group_by, agg_mode) else { continue; }; - let mut finer = if let Some(req) = requirement.take() { - eq_properties.get_finer_ordering(req, aggr_req) + let mut finer = if let Some(req) = requirement.as_ref() { + eq_properties.get_finer_ordering(req.clone(), aggr_req) } else { eq_properties.normalize_sort_exprs(aggr_req) }; @@ -1126,8 +1126,8 @@ pub fn get_finer_aggregate_exprs_requirement( else { unreachable!("Reverse aggregate expressions have ordering requirements if forward ones do"); }; - finer_rev = if let Some(req) = requirement.take() { - eq_properties.get_finer_ordering(req, aggr_req) + finer_rev = if let Some(req) = requirement.as_ref() { + eq_properties.get_finer_ordering(req.clone(), aggr_req) } else { eq_properties.normalize_sort_exprs(aggr_req) }; From 84ccfd540c1165de47879c2154e4ff96e2df1ed3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 21:15:09 +0300 Subject: [PATCH 071/167] Simplify the get_finer_aggregate_exprs_requirement function --- .../physical-expr-common/src/sort_expr.rs | 24 ++-- .../physical-expr/src/equivalence/class.rs | 20 +--- .../src/equivalence/properties/mod.rs | 25 ---- .../physical-plan/src/aggregates/mod.rs | 112 ++++++++++-------- 4 files changed, 78 insertions(+), 103 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index b2f13ff991a1e..8c25cb22107f4 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -17,6 +17,7 @@ //! Sort expressions +use std::cmp::Ordering; use std::fmt::{self, Display, Formatter}; use std::hash::{Hash, Hasher}; use std::ops::{Deref, DerefMut}; @@ -381,17 +382,6 @@ impl LexOrdering { Self { inner: orderings } } - /// Reverses each element in the ordering. For instance, `[a ASC NULLS LAST]` - /// turns into `[a DESC NULLS FIRST]`. Such reversals are useful in planning, - /// e.g. when constructing equivalent window expressions. - pub fn reverse_each(&self) -> Self { - let mut result = self.clone(); - for sort_expr in result.iter_mut() { - sort_expr.options = !sort_expr.options; - } - result - } - /// Truncates the `LexOrdering`, keeping only the first `len` elements. /// Returns `true` if truncation made a change, `false` otherwise. Negative /// cases happen in two scenarios: (1) When `len` is greater than or equal @@ -412,6 +402,18 @@ impl LexOrdering { } } +impl PartialOrd for LexOrdering { + /// There is a partial ordering among `LexOrdering` objects. For example, the + /// ordering `[a ASC]` is coarser (less) than ordering `[a ASC, b ASC]`. + /// If two orderings do not share a prefix, they are incomparable. + fn partial_cmp(&self, other: &Self) -> Option { + self.iter() + .zip(other.iter()) + .all(|(lhs, rhs)| lhs == rhs) + .then(|| self.len().cmp(&other.len())) + } +} + impl From> for LexOrdering { fn from(value: Vec) -> Self { Self::new(value) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7d3e022212851..987b02c36c94c 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -214,22 +214,13 @@ pub fn const_exprs_contains( /// /// Two `EquivalenceClass`es are equal if they contains the same expressions in /// without any ordering. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct EquivalenceClass { - /// The expressions in this equivalence class. The order doesn't - /// matter for equivalence purposes - /// + /// The expressions in this equivalence class. The order doesn't matter for + /// equivalence purposes. exprs: IndexSet>, } -impl PartialEq for EquivalenceClass { - /// Returns true if other is equal in the sense - /// of bags (multi-sets), disregarding their orderings. - fn eq(&self, other: &Self) -> bool { - self.exprs.eq(&other.exprs) - } -} - impl EquivalenceClass { /// Create a new empty equivalence class pub fn new_empty() -> Self { @@ -264,10 +255,7 @@ impl EquivalenceClass { /// Inserts all the expressions from other into this class pub fn extend(&mut self, other: Self) { - for expr in other.exprs { - // use push so entries are deduplicated - self.push(expr); - } + self.exprs.extend(other.exprs); } /// Returns true if this equivalence class contains t expression diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 963dea409aeed..9456e275a7164 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -759,31 +759,6 @@ impl EquivalenceProperties { .all(|(reference, given)| given.compatible(&reference)) } - /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking - /// any ties by choosing `lhs`. - /// - /// The finer ordering is the ordering that satisfies both of the orderings. - /// If the orderings are incomparable, returns `None`. - /// - /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is - /// the latter. - pub fn get_finer_ordering( - &self, - lhs: LexOrdering, - rhs: LexOrdering, - ) -> Option { - let Some(mut rhs) = self.normalize_sort_exprs(rhs) else { - return self.normalize_sort_exprs(lhs); - }; - let Some(mut lhs) = self.normalize_sort_exprs(lhs) else { - return Some(rhs); - }; - lhs.iter_mut() - .zip(rhs.iter_mut()) - .all(|(lhs, rhs)| lhs.expr.eq(&rhs.expr) && lhs.options == rhs.options) - .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) - } - /// Returns the finer ordering among the requirements `lhs` and `rhs`, /// breaking any ties by choosing `lhs`. /// diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f8ea5a01d7f2a..5ecfe8a359731 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1076,7 +1076,23 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { [lhs, rhs].concat() } -/// Get the common requirement that satisfies all the aggregate expressions. +// Determines if the candidate ordering is finer than the current ordering. +// Returns `None` if they are incomparable, `Some(true)` if there is no current +// ordering or candidate ordering is finer, and `Some(false)` otherwise. +fn determine_finer( + current: &Option, + candidate: &LexOrdering, +) -> Option { + if let Some(ordering) = current { + candidate.partial_cmp(ordering).map(|cmp| cmp.is_gt()) + } else { + Some(true) + } +} + +/// Gets the common requirement that satisfies all the aggregate expressions. +/// When possible, chooses the requirement that is already satisfied by the +/// equivalence properties. /// /// # Parameters /// @@ -1100,69 +1116,63 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - let mut requirement: Option = None; + let mut requirement = None; for aggr_expr in aggr_exprs.iter_mut() { - let Some(aggr_req) = get_aggregate_expr_req(aggr_expr, group_by, agg_mode) else { + let Some(aggr_req) = get_aggregate_expr_req(aggr_expr, group_by, agg_mode) + .and_then(|o| eq_properties.normalize_sort_exprs(o)) + else { + // There is no aggregate ordering requirement, or it is trivially + // satisfied -- we can skip this expression. continue; }; - let mut finer = if let Some(req) = requirement.as_ref() { - eq_properties.get_finer_ordering(req.clone(), aggr_req) - } else { - eq_properties.normalize_sort_exprs(aggr_req) - }; - if let Some(finer_ordering) = finer.take() { - // Requirement is satisfied by the existing ordering: - if eq_properties.ordering_satisfy(finer_ordering.clone()) { - requirement = Some(finer_ordering); + // If the common requirement is finer than the current expression's, + // we can skip this expression. If the latter is finer than the former, + // adopt it if it is satisfied by the equivalence properties. Otherwise, + // defer the analysis to the reverse expression. + let forward_finer = determine_finer(&requirement, &aggr_req); + if let Some(finer) = forward_finer { + if !finer { + continue; + } else if eq_properties.ordering_satisfy(aggr_req.clone()) { + requirement = Some(aggr_req); continue; } - let _ = finer.insert(finer_ordering); } - let mut finer_rev = None; - let mut rev_expr = aggr_expr.reverse_expr(); - if let Some(reverse_aggr_expr) = rev_expr.take() { - let Some(aggr_req) = + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + let Some(rev_aggr_req) = get_aggregate_expr_req(&reverse_aggr_expr, group_by, agg_mode) + .and_then(|o| eq_properties.normalize_sort_exprs(o)) else { - unreachable!("Reverse aggregate expressions have ordering requirements if forward ones do"); - }; - finer_rev = if let Some(req) = requirement.as_ref() { - eq_properties.get_finer_ordering(req.clone(), aggr_req) - } else { - eq_properties.normalize_sort_exprs(aggr_req) + // The reverse requirement is trivially satisfied -- just reverse + // the expression and continue with the next one: + *aggr_expr = Arc::new(reverse_aggr_expr); + continue; }; - if let Some(finer_ordering) = finer_rev.take() { - // Reverse requirement is satisfied by the existing ordering. - // Hence, we need to reverse the aggregate expression: - if eq_properties.ordering_satisfy(finer_ordering.clone()) { - requirement = Some(finer_ordering); + // If the common requirement is finer than the reverse expression's, + // just reverse it and continue the loop with the next aggregate + // expression. If the latter is finer than the former, adopt it if + // it is satisfied by the equivalence properties. Otherwise, adopt + // the forward expression. + if let Some(finer) = determine_finer(&requirement, &rev_aggr_req) { + if !finer { *aggr_expr = Arc::new(reverse_aggr_expr); - continue; + } else if eq_properties.ordering_satisfy(rev_aggr_req.clone()) { + *aggr_expr = Arc::new(reverse_aggr_expr); + requirement = Some(rev_aggr_req); + } else { + requirement = Some(aggr_req); } - let _ = finer_rev.insert(finer_ordering); - } - let _ = rev_expr.insert(reverse_aggr_expr); - } - if finer.is_some() { - // There is a requirement that satisfies both the existing requirement - // and the aggregate requirement. Use updated requirement: - requirement = finer; - continue; - } else if let Some(reverse_aggr_expr) = rev_expr { - if finer_rev.is_some() { - // There is a requirement that satisfies both the existing requirement - // and the reverse aggregate requirement. Use updated requirement: - *aggr_expr = Arc::new(reverse_aggr_expr); - requirement = finer_rev; - continue; + } else if forward_finer.is_some() { + requirement = Some(aggr_req); + } else { + // Neither the existing requirement nor the current aggregate + // requirement satisfy the other (forward or reverse), this + // means they are conflicting. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); } } - // Neither the existing requirement nor the aggregate requirement - // satisfy the other, this means requirements are conflicting. - // Currently, we do not support conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); } Ok(requirement.map_or_else(Vec::new, |o| o.into_iter().map(Into::into).collect())) From eb0773975dfe7ad635e7b78b1a3d0a33e5607282 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 22:43:23 +0300 Subject: [PATCH 072/167] Simplify the function with_reorder --- .../physical-expr/src/equivalence/ordering.rs | 8 +++--- .../src/equivalence/properties/mod.rs | 25 ++++++++----------- 2 files changed, 14 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 8e7c6be905aea..557d6db8fb875 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -77,7 +77,7 @@ impl OrderingEquivalenceClass { /// Checks whether this ordering equivalence class is empty. pub fn is_empty(&self) -> bool { - self.len() == 0 + self.orderings.is_empty() } /// Returns an iterator over the equivalent orderings in this class. @@ -201,9 +201,9 @@ impl OrderingEquivalenceClass { .collect(); // Suffix orderings of other to the current orderings. for (outer_idx, ordering) in other.iter().enumerate() { - for idx in 0..n_ordering { - // Calculate cross product index - let idx = outer_idx * n_ordering + idx; + let base = outer_idx * n_ordering; + // Use the cross product index: + for idx in base..(base + n_ordering) { self.orderings[idx].extend(ordering.iter().cloned()); } } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 9456e275a7164..33fd2b84d7d7a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -425,24 +425,19 @@ impl EquivalenceProperties { .filter(|expr| !self.is_expr_constant(&expr.expr)) .collect::>(); - if filtered_exprs.is_empty() { - return self; - } - - let filtered_exprs = LexOrdering::new(filtered_exprs); - let mut new_orderings = vec![filtered_exprs.clone()]; + if !filtered_exprs.is_empty() { + let filtered_exprs = LexOrdering::new(filtered_exprs); + // Preserve valid suffixes from existing orderings: + let oeq_class = mem::take(&mut self.oeq_class); + let mut new_orderings = oeq_class + .into_iter() + .filter(|existing| self.is_prefix_of(&filtered_exprs, existing)) + .collect::>(); + new_orderings.push(filtered_exprs); - // Preserve valid suffixes from existing orderings - let oeq_class = mem::take(&mut self.oeq_class); - for existing in oeq_class { - if self.is_prefix_of(&filtered_exprs, &existing) { - let mut extended = filtered_exprs.clone(); - extended.extend(existing.into_iter().skip(filtered_exprs.len())); - new_orderings.push(extended); - } + self.oeq_class = OrderingEquivalenceClass::new(new_orderings); } - self.oeq_class = OrderingEquivalenceClass::new(new_orderings); self } From 0c9c93f4851d260a1bb5fbf466212f49712727ec Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 7 Apr 2025 23:19:13 +0300 Subject: [PATCH 073/167] Fix with_reorder bug --- .../src/equivalence/properties/mod.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 33fd2b84d7d7a..587eecd8b9b4e 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -415,12 +415,12 @@ impl EquivalenceProperties { } /// Updates the ordering equivalence group within assuming that the table - /// is re-sorted according to the argument `sort_exprs`. Note that constants + /// is re-sorted according to the argument `ordering`. Note that constants /// and equivalence classes are unchanged as they are unaffected by a re-sort. /// If the given ordering is already satisfied, the function does nothing. - pub fn with_reorder(mut self, sort_exprs: LexOrdering) -> Self { + pub fn with_reorder(mut self, ordering: LexOrdering) -> Self { // Filter out constant expressions as they don't affect ordering - let filtered_exprs = sort_exprs + let filtered_exprs = ordering .into_iter() .filter(|expr| !self.is_expr_constant(&expr.expr)) .collect::>(); @@ -432,12 +432,18 @@ impl EquivalenceProperties { let mut new_orderings = oeq_class .into_iter() .filter(|existing| self.is_prefix_of(&filtered_exprs, existing)) + .map(|existing| { + filtered_exprs + .clone() + .into_iter() + .chain(existing.into_iter().skip(filtered_exprs.len())) + .collect() + }) .collect::>(); new_orderings.push(filtered_exprs); self.oeq_class = OrderingEquivalenceClass::new(new_orderings); } - self } From 2fa9da8f8c165b4d8cc4a091e12917384d54f268 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 00:15:09 +0300 Subject: [PATCH 074/167] Simplify the function with_reorder (Part 2) --- .../src/equivalence/properties/dependency.rs | 4 ++-- .../physical-expr/src/equivalence/properties/mod.rs | 11 +++-------- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 2ea09fa7a08ac..c488522b37214 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1515,13 +1515,13 @@ mod tests { let result = eq_properties.with_reorder(new_order); - // Should only contain [b ASC, c ASC] + // Should only contain [a/b ASC, c ASC] assert_eq!(result.oeq_class().len(), 1); // Verify orderings let ordering = result.oeq_class().iter().next().unwrap(); assert_eq!(ordering.len(), 2); - assert!(ordering[0].expr.eq(&col_b)); + assert!(ordering[0].expr.eq(&col_a) || ordering[0].expr.eq(&col_b)); assert!(ordering[0].options.eq(&asc)); assert!(ordering[1].expr.eq(&col_c)); assert!(ordering[1].options.eq(&asc)); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 587eecd8b9b4e..28641a12333bc 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -432,15 +432,10 @@ impl EquivalenceProperties { let mut new_orderings = oeq_class .into_iter() .filter(|existing| self.is_prefix_of(&filtered_exprs, existing)) - .map(|existing| { - filtered_exprs - .clone() - .into_iter() - .chain(existing.into_iter().skip(filtered_exprs.len())) - .collect() - }) .collect::>(); - new_orderings.push(filtered_exprs); + if new_orderings.is_empty() { + new_orderings.push(filtered_exprs); + } self.oeq_class = OrderingEquivalenceClass::new(new_orderings); } From 06f9ef0e1cef8c8cd962409812dbf34d3f7c3dc0 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 01:58:00 +0300 Subject: [PATCH 075/167] Simplify --- .../src/equivalence/properties/mod.rs | 49 ++++++++----------- 1 file changed, 21 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 28641a12333bc..7eadb24409a84 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -313,12 +313,6 @@ impl EquivalenceProperties { Ok(()) } - /// Remove the specified constant - pub fn remove_constant(mut self, c: &ConstExpr) -> Self { - self.constants.retain(|existing| existing != c); - self - } - /// Track/register physical expressions with constant values. pub fn with_constants( mut self, @@ -436,7 +430,6 @@ impl EquivalenceProperties { if new_orderings.is_empty() { new_orderings.push(filtered_exprs); } - self.oeq_class = OrderingEquivalenceClass::new(new_orderings); } self @@ -475,12 +468,11 @@ impl EquivalenceProperties { &self, sort_exprs: impl IntoIterator, ) -> Option { - let normalized_constants = self.normalized_constant_exprs(); // Prune redundant sections in the ordering: let sort_exprs = sort_exprs .into_iter() .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) - .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) + .filter(|order| !self.is_normalized_expr_constant(&order.expr)) .collect::>(); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) } @@ -503,12 +495,11 @@ impl EquivalenceProperties { &self, sort_reqs: impl IntoIterator, ) -> Option { - let normalized_constants = self.normalized_constant_exprs(); // Prune redundant sections in the requirement: let reqs = sort_reqs .into_iter() .map(|req| self.eq_group.normalize_sort_requirement(req.clone())) - .filter(|order| !physical_exprs_contains(&normalized_constants, &order.expr)) + .filter(|order| !self.is_normalized_expr_constant(&order.expr)) .collect::>(); (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) } @@ -1140,16 +1131,6 @@ impl EquivalenceProperties { self.constraints.project(&indices) } - /// Returns normalized versions of the expressions defining constants. - /// Normalization removes duplicates and standardizes expressions - /// according to the equivalence group within. - fn normalized_constant_exprs(&self) -> Vec> { - self.constants - .iter() - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) - .collect() - } - /// Projects the equivalences within according to `mapping` /// and `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { @@ -1248,7 +1229,9 @@ impl EquivalenceProperties { } /// This function determines whether the provided expression is constant - /// based on the known constants. + /// based on the known constants. For example, if columns `a` and `b` are + /// constant, then expressions `a`, `b` and `a + b` will all return `true` + /// whereas expression `c` will return `false`. /// /// # Parameters /// @@ -1260,12 +1243,22 @@ impl EquivalenceProperties { /// Returns `true` if the expression is constant according to equivalence /// group, `false` otherwise. pub fn is_expr_constant(&self, expr: &Arc) -> bool { - // As an example, assume that we know columns `a` and `b` are constant. - // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will - // return `false`. - let normalized_constants = self.normalized_constant_exprs(); let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - is_constant_recurse(&normalized_constants, &normalized_expr) + self.is_normalized_expr_constant(&normalized_expr) + } + + /// Helper of the [`Self::is_expr_constant`] function, assumes that the + /// given expression is normalized. + fn is_normalized_expr_constant( + &self, + normalized_expr: &Arc, + ) -> bool { + let normalized_constants = self + .constants + .iter() + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .collect::>(); + is_constant_recurse(&normalized_constants, normalized_expr) } /// This function determines whether the provided expression is constant @@ -1495,7 +1488,7 @@ fn update_properties( .eq_group .normalize_expr(Arc::clone(&node.expr)); let oeq_class = eq_properties.normalized_oeq_class(); - if eq_properties.is_expr_constant(&normalized_expr) + if eq_properties.is_normalized_expr_constant(&normalized_expr) || oeq_class.is_expr_partial_const(&normalized_expr) { node.data.sort_properties = SortProperties::Singleton; From 65b546b7e11f97eeb55446890fbff955859a3ba2 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 11:34:42 +0300 Subject: [PATCH 076/167] DRY --- .../src/equivalence/properties/mod.rs | 35 ++++++++----------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 7eadb24409a84..91a2513e2749a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -472,7 +472,7 @@ impl EquivalenceProperties { let sort_exprs = sort_exprs .into_iter() .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) - .filter(|order| !self.is_normalized_expr_constant(&order.expr)) + .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)) .collect::>(); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) } @@ -499,7 +499,7 @@ impl EquivalenceProperties { let reqs = sort_reqs .into_iter() .map(|req| self.eq_group.normalize_sort_requirement(req.clone())) - .filter(|order| !self.is_normalized_expr_constant(&order.expr)) + .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)) .collect::>(); (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) } @@ -1244,7 +1244,7 @@ impl EquivalenceProperties { /// group, `false` otherwise. pub fn is_expr_constant(&self, expr: &Arc) -> bool { let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - self.is_normalized_expr_constant(&normalized_expr) + self.is_normalized_expr_constant(&normalized_expr, false) } /// Helper of the [`Self::is_expr_constant`] function, assumes that the @@ -1252,17 +1252,26 @@ impl EquivalenceProperties { fn is_normalized_expr_constant( &self, normalized_expr: &Arc, + across_partitions: bool, ) -> bool { let normalized_constants = self .constants .iter() + .filter(|const_expr| { + !across_partitions + || matches!( + const_expr.across_partitions(), + AcrossPartitions::Uniform { .. } + ) + }) .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) .collect::>(); is_constant_recurse(&normalized_constants, normalized_expr) } /// This function determines whether the provided expression is constant - /// across partitions based on the known constants. + /// across partitions based on the known constants. For more details, see + /// [`Self::is_expr_constant`]. /// /// # Parameters /// @@ -1277,22 +1286,8 @@ impl EquivalenceProperties { &self, expr: &Arc, ) -> bool { - // As an example, assume that we know columns `a` and `b` are constant. - // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will - // return `false`. - let normalized_constants = self - .constants - .iter() - .filter(|const_expr| { - matches!( - const_expr.across_partitions(), - AcrossPartitions::Uniform { .. } - ) - }) - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) - .collect::>(); let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - is_constant_recurse(&normalized_constants, &normalized_expr) + self.is_normalized_expr_constant(&normalized_expr, true) } /// Retrieves the constant value of a given physical expression, if it exists. @@ -1488,7 +1483,7 @@ fn update_properties( .eq_group .normalize_expr(Arc::clone(&node.expr)); let oeq_class = eq_properties.normalized_oeq_class(); - if eq_properties.is_normalized_expr_constant(&normalized_expr) + if eq_properties.is_normalized_expr_constant(&normalized_expr, false) || oeq_class.is_expr_partial_const(&normalized_expr) { node.data.sort_properties = SortProperties::Singleton; From 25175901f49ea139be6db5d0b56f4f6e972f2557 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 19:40:23 +0300 Subject: [PATCH 077/167] Simplifications --- .../src/equivalence/properties/mod.rs | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 91a2513e2749a..cd59f7d0dfb91 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -212,9 +212,8 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { // Prune out constant expressions: - let constants = self.constants(); let mut sort_exprs = self.oeq_class().output_ordering()?.take(); - sort_exprs.retain(|item| !const_exprs_contains(constants, &item.expr)); + sort_exprs.retain(|item| !const_exprs_contains(&self.constants, &item.expr)); (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs)) } @@ -1090,17 +1089,13 @@ impl EquivalenceProperties { if self.is_expr_constant(source) && !const_exprs_contains(&projected_constants, target) { - if self.is_expr_constant_across_partitions(source) { - projected_constants.push( - ConstExpr::from(target) - .with_across_partitions(self.get_expr_constant_value(source)), - ) + let uniform = if self.is_expr_constant_across_partitions(source) { + self.get_expr_constant_value(source) } else { - projected_constants.push( - ConstExpr::from(target) - .with_across_partitions(AcrossPartitions::Heterogeneous), - ) - } + AcrossPartitions::Heterogeneous + }; + let const_expr = ConstExpr::from(target).with_across_partitions(uniform); + projected_constants.push(const_expr); } } projected_constants From 842b7d5baf9bce2584f9e77bbaf640fff22d564c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 21:15:19 +0300 Subject: [PATCH 078/167] Improve add_equal_condition --- .../physical-expr/src/equivalence/class.rs | 46 +++++++------------ 1 file changed, 17 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 987b02c36c94c..873d38ee2b229 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -55,18 +55,6 @@ use indexmap::{IndexMap, IndexSet}; /// // create a constant expression from a physical expression /// let const_expr = ConstExpr::from(col); /// ``` -// TODO: Consider refactoring the `across_partitions` and `value` fields into an enum: -// -// ``` -// enum PartitionValues { -// Uniform(Option), // Same value across all partitions -// Heterogeneous(Vec>) // Different values per partition -// } -// ``` -// -// This would provide more flexible representation of partition values. -// Note: This is a breaking change for the equivalence API and should be -// addressed in a separate issue/PR. #[derive(Debug, Clone)] pub struct ConstExpr { /// The expression that is known to be constant (e.g. a `Column`) @@ -76,15 +64,13 @@ pub struct ConstExpr { across_partitions: AcrossPartitions, } +/// Represents whether a constant expression's value is uniform or varies across +/// partitions. Has two variants: +/// - `Heterogeneous`: The constant expression may have different values for +/// different partitions. +/// - `Uniform(Option)`: The constant expression has the same value +/// across all partitions, or is `None` if the value is unknown. #[derive(PartialEq, Clone, Debug)] -/// Represents whether a constant expression's value is uniform or varies across partitions. -/// -/// The `AcrossPartitions` enum is used to describe the nature of a constant expression -/// in a physical execution plan: -/// -/// - `Heterogeneous`: The constant expression may have different values for different partitions. -/// - `Uniform(Option)`: The constant expression has the same value across all partitions, -/// or is `None` if the value is not specified. pub enum AcrossPartitions { Heterogeneous, Uniform(Option), @@ -138,8 +124,7 @@ impl ConstExpr { where F: Fn(&Arc) -> Option>, { - let maybe_expr = f(&self.expr); - maybe_expr.map(|expr| Self { + f(&self.expr).map(|expr| Self { expr, across_partitions: self.across_partitions.clone(), }) @@ -328,7 +313,7 @@ impl EquivalenceGroup { /// Checks whether this equivalence group is empty. pub fn is_empty(&self) -> bool { - self.len() == 0 + self.classes.is_empty() } /// Returns an iterator over the equivalence classes in this group. @@ -344,15 +329,19 @@ impl EquivalenceGroup { left: &Arc, right: &Arc, ) { + let mut idx = 0; + let size = self.classes.len(); let mut first_class = None; let mut second_class = None; - for (idx, cls) in self.classes.iter().enumerate() { - if cls.contains(left) { + while (idx < size) && (first_class.is_none() || second_class.is_none()) { + let cls = &self.classes[idx]; + if first_class.is_none() && cls.contains(left) { first_class = Some(idx); } - if cls.contains(right) { + if second_class.is_none() && cls.contains(right) { second_class = Some(idx); } + idx += 1; } match (first_class, second_class) { (Some(mut first_idx), Some(mut second_idx)) => { @@ -594,9 +583,8 @@ impl EquivalenceGroup { let new_column = Arc::new(Column::new( column.name(), column.index() + left_size, - )) - as _; - return Ok(Transformed::yes(new_column)); + )); + return Ok(Transformed::yes(new_column as _)); } Ok(Transformed::no(expr)) From 16bc51540fd907c7b41cf04d9cb13c15ed050e8a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 8 Apr 2025 22:22:46 +0300 Subject: [PATCH 079/167] Improve docs --- .../src/equivalence/properties/mod.rs | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index cd59f7d0dfb91..4d22d875924a1 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -52,14 +52,12 @@ use datafusion_physical_expr_common::utils::ExprPropertiesNode; use indexmap::IndexSet; use itertools::Itertools; -/// `EquivalenceProperties` stores information about the output -/// of a plan node, that can be used to optimize the plan. -/// -/// Currently, it keeps track of: -/// - Sort expressions (orderings) -/// - Equivalent expressions: expressions that are known to have same value. -/// - Constants expressions: expressions that are known to contain a single -/// constant value. +/// `EquivalenceProperties` stores information about the output of a plan node +/// that can be used to optimize the plan. Currently, it keeps track of: +/// - Sort expressions (orderings), +/// - Equivalent expressions; i.e. expressions known to have the same value. +/// - Constants expressions; i.e. expressions known to contain a single constant +/// value. /// /// Please see the [Using Ordering for Better Plans] blog for more details. /// @@ -81,8 +79,8 @@ use itertools::Itertools; /// ``` /// /// In this case, both `a ASC` and `b DESC` can describe the table ordering. -/// `EquivalenceProperties`, tracks these different valid sort expressions and -/// treat `a ASC` and `b DESC` on an equal footing. For example if the query +/// `EquivalenceProperties` tracks these different valid sort expressions and +/// treat `a ASC` and `b DESC` on an equal footing. For example, if the query /// specifies the output sorted by EITHER `a ASC` or `b DESC`, the sort can be /// avoided. /// @@ -101,12 +99,11 @@ use itertools::Itertools; /// └---┴---┘ /// ``` /// -/// In this case, columns `a` and `b` always have the same value, which can of -/// such equivalences inside this object. With this information, Datafusion can -/// optimize operations such as. For example, if the partition requirement is -/// `Hash(a)` and output partitioning is `Hash(b)`, then DataFusion avoids -/// repartitioning the data as the existing partitioning satisfies the -/// requirement. +/// In this case, columns `a` and `b` always have the same value. With this +/// information, Datafusion can optimize various operations. For example, if +/// the partition requirement is `Hash(a)` and output partitioning is +/// `Hash(b)`, then DataFusion avoids repartitioning the data as the existing +/// partitioning satisfies the requirement. /// /// # Code Example /// ``` @@ -200,11 +197,12 @@ impl EquivalenceProperties { &self.eq_group } - /// Returns a reference to the constant expressions + /// Returns a reference to the constants within. pub fn constants(&self) -> &[ConstExpr] { &self.constants } + /// Returns a reference to the constraints within. pub fn constraints(&self) -> &Constraints { &self.constraints } From ddf8ef92a5e96cacb851bfc121a2316e7d4623b4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 10 Apr 2025 10:27:51 +0300 Subject: [PATCH 080/167] Simplifications --- .../common/src/functional_dependencies.rs | 2 +- .../physical-expr-common/src/sort_expr.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 51 +++++++++++-------- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../src/equivalence/properties/mod.rs | 21 ++++---- .../src/equivalence/properties/union.rs | 26 ++++------ 6 files changed, 50 insertions(+), 54 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 5f262d634af37..0aee9fa12266a 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -99,7 +99,7 @@ impl Default for Constraints { impl IntoIterator for Constraints { type Item = Constraint; - type IntoIter = IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.inner.into_iter() diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 8c25cb22107f4..129b10bdcdcf4 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -457,7 +457,7 @@ impl FromIterator for LexOrdering { impl IntoIterator for LexOrdering { type Item = PhysicalSortExpr; - type IntoIter = IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.inner.into_iter() diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 873d38ee2b229..2d5610184e9d0 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -58,7 +58,7 @@ use indexmap::{IndexMap, IndexSet}; #[derive(Debug, Clone)] pub struct ConstExpr { /// The expression that is known to be constant (e.g. a `Column`) - expr: Arc, + pub(crate) expr: Arc, /// Does the constant have the same value across all partitions? See /// struct docs for more details across_partitions: AcrossPartitions, @@ -116,10 +116,6 @@ impl ConstExpr { &self.expr } - pub fn owned_expr(self) -> Arc { - self.expr - } - pub fn map(&self, f: F) -> Option where F: Fn(&Arc) -> Option>, @@ -207,63 +203,59 @@ pub struct EquivalenceClass { } impl EquivalenceClass { - /// Create a new empty equivalence class + /// Create a new empty equivalence class. pub fn new_empty() -> Self { Self { exprs: IndexSet::new(), } } - // Create a new equivalence class from a pre-existing `Vec` + // Create a new equivalence class from a pre-existing `Vec`. pub fn new(exprs: Vec>) -> Self { Self { exprs: exprs.into_iter().collect(), } } - /// Return the inner vector of expressions - pub fn into_vec(self) -> Vec> { - self.exprs.into_iter().collect() - } - /// Return the "canonical" expression for this class (the first element) - /// if any + /// if non-empty. fn canonical_expr(&self) -> Option> { self.exprs.iter().next().cloned() } /// Insert the expression into this class, meaning it is known to be equal to - /// all other expressions in this class + /// all other expressions in this class. pub fn push(&mut self, expr: Arc) { self.exprs.insert(expr); } - /// Inserts all the expressions from other into this class + /// Inserts all the expressions from other into this class. pub fn extend(&mut self, other: Self) { self.exprs.extend(other.exprs); } - /// Returns true if this equivalence class contains t expression + /// Returns true if this equivalence class contains the given expression. pub fn contains(&self, expr: &Arc) -> bool { self.exprs.contains(expr) } - /// Returns true if this equivalence class has any entries in common with `other` + /// Returns true if this equivalence class has any entries in common with + /// `other`. pub fn contains_any(&self, other: &Self) -> bool { - self.exprs.iter().any(|e| other.contains(e)) + self.exprs.intersection(&other.exprs).next().is_some() } - /// return the number of items in this class + /// Returns the number of items in this equivalence class. pub fn len(&self) -> usize { self.exprs.len() } - /// return true if this class is empty + /// Returns whether this equivalence class is empty. pub fn is_empty(&self) -> bool { self.exprs.is_empty() } - /// Iterate over all elements in this class, in some arbitrary order + /// Iterate over all elements in this class (in some arbitrary order). pub fn iter(&self) -> impl Iterator> { self.exprs.iter() } @@ -281,12 +273,27 @@ impl EquivalenceClass { } } +impl IntoIterator for EquivalenceClass { + type Item = Arc; + type IntoIter = as IntoIterator>::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.exprs.into_iter() + } +} + impl Display for EquivalenceClass { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "[{}]", format_physical_expr_list(&self.exprs)) } } +impl From for Vec> { + fn from(cls: EquivalenceClass) -> Self { + cls.exprs.into_iter().collect() + } +} + /// A collection of distinct `EquivalenceClass`es #[derive(Debug, Clone)] pub struct EquivalenceGroup { @@ -662,7 +669,7 @@ impl EquivalenceGroup { impl IntoIterator for EquivalenceGroup { type Item = EquivalenceClass; - type IntoIter = IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.classes.into_iter() diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 557d6db8fb875..e0c496bdef3bf 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -312,7 +312,7 @@ impl OrderingEquivalenceClass { /// Convert the `OrderingEquivalenceClass` into an iterator of LexOrderings impl IntoIterator for OrderingEquivalenceClass { type Item = LexOrdering; - type IntoIter = IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.orderings.into_iter() diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 4d22d875924a1..9ba11865718ce 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -137,7 +137,8 @@ pub struct EquivalenceProperties { eq_group: EquivalenceGroup, /// Equivalent sort expressions oeq_class: OrderingEquivalenceClass, - /// Expressions whose values are constant + /// Expressions whose values are constant. These expressions are in + /// normalized form (w.r.t. the equivalence group). /// /// TODO: We do not need to track constants separately, they can be tracked /// inside `eq_group` as `Literal` expressions. @@ -319,8 +320,7 @@ impl EquivalenceProperties { .into_iter() .filter_map(|c| { let across_partitions = c.across_partitions(); - let expr = c.owned_expr(); - let normalized_expr = self.eq_group.normalize_expr(expr); + let normalized_expr = self.eq_group.normalize_expr(c.expr); (!const_exprs_contains(&self.constants, &normalized_expr)).then(|| { let const_expr = ConstExpr::from(normalized_expr); const_expr.with_across_partitions(across_partitions) @@ -351,7 +351,7 @@ impl EquivalenceProperties { .eq_group .iter() .find(|class| class.contains(&normalized_expr)) - .map(|class| class.clone().into_vec()) + .map(|class| class.clone().into()) .unwrap_or_else(|| vec![Arc::clone(&normalized_expr)]); let mut new_orderings = vec![]; @@ -1257,7 +1257,7 @@ impl EquivalenceProperties { AcrossPartitions::Uniform { .. } ) }) - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(const_expr.expr()))) + .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(&const_expr.expr))) .collect::>(); is_constant_recurse(&normalized_constants, normalized_expr) } @@ -1307,7 +1307,7 @@ impl EquivalenceProperties { } for const_expr in self.constants.iter() { - if normalized_expr.eq(const_expr.expr()) { + if normalized_expr.eq(&const_expr.expr) { return const_expr.across_partitions(); } } @@ -1365,11 +1365,9 @@ impl EquivalenceProperties { let new_constants = self .constants .into_iter() - .map(|const_expr| { - let across_partitions = const_expr.across_partitions(); - let new_const_expr = with_new_schema(const_expr.owned_expr(), &schema)?; - Ok(ConstExpr::new(new_const_expr) - .with_across_partitions(across_partitions)) + .map(|mut const_expr| { + const_expr.expr = with_new_schema(const_expr.expr, &schema)?; + Ok(const_expr) }) .collect::>>()?; @@ -1390,7 +1388,6 @@ impl EquivalenceProperties { let mut eq_classes = vec![]; for eq_class in self.eq_group { let new_eq_exprs = eq_class - .into_vec() .into_iter() .map(|expr| with_new_schema(expr, &schema)) .collect::>()?; diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index b2327044a6454..9e1c50c2f44b7 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -55,23 +55,15 @@ fn calculate_union_binary( // Find matching constant expression in RHS rhs.constants() .iter() - .find(|rhs_const| rhs_const.expr().eq(lhs_const.expr())) + .find(|rhs_const| rhs_const.expr.eq(&lhs_const.expr)) .map(|rhs_const| { - let mut const_expr = ConstExpr::new(Arc::clone(lhs_const.expr())); - - // If both sides have matching constant values, preserve the value and set across_partitions=true - if let ( - AcrossPartitions::Uniform(Some(lhs_val)), - AcrossPartitions::Uniform(Some(rhs_val)), - ) = (lhs_const.across_partitions(), rhs_const.across_partitions()) - { - if lhs_val == rhs_val { - const_expr = const_expr.with_across_partitions( - AcrossPartitions::Uniform(Some(lhs_val)), - ) - } + let const_expr = lhs_const.clone(); + // If both sides have matching constant values, preserve the value: + if lhs_const.across_partitions() == rhs_const.across_partitions() { + const_expr + } else { + const_expr.with_across_partitions(AcrossPartitions::Heterogeneous) } - const_expr }) }) .collect::>(); @@ -301,8 +293,8 @@ fn advance_if_matches_constant( constants: &[ConstExpr], ) -> Option { let expr = iter.peek()?; - let const_expr = constants.iter().find(|c| expr.expr.eq(c.expr()))?; - let found_expr = PhysicalSortExpr::new(Arc::clone(const_expr.expr()), expr.options); + let const_expr = constants.iter().find(|c| expr.expr.eq(&c.expr))?; + let found_expr = PhysicalSortExpr::new(Arc::clone(&const_expr.expr), expr.options); iter.next(); Some(found_expr) } From d1173aacb3761e893ab3a8d9ac9418c05fdf6b5a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 11 Apr 2025 11:40:24 +0300 Subject: [PATCH 081/167] Simplifications --- .../src/equivalence/properties/mod.rs | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 794a337e1d0ca..1298ee7ff8761 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -513,7 +513,7 @@ impl EquivalenceProperties { return true; }; let length = normalized_ordering.len(); - self.compute_common_sort_prefix_length(&normalized_ordering) == length + self.common_sort_prefix_length(normalized_ordering) == length } /// Checks whether the given sort requirements are satisfied by any of the @@ -565,27 +565,25 @@ impl EquivalenceProperties { // From the analysis above, we know that `[a ASC]` is satisfied. Then, // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - eq_properties = eq_properties - .with_constants(std::iter::once(ConstExpr::from(element.expr))); + let const_expr = ConstExpr::from(element.expr); + eq_properties = eq_properties.with_constants(std::iter::once(const_expr)); } true } /// Returns the number of consecutive sort expressions (starting from the /// left) that are satisfied by the existing ordering. - fn compute_common_sort_prefix_length( - &self, - normalized_ordering: &LexOrdering, - ) -> usize { + fn common_sort_prefix_length(&self, normalized_ordering: LexOrdering) -> usize { + let full_length = normalized_ordering.len(); // Check whether the given ordering is satisfied by constraints: - if self.satisfied_by_constraints_ordering(normalized_ordering) { + if self.satisfied_by_constraints_ordering(&normalized_ordering) { // If constraints satisfy all sort expressions, return the full // length: - return normalized_ordering.len(); + return full_length; } let schema = self.schema(); let mut eq_properties = self.clone(); - for (idx, element) in normalized_ordering.iter().enumerate() { + for (idx, element) in normalized_ordering.into_iter().enumerate() { // Check whether given ordering is satisfied: let ExprProperties { sort_properties, .. @@ -596,7 +594,7 @@ impl EquivalenceProperties { expr: Arc::clone(&element.expr), options, }; - sort_expr.satisfy_expr(element, schema) + sort_expr.satisfy_expr(&element, schema) } // Singleton expressions satisfies any ordering. SortProperties::Singleton => true, @@ -619,12 +617,12 @@ impl EquivalenceProperties { // From the analysis above, we know that `[a ASC]` is satisfied. Then, // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - let const_expr = ConstExpr::from(Arc::clone(&element.expr)); + let const_expr = ConstExpr::from(element.expr); eq_properties = eq_properties.with_constants(std::iter::once(const_expr)); } - // All sort expressions are satisfied: - normalized_ordering.len() + // All sort expressions are satisfied, return full length: + full_length } /// Determines the longest normalized prefix of `ordering` satisfied by the @@ -639,12 +637,13 @@ impl EquivalenceProperties { // If the ordering vanishes after normalization, it is satisfied: return (vec![], true); }; - let prefix_len = self.compute_common_sort_prefix_length(&normalized_ordering); - if prefix_len == normalized_ordering.len() { - (normalized_ordering.take(), true) - } else { - (normalized_ordering[..prefix_len].to_vec(), false) + let prefix_len = self.common_sort_prefix_length(normalized_ordering.clone()); + let flag = prefix_len == normalized_ordering.len(); + let mut sort_exprs = normalized_ordering.take(); + if !flag { + sort_exprs.truncate(prefix_len); } + (sort_exprs, flag) } /// Checks if the sort expressions are satisfied by any of the table From 32d44318f9f5d569cf6d35796989ec9a62bbf094 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 13 Apr 2025 22:57:21 +0300 Subject: [PATCH 082/167] RequiredInputOrdering -> OrderingAlternatives --- .../enforce_distribution.rs | 19 +- .../physical_optimizer/enforce_sorting.rs | 19 +- .../physical_optimizer/projection_pushdown.rs | 24 +- .../tests/physical_optimizer/test_utils.rs | 13 +- datafusion/datasource/src/sink.rs | 13 +- .../physical-expr-common/src/sort_expr.rs | 65 ++++ .../physical-expr/src/equivalence/ordering.rs | 2 +- .../src/equivalence/properties/dependency.rs | 81 +--- .../src/equivalence/properties/mod.rs | 179 ++++----- .../physical-expr/src/expressions/cast.rs | 6 +- datafusion/physical-expr/src/lib.rs | 3 +- .../src/enforce_distribution.rs | 7 +- .../src/enforce_sorting/mod.rs | 9 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/enforce_sorting/sort_pushdown.rs | 355 +++++++++--------- .../src/output_requirements.rs | 24 +- .../physical-optimizer/src/sanity_checker.rs | 5 +- .../src/update_aggr_exprs.rs | 4 +- datafusion/physical-optimizer/src/utils.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 21 +- .../physical-plan/src/execution_plan.rs | 63 +--- .../src/joins/sort_merge_join.rs | 15 +- .../src/joins/symmetric_hash_join.rs | 15 +- .../src/sorts/sort_preserving_merge.rs | 10 +- .../src/windows/bounded_window_agg_exec.rs | 8 +- datafusion/physical-plan/src/windows/mod.rs | 26 +- .../src/windows/window_agg_exec.rs | 8 +- 27 files changed, 444 insertions(+), 554 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 9d6e4465f130d..b967497fb246d 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -38,10 +38,10 @@ use datafusion_common::ScalarValue; use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_expr::{JoinType, Operator}; -use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr::{ - expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, +use datafusion_physical_expr::expressions::{binary, lit, BinaryExpr, Column, Literal}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, OrderingRequirements, PhysicalSortExpr, }; use datafusion_physical_optimizer::enforce_distribution::*; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; @@ -51,7 +51,7 @@ use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::execution_plan::{ExecutionPlan, RequiredInputOrdering}; +use datafusion_physical_plan::execution_plan::ExecutionPlan; use datafusion_physical_plan::expressions::col; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::JoinOn; @@ -60,10 +60,9 @@ use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; -use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_plan::PlanProperties; use datafusion_physical_plan::{ - get_plan_string, DisplayAs, DisplayFormatType, Statistics, + get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, + PlanProperties, Statistics, }; /// Models operators like BoundedWindowExec that require an input @@ -139,8 +138,8 @@ impl ExecutionPlan for SortRequiredExec { } // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec> { - vec![Some(RequiredInputOrdering::from(self.expr.clone()))] + fn required_input_ordering(&self) -> Vec> { + vec![Some(OrderingRequirements::from(self.expr.clone()))] } fn with_new_children( diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index ca04e57d9aedb..b2c3fa4b4d9c6 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -35,12 +35,18 @@ use datafusion_common::tree_node::{TreeNode, TransformedResult}; use datafusion_common::{Result, ScalarValue}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; +use datafusion_expr_common::operator::Operator; use datafusion_expr::{JoinType, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition}; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_functions_aggregate::average::avg_udaf; +use datafusion_functions_aggregate::count::count_udaf; +use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement, PhysicalSortExpr}; -use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexRequirement, PhysicalSortExpr, OrderingRequirements +}; use datafusion_physical_expr::{Distribution, Partitioning}; +use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -54,13 +60,8 @@ use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting, PlanWithCor use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{SortPushDown, assign_initial_requirements, pushdown_sorts}; use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_functions_aggregate::average::avg_udaf; -use datafusion_functions_aggregate::count::count_udaf; -use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; -use datafusion_expr_common::operator::Operator; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use rstest::rstest; @@ -998,7 +999,7 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ let output_requirements: Arc = Arc::new(OutputRequirementExec::new( bounded_window2, - Some(RequiredInputOrdering::new(LexRequirement::from( + Some(OrderingRequirements::new_single(LexRequirement::from( LexOrdering::from(sort_exprs2), ))), Distribution::SinglePartition, diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index c8c73f6239836..ce9f6dc0614ce 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -25,26 +25,27 @@ use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::physical_plan::CsvSource; use datafusion::datasource::source::DataSourceExec; use datafusion_common::config::ConfigOptions; -use datafusion_common::Result; -use datafusion_common::{JoinSide, JoinType, ScalarValue}; +use datafusion_common::{JoinSide, JoinType, Result, ScalarValue}; +use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{ Operator, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_expr_common::columnar_value::ColumnarValue; use datafusion_physical_expr::expressions::{ binary, cast, col, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, }; -use datafusion_physical_expr::ScalarFunctionExpr; -use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, +use datafusion_physical_expr::{Distribution, Partitioning, ScalarFunctionExpr}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, + PhysicalSortRequirement, }; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion_physical_plan::joins::{ @@ -55,13 +56,10 @@ use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_physical_plan::streaming::PartitionStream; -use datafusion_physical_plan::streaming::StreamingTableExec; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; -use datafusion_datasource::file_scan_config::FileScanConfigBuilder; -use datafusion_expr_common::columnar_value::ColumnarValue; use itertools::Itertools; /// Mocked UDF @@ -654,7 +652,7 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv, - Some(RequiredInputOrdering::new(LexRequirement::new(vec![ + Some(OrderingRequirements::new_single(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), options: Some(SortOptions::default()), @@ -700,7 +698,7 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = RequiredInputOrdering::new(LexRequirement::new(vec![ + let expected_reqs = OrderingRequirements::new_single(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 2)), options: Some(SortOptions::default()), diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 87839d5ce224b..56ad628c354af 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -39,9 +39,11 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::{expressions, PhysicalExpr}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::expressions::{self, col}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, OrderingRequirements, PhysicalSortExpr, +}; use datafusion_physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::{ @@ -49,7 +51,6 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; @@ -393,11 +394,11 @@ impl ExecutionPlan for RequirementsTestExec { self.input.properties() } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self .required_input_ordering .as_ref() - .map(|ordering| RequiredInputOrdering::from(ordering.clone()))] + .map(|ordering| OrderingRequirements::from(ordering.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index ffeb517f53344..d92d168f8e5f5 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -26,8 +26,8 @@ use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{Distribution, EquivalenceProperties, LexRequirement}; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::{Distribution, EquivalenceProperties}; +use datafusion_physical_expr_common::sort_expr::{LexRequirement, OrderingRequirements}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::stream::RecordBatchStreamAdapter; use datafusion_physical_plan::{ @@ -182,13 +182,10 @@ impl ExecutionPlan for DataSinkExec { vec![Distribution::SinglePartition; self.children().len()] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { // The required input ordering is set externally (e.g. by a `ListingTable`). - // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). - vec![self - .sort_order - .as_ref() - .map(|req| RequiredInputOrdering::new(req.clone()))] + // Otherwise, there is no specific requirement (i.e. `sort_order` is `None`). + vec![self.sort_order.as_ref().cloned().map(Into::into)] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 129b10bdcdcf4..2c79c9bba8cfe 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -600,3 +600,68 @@ impl From> for LexOrdering { Self::new(value.into_iter().map(Into::into).collect()) } } + +/// Represents a plan's input ordering requirements. Vector elements represent +/// alternative ordering requirements in the order of preference. +#[derive(Debug, Clone, PartialEq)] +pub enum OrderingRequirements { + /// The operator is not able to work without one of these requirements. + Hard(Vec), + /// The operator can benefit from these input orderings when available, + /// but can still work in the absence of any input ordering. + Soft(Vec), +} + +impl OrderingRequirements { + /// Creates a new instance from the given alternatives. If an empty list of + /// alternatives are given, returns `None`. + pub fn new(alternatives: Vec, soft: bool) -> Option { + (!alternatives.is_empty()).then(|| { + if soft { + Self::Soft(alternatives) + } else { + Self::Hard(alternatives) + } + }) + } + + /// Creates a new instance with a single hard requirement. + pub fn new_single(requirement: LexRequirement) -> Self { + Self::Hard(vec![requirement]) + } + + /// Adds an alternative requirement to the list of alternatives. + pub fn add_alternative(&mut self, requirement: LexRequirement) { + match self { + Self::Hard(alts) | Self::Soft(alts) => alts.push(requirement), + } + } + + /// Returns the first (i.e. most preferred) `LexRequirement` among + /// alternative requirements. + pub fn into_single(self) -> LexRequirement { + match self { + Self::Hard(mut alts) | Self::Soft(mut alts) => alts.swap_remove(0), + } + } + + /// Returns a reference to the first (i.e. most preferred) `LexRequirement` + /// among alternative requirements. + pub fn first(&self) -> &LexRequirement { + match self { + Self::Hard(alts) | Self::Soft(alts) => &alts[0], + } + } +} + +impl From for OrderingRequirements { + fn from(requirement: LexRequirement) -> Self { + Self::new_single(requirement) + } +} + +impl From for OrderingRequirements { + fn from(ordering: LexOrdering) -> Self { + Self::new_single(ordering.into()) + } +} diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index e0c496bdef3bf..3ec8459e5b469 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -118,7 +118,7 @@ impl OrderingEquivalenceClass { &mut self, ordering: impl IntoIterator, ) { - self.add_new_orderings([ordering]); + self.add_new_orderings(std::iter::once(ordering)); } /// Removes redundant orderings from this equivalence class. diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index c488522b37214..dd8ac7baee310 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -915,65 +915,6 @@ mod tests { Ok(()) } - #[test] - fn test_get_finer() -> Result<()> { - let schema = create_test_schema()?; - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let eq_properties = EquivalenceProperties::new(schema); - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. - // Third entry is the expected result. - let tests_cases = vec![ - // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] - // result should be [a Some(ASC), b Some(ASC)] - ( - vec![(col_a, Some(option_asc))], - vec![(col_a, None), (col_b, Some(option_asc))], - Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), - ), - // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] - // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] - ( - vec![ - (col_a, Some(option_asc)), - (col_b, Some(option_asc)), - (col_c, Some(option_asc)), - ], - vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], - Some(vec![ - (col_a, Some(option_asc)), - (col_b, Some(option_asc)), - (col_c, Some(option_asc)), - ]), - ), - // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] - // result should be None - ( - vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], - vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], - None, - ), - ]; - for (lhs, rhs, expected) in tests_cases { - let lhs = convert_to_sort_reqs(&lhs); - let rhs = convert_to_sort_reqs(&rhs); - let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); - let finer = eq_properties.get_finer_requirement(&lhs, &rhs); - assert_eq!(finer, expected) - } - - Ok(()) - } - #[test] fn test_normalize_sort_reqs() -> Result<()> { // Schema satisfies following properties @@ -1041,7 +982,7 @@ mod tests { let expected_normalized = convert_to_sort_reqs(&expected_normalized); assert_eq!( - eq_properties.normalize_sort_requirements(&req).unwrap(), + eq_properties.normalize_sort_requirements(req).unwrap(), expected_normalized ); } @@ -1074,8 +1015,9 @@ mod tests { for (reqs, expected) in test_cases.into_iter() { let reqs = convert_to_sort_reqs(&reqs); let expected = convert_to_sort_reqs(&expected); - - let normalized = eq_properties.normalize_sort_requirements(&reqs).unwrap(); + let normalized = eq_properties + .normalize_sort_requirements(reqs.clone()) + .unwrap(); assert!( expected.eq(&normalized), "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" @@ -1381,17 +1323,10 @@ mod tests { options: None, }]); - let res = eq_properties.requirements_compatible(&lex_a, &lex_a); - assert!(res); - - let res = eq_properties.requirements_compatible(&lex_a, &lex_a_b); - assert!(!res); - - let res = eq_properties.requirements_compatible(&lex_a_b, &lex_a); - assert!(res); - - let res = eq_properties.requirements_compatible(&lex_c, &lex_a); - assert!(!res); + assert!(eq_properties.requirements_compatible(lex_a.clone(), lex_a.clone())); + assert!(!eq_properties.requirements_compatible(lex_a.clone(), lex_a_b.clone())); + assert!(eq_properties.requirements_compatible(lex_a_b, lex_a.clone())); + assert!(!eq_properties.requirements_compatible(lex_c, lex_a)); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 1298ee7ff8761..2373244c2a0f5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -26,6 +26,7 @@ use dependency::{ pub use joins::*; pub use union::*; +use std::collections::VecDeque; use std::fmt::Display; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -488,9 +489,9 @@ impl EquivalenceProperties { /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function /// would return `[a ASC, c ASC]`. Internally, it would first normalize to /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. - fn normalize_sort_requirements<'a>( + fn normalize_sort_requirements( &self, - sort_reqs: impl IntoIterator, + sort_reqs: impl IntoIterator, ) -> Option { // Prune redundant sections in the requirement: let reqs = sort_reqs @@ -518,9 +519,9 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - pub fn ordering_satisfy_requirement<'a>( + pub fn ordering_satisfy_requirement( &self, - given: impl IntoIterator, + given: impl IntoIterator, ) -> bool { // First, standardize the given requirement: let Some(normalized_reqs) = self.normalize_sort_requirements(given) else { @@ -658,13 +659,15 @@ impl EquivalenceProperties { self.constraints.iter().any(|constraint| match constraint { Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { let check_null = matches!(constraint, Constraint::Unique(_)); - indices.len() <= normalized_exprs.len() + let normalized_size = normalized_exprs.len(); + indices.len() <= normalized_size && self.oeq_class.iter().any(|ordering| { - if indices.len() > ordering.len() { + let length = ordering.len(); + if indices.len() > length || normalized_size < length { return false; } // Build a map of column positions in the ordering: - let mut col_positions = HashMap::with_capacity(ordering.len()); + let mut col_positions = HashMap::with_capacity(length); for (pos, req) in ordering.iter().enumerate() { if let Some(col) = req.expr.as_any().downcast_ref::() { @@ -673,26 +676,23 @@ impl EquivalenceProperties { } } // Check if all constraint indices appear in valid positions: - if !indices.iter().all(|&idx| { - col_positions - .get(&idx) - .map(|&(pos, nullable)| { - // For unique constraints, verify column is not nullable if it's first/last: - !check_null - || !nullable - || (pos != 0 && pos != ordering.len() - 1) - }) - .unwrap_or(false) + if !indices.iter().all(|idx| { + col_positions.get(idx).is_some_and(|&(pos, nullable)| { + // For unique constraints, verify column is not nullable if it's first/last: + !check_null + || !nullable + || (pos != 0 && pos != length - 1) + }) }) { return false; } // Check if this ordering matches the prefix: - let ordering_len = ordering.len(); - normalized_exprs.len() >= ordering_len - && normalized_exprs[..ordering_len] - .iter() - .zip(ordering) - .all(|(req, existing)| req == existing) + normalized_exprs + .iter() + .zip(ordering) + .all(|(given, existing)| { + existing.satisfy_expr(given, &self.schema) + }) }) } }) @@ -710,13 +710,15 @@ impl EquivalenceProperties { self.constraints.iter().any(|constraint| match constraint { Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { let check_null = matches!(constraint, Constraint::Unique(_)); - indices.len() <= normalized_reqs.len() + let normalized_size = normalized_reqs.len(); + indices.len() <= normalized_size && self.oeq_class.iter().any(|ordering| { - if indices.len() > ordering.len() { + let length = ordering.len(); + if indices.len() > length || normalized_size < length { return false; } // Build a map of column positions in the ordering: - let mut col_positions = HashMap::with_capacity(ordering.len()); + let mut col_positions = HashMap::with_capacity(length); for (pos, req) in ordering.iter().enumerate() { if let Some(col) = req.expr.as_any().downcast_ref::() { @@ -725,30 +727,23 @@ impl EquivalenceProperties { } } // Check if all constraint indices appear in valid positions: - if !indices.iter().all(|&idx| { - col_positions - .get(&idx) - .map(|&(pos, nullable)| { - // For unique constraints, verify column is not nullable if it's first/last: - !check_null - || !nullable - || (pos != 0 && pos != ordering.len() - 1) - }) - .unwrap_or(false) + if !indices.iter().all(|idx| { + col_positions.get(idx).is_some_and(|&(pos, nullable)| { + // For unique constraints, verify column is not nullable if it's first/last: + !check_null + || !nullable + || (pos != 0 && pos != length - 1) + }) }) { return false; } // Check if this ordering matches the prefix: - let ordering_len = ordering.len(); - normalized_reqs.len() >= ordering_len - && normalized_reqs[..ordering_len].iter().zip(ordering).all( - |(req, existing)| { - req.expr.eq(&existing.expr) - && req.options.is_none_or(|req_opts| { - req_opts == existing.options - }) - }, - ) + normalized_reqs + .iter() + .zip(ordering) + .all(|(given, existing)| { + existing.satisfy(given, &self.schema) + }) }) } }) @@ -758,8 +753,8 @@ impl EquivalenceProperties { /// than the `reference` sort requirements. pub fn requirements_compatible( &self, - given: &LexRequirement, - reference: &LexRequirement, + given: LexRequirement, + reference: LexRequirement, ) -> bool { let Some(normalized_given) = self.normalize_sort_requirements(given) else { return true; @@ -776,45 +771,6 @@ impl EquivalenceProperties { .all(|(reference, given)| given.compatible(&reference)) } - /// Returns the finer ordering among the requirements `lhs` and `rhs`, - /// breaking any ties by choosing `lhs`. - /// - /// The finer requirements are the ones that satisfy both of the given - /// requirements. If the requirements are incomparable, returns `None`. - /// - /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` - /// is the latter. - pub fn get_finer_requirement( - &self, - lhs: &LexRequirement, - rhs: &LexRequirement, - ) -> Option { - let Some(mut rhs) = self.normalize_sort_requirements(rhs) else { - return self.normalize_sort_requirements(lhs); - }; - let Some(mut lhs) = self.normalize_sort_requirements(lhs) else { - return Some(rhs); - }; - lhs.iter_mut() - .zip(rhs.iter_mut()) - .all(|(lhs, rhs)| { - lhs.expr.eq(&rhs.expr) - && match (lhs.options, rhs.options) { - (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, - (Some(options), None) => { - rhs.options = Some(options); - true - } - (None, Some(options)) => { - lhs.options = Some(options); - true - } - (None, None) => true, - } - }) - .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) - } - /// we substitute the ordering according to input expression type, this is a simplified version /// In this case, we just substitute when the expression satisfy the following condition: /// I. just have one column and is a CAST expression @@ -826,44 +782,44 @@ impl EquivalenceProperties { pub fn substitute_ordering_component( &self, mapping: &ProjectionMapping, - sort_expr: &LexOrdering, + sort_expr: LexOrdering, ) -> Result> { let new_orderings = sort_expr - .iter() + .into_iter() .map(|sort_expr| { - let referring_exprs: Vec<_> = mapping + let referring_exprs = mapping .iter() .map(|(source, _target)| source) .filter(|source| expr_refers(source, &sort_expr.expr)) - .cloned() - .collect(); - let mut res = LexOrdering::new(vec![sort_expr.clone()]); - // TODO: Add one-to-ones analysis for ScalarFunctions. + .cloned(); + let mut result = VecDeque::new(); + let expr_type = sort_expr.expr.data_type(&self.schema)?; + // TODO: Add one-to-one analysis for ScalarFunctions. for r_expr in referring_exprs { - // we check whether this expression is substitutable or not + // We check whether this expression is substitutable. if let Some(cast_expr) = r_expr.as_any().downcast_ref::() { - // we need to know whether the Cast Expr matches or not - let expr_type = sort_expr.expr.data_type(&self.schema)?; + // For casts, we need to know whether the cast expression matches: if cast_expr.expr.eq(&sort_expr.expr) - && cast_expr.is_bigger_cast(expr_type) + && cast_expr.is_bigger_cast(&expr_type) { - res.push(PhysicalSortExpr { - expr: Arc::clone(&r_expr), + result.push_back(PhysicalSortExpr { + expr: r_expr, options: sort_expr.options, }); } } } - Ok(res) + result.push_front(sort_expr); + Ok(result) }) .collect::>>()?; // Generate all valid orderings, given substituted expressions. - let res = new_orderings + let result = new_orderings .into_iter() .multi_cartesian_product() .map(LexOrdering::new) - .collect::>(); - Ok(res) + .collect(); + Ok(result) } /// In projection, supposed we have a input function 'A DESC B DESC' and the output shares the same expression @@ -872,15 +828,18 @@ impl EquivalenceProperties { /// Since it would cause bug in dependency constructions, we should substitute the input order in order to get correct /// dependency map, happen in issue 8838: pub fn substitute_oeq_class(&mut self, mapping: &ProjectionMapping) -> Result<()> { - let new_order = self - .oeq_class - .iter() + let oeq_class = mem::take(&mut self.oeq_class); + let new_orderings = oeq_class + .into_iter() .map(|order| self.substitute_ordering_component(mapping, order)) - .collect::>>()?; - let new_order = new_order.into_iter().flatten().collect(); - self.oeq_class = OrderingEquivalenceClass::new(new_order); + .collect::>>()? + .into_iter() + .flatten() + .collect(); + self.oeq_class = OrderingEquivalenceClass::new(new_orderings); Ok(()) } + /// Projects argument `expr` according to `projection_mapping`, taking /// equivalences into account. /// diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index a6766687a881a..fd4fc400420a1 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -97,8 +97,10 @@ impl CastExpr { pub fn cast_options(&self) -> &CastOptions<'static> { &self.cast_options } - pub fn is_bigger_cast(&self, src: DataType) -> bool { - if src == self.cast_type { + + /// Check if the cast is a widening cast (e.g. from `Int8` to `Int16`). + pub fn is_bigger_cast(&self, src: &DataType) -> bool { + if self.cast_type.eq(src) { return true; } matches!( diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index f0d1d57845b78..23cc61db67e5b 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -61,7 +61,8 @@ pub use physical_expr::{ pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; pub use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, + PhysicalSortRequirement, }; pub use planner::{create_physical_expr, create_physical_exprs}; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 5e3815cfb5f12..d052cf093784d 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1276,12 +1276,11 @@ pub fn ensure_distribution( // Either: // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. + let sort_req = required_input_ordering.into_single(); let ordering_satisfied = child .plan .equivalence_properties() - .ordering_satisfy_requirement( - required_input_ordering.lex_requirement(), - ); + .ordering_satisfy_requirement(sort_req.clone()); if (!ordering_satisfied || !order_preserving_variants_desirable) && child.data @@ -1293,7 +1292,7 @@ pub fn ensure_distribution( // Make sure to satisfy ordering requirement: child = add_sort_above_with_check( child, - required_input_ordering.lex_requirement().clone(), + sort_req, plan.as_any() .downcast_ref::() .map(|output| output.fetch()) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 2f3d817d1dcf4..33cca0b358dc4 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -283,7 +283,7 @@ fn replace_with_partial_sort( let mut common_prefix_length = 0; while child_eq_properties - .ordering_satisfy_requirement(&sort_req[0..common_prefix_length + 1]) + .ordering_satisfy_requirement(sort_req[0..common_prefix_length + 1].to_vec()) { common_prefix_length += 1; } @@ -496,14 +496,15 @@ pub fn ensure_sorting( if let Some(required) = required_ordering { let eq_properties = child.plan.equivalence_properties(); - if !eq_properties.ordering_satisfy_requirement(required.lex_requirement()) { + let req = required.into_single(); + if !eq_properties.ordering_satisfy_requirement(req.clone()) { // Make sure we preserve the ordering requirements: if physical_ordering.is_some() { child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; } child = add_sort_above( child, - required.lex_requirement().clone(), + req, plan.as_any() .downcast_ref::() .map(|output| output.fetch()) @@ -646,7 +647,7 @@ fn adjust_window_sort_removal( // Satisfy the ordering requirement so that the window can run: let mut child_node = window_tree.children.swap_remove(0); if let Some(reqs) = reqs { - child_node = add_sort_above(child_node, reqs.lex_requirement().clone(), None); + child_node = add_sort_above(child_node, reqs.into_single(), None); } let child_plan = Arc::clone(&child_node.plan); window_tree.children.push(child_node); diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 36ab0785e8f31..392aa70ff3ee4 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -170,7 +170,7 @@ fn plan_with_order_breaking_variants( || !required_ordering.is_some_and(|required_ordering| { node.plan .equivalence_properties() - .ordering_satisfy_requirement(required_ordering.lex_requirement()) + .ordering_satisfy_requirement(required_ordering.into_single()) })) { plan_with_order_breaking_variants(node) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 622b93cee1a2c..07e872a4ce34e 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -24,15 +24,13 @@ use crate::utils::{ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{ - internal_datafusion_err, internal_err, plan_err, HashSet, JoinSide, Result, -}; +use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortRequirement, +}; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ calculate_join_output_ordering, ColumnIndex, @@ -53,7 +51,7 @@ use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; /// [`EnforceSorting`]: crate::enforce_sorting::EnforceSorting #[derive(Default, Clone, Debug)] pub struct ParentRequirements { - ordering_requirement: Option, + ordering_requirement: Option, fetch: Option, } @@ -91,105 +89,107 @@ fn min_fetch(f1: Option, f2: Option) -> Option { fn pushdown_sorts_helper( mut sort_push_down: SortPushDown, ) -> Result> { - let plan = &Arc::clone(&sort_push_down.plan); - let parent_reqs = sort_push_down.data.ordering_requirement.clone(); - let satisfy_parent = parent_reqs.as_ref().is_none_or(|reqs| { - plan.equivalence_properties() - .ordering_satisfy_requirement(reqs.lex_requirement()) - }); - - if is_sort(plan) { - let current_sort_fetch = plan.fetch(); - let parent_req_fetch = sort_push_down.data.fetch; - - let Some(current_plan_ordering) = plan.output_ordering() else { + let plan = sort_push_down.plan; + let parent_fetch = sort_push_down.data.fetch; + + let Some(parent_requirement) = sort_push_down.data.ordering_requirement.clone() + else { + // If there are no ordering requirements from the parent, nothing to do + // unless we have a sort. + if is_sort(&plan) { + let Some(sort_ordering) = plan.output_ordering().cloned() else { + return internal_err!("SortExec should have output ordering"); + }; + // The sort is unnecessary, just propagate the stricter fetch and + // ordering requirements. + let fetch = min_fetch(plan.fetch(), parent_fetch); + sort_push_down = sort_push_down + .children + .swap_remove(0) + .update_plan_from_children()?; + sort_push_down.data.fetch = fetch; + sort_push_down.data.ordering_requirement = + Some(OrderingRequirements::from(sort_ordering)); + // Recursive call to helper, so it doesn't transform_down and miss + // the new node (previous child of sort): + return pushdown_sorts_helper(sort_push_down); + } + sort_push_down.plan = plan; + return Ok(Transformed::no(sort_push_down)); + }; + + let eqp = plan.equivalence_properties(); + let satisfy_parent = + eqp.ordering_satisfy_requirement(parent_requirement.first().clone()); + + if is_sort(&plan) { + let Some(sort_ordering) = plan.output_ordering().cloned() else { return internal_err!("SortExec should have output ordering"); }; - let current_plan_ordering_as_req = - RequiredInputOrdering::from(current_plan_ordering.clone()); - let parent_is_stricter = parent_reqs.as_ref().is_some_and(|parent_req| { - plan.equivalence_properties().requirements_compatible( - parent_req.lex_requirement(), - current_plan_ordering_as_req.lex_requirement(), - ) - }); - let current_is_stricter = parent_reqs.as_ref().is_none_or(|parent_req| { - plan.equivalence_properties().requirements_compatible( - current_plan_ordering_as_req.lex_requirement(), - parent_req.lex_requirement(), - ) - }); + let sort_fetch = plan.fetch(); + let parent_is_stricter = eqp.requirements_compatible( + parent_requirement.first().clone(), + sort_ordering.clone().into(), + ); + // Remove the current sort as we are either going to prove that it is + // unnecessary, or replace it with a stricter sort. + sort_push_down = sort_push_down + .children + .swap_remove(0) + .update_plan_from_children()?; if !satisfy_parent && !parent_is_stricter { - // This new sort has different requirements than the ordering being pushed down. - // 1. add a `SortExec` here for the pushed down ordering (parent reqs). - // 2. continue sort pushdown, but with the new ordering of the new sort. - - // remove current sort (which will be the new ordering to pushdown) - let new_reqs = current_plan_ordering_as_req; - sort_push_down = sort_push_down.children.swap_remove(0); - sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan - - if let Some(parent_reqs) = &parent_reqs { - // add back sort exec matching parent - sort_push_down = add_sort_above( - sort_push_down, - parent_reqs.lex_requirement().clone(), - parent_req_fetch, - ); - } - - // make pushdown requirements be the new ones. + // The sort was imposing a different ordering than the one being + // pushed down. Replace it with a sort that matches the pushed-down + // ordering, and continue the pushdown. + // Add back the sort: + sort_push_down = add_sort_above( + sort_push_down, + parent_requirement.into_single(), + parent_fetch, + ); + // Update pushdown requirements: sort_push_down.children[0].data = ParentRequirements { - ordering_requirement: Some(new_reqs), - fetch: current_sort_fetch, + ordering_requirement: Some(OrderingRequirements::from(sort_ordering)), + fetch: sort_fetch, }; return Ok(Transformed::yes(sort_push_down)); } else { - // Don't add a SortExec - // Do update what sort requirements to keep pushing down - - // remove current sort, and get the sort's child - sort_push_down = sort_push_down.children.swap_remove(0); - sort_push_down = sort_push_down.update_plan_from_children()?; // changed plan - - // set the stricter fetch - sort_push_down.data.fetch = min_fetch(current_sort_fetch, parent_req_fetch); - - // set the stricter ordering + // Sort was unnecessary, just propagate the stricter fetch and + // ordering requirements: + sort_push_down.data.fetch = min_fetch(sort_fetch, parent_fetch); + let current_is_stricter = eqp.requirements_compatible( + sort_ordering.clone().into(), + parent_requirement.first().clone(), + ); sort_push_down.data.ordering_requirement = if current_is_stricter { - Some(current_plan_ordering_as_req) + Some(OrderingRequirements::from(sort_ordering)) } else { - parent_reqs + Some(parent_requirement) }; - - // recursive call to helper, so it doesn't transform_down and miss the new node (previous child of sort) + // Recursive call to helper, so it doesn't transform_down and miss + // the new node (previous child of sort): return pushdown_sorts_helper(sort_push_down); } } - let Some(parent_requirement) = parent_reqs else { - // note: this `satisfy_parent`, but we don't want to push down anything. - // Nothing to do. - return Ok(Transformed::no(sort_push_down)); - }; - - let parent_fetch = sort_push_down.data.fetch; + sort_push_down.plan = plan; if satisfy_parent { // For non-sort operators which satisfy ordering: - let reqs = plan.required_input_ordering(); + let reqs = sort_push_down.plan.required_input_ordering(); for (child, order) in sort_push_down.children.iter_mut().zip(reqs) { child.data.ordering_requirement = order; child.data.fetch = min_fetch(parent_fetch, child.data.fetch); } - } else if let Some(adjusted) = - pushdown_requirement_to_children(plan, &parent_requirement)? - { - // For operators that can take a sort pushdown. - // Continue pushdown, with updated requirements: - let current_fetch = plan.fetch(); + } else if let Some(adjusted) = pushdown_requirement_to_children( + &sort_push_down.plan, + parent_requirement.clone(), + )? { + // For operators that can take a sort pushdown, continue with updated + // requirements: + let current_fetch = sort_push_down.plan.fetch(); for (child, order) in sort_push_down.children.iter_mut().zip(adjusted) { child.data.ordering_requirement = order; child.data.fetch = min_fetch(current_fetch, parent_fetch); @@ -197,14 +197,13 @@ fn pushdown_sorts_helper( sort_push_down.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - if let Some(sort_reqs) = &sort_push_down.data.ordering_requirement { - let sort_requirements = sort_reqs.lex_requirement().clone(); - sort_push_down = - add_sort_above(sort_push_down, sort_requirements, parent_fetch); - } + sort_push_down = add_sort_above( + sort_push_down, + parent_requirement.into_single(), + parent_fetch, + ); assign_initial_requirements(&mut sort_push_down); } - Ok(Transformed::yes(sort_push_down)) } @@ -212,8 +211,8 @@ fn pushdown_sorts_helper( /// If sort cannot be pushed down, return None. fn pushdown_requirement_to_children( plan: &Arc, - parent_required: &RequiredInputOrdering, -) -> Result>>> { + parent_required: OrderingRequirements, +) -> Result>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let mut required_input_ordering = plan.required_input_ordering(); @@ -222,7 +221,7 @@ fn pushdown_requirement_to_children( let Some(child_req) = maybe_child_requirement else { return Ok(None); }; - match determine_children_requirement(parent_required, &child_req, child_plan) { + match determine_children_requirement(&parent_required, &child_req, child_plan) { RequirementsCompatibility::Satisfy => Ok(Some(vec![Some(child_req)])), RequirementsCompatibility::Compatible(adjusted) => { // If parent requirements are more specific than output ordering @@ -231,7 +230,7 @@ fn pushdown_requirement_to_children( // that's the case, we block the pushdown of sort operation. if !plan .equivalence_properties() - .ordering_satisfy_requirement(parent_required.lex_requirement()) + .ordering_satisfy_requirement(parent_required.into_single()) { return Ok(None); } @@ -241,75 +240,63 @@ fn pushdown_requirement_to_children( RequirementsCompatibility::NonCompatible => Ok(None), } } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let sort_req = RequiredInputOrdering::from( - sort_exec - .properties() - .output_ordering() - .cloned() - .ok_or_else(|| { - internal_datafusion_err!("SortExec should have output ordering") - })?, - ); + let Some(sort_ordering) = sort_exec.properties().output_ordering().cloned() + else { + return internal_err!("SortExec should have output ordering"); + }; sort_exec .properties() .eq_properties .requirements_compatible( - parent_required.lex_requirement(), - sort_req.lex_requirement(), + parent_required.first().clone(), + sort_ordering.into(), ) - .then(|| Ok(vec![Some(parent_required.clone())])) + .then(|| Ok(vec![Some(parent_required)])) .transpose() } else if plan.fetch().is_some() && plan.supports_limit_pushdown() && plan .maintains_input_order() - .iter() - .all(|maintain| *maintain) + .into_iter() + .all(|maintain| maintain) { // Push down through operator with fetch when: // - requirement is aligned with output ordering // - it preserves ordering during execution let Some(ordering) = plan.properties().output_ordering() else { - return Ok(Some(vec![Some(parent_required.clone())])); + return Ok(Some(vec![Some(parent_required)])); }; - let output_req = LexRequirement::from(ordering.clone()); - if plan - .properties() - .eq_properties - .requirements_compatible(parent_required.lex_requirement(), &output_req) - { - Ok(Some(vec![Some(parent_required.clone())])) + if plan.properties().eq_properties.requirements_compatible( + parent_required.first().clone(), + ordering.clone().into(), + ) { + Ok(Some(vec![Some(parent_required)])) } else { Ok(None) } } else if is_union(plan) { - // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and - // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - let req = (!parent_required.is_empty()).then(|| parent_required.clone()); - Ok(Some(vec![req; plan.children().len()])) + // `UnionExec` does not have real sort requirements for its input, we + // just propagate the sort requirements down: + Ok(Some(vec![Some(parent_required); plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { - // If the current plan is SortMergeJoinExec let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = - LexOrdering::from(parent_required.lex_requirement().clone()); + let parent_required_expr = LexOrdering::from(parent_required.first().clone()); match expr_source_side(&parent_required_expr, smj.join_type(), left_columns_len) { Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, - parent_required.lex_requirement(), + parent_required.into_single(), &parent_required_expr, JoinSide::Left, ), Some(JoinSide::Right) => { let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); - let new_right_required = shift_right_required( - parent_required.lex_requirement(), - right_offset, - )?; + let new_right_required = + shift_right_required(parent_required.first(), right_offset)?; let new_right_required_expr = LexOrdering::from(new_right_required); try_pushdown_requirements_to_join( smj, - parent_required.lex_requirement(), + parent_required.into_single(), &new_right_required_expr, JoinSide::Right, ) @@ -325,14 +312,14 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // TODO: Add support for Projection push down || plan.as_any().is::() - || pushdown_would_violate_requirements(parent_required.lex_requirement(), plan.as_ref()) + || pushdown_would_violate_requirements(&parent_required, plan.as_ref()) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) } else if is_sort_preserving_merge(plan) { - let new_ordering = LexOrdering::from(parent_required.lex_requirement().clone()); + let new_ordering = LexOrdering::from(parent_required.first().clone()); let mut spm_eqs = plan.equivalence_properties().clone(); let old_ordering = spm_eqs.output_ordering().unwrap(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. @@ -340,7 +327,7 @@ fn pushdown_requirement_to_children( if spm_eqs.ordering_satisfy(old_ordering) { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - Ok(Some(vec![Some(parent_required.clone())])) + Ok(Some(vec![Some(parent_required)])) } else { // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. @@ -357,23 +344,21 @@ fn pushdown_requirement_to_children( /// Return true if pushing the sort requirements through a node would violate /// the input sorting requirements for the plan fn pushdown_would_violate_requirements( - parent_required: &LexRequirement, + parent_required: &OrderingRequirements, child: &dyn ExecutionPlan, ) -> bool { child .required_input_ordering() - .iter() + .into_iter() + // If there is no requirement, pushing down would not violate anything. + .flatten() .any(|child_required| { - let Some(child_required) = child_required.as_ref() else { - // No requirements, so pushing down would not violate anything. - return false; - }; // Check if the plan's requirements would still be satisfied if we // pushed down the parent requirements: child_required - .lex_requirement() + .into_single() .iter() - .zip(parent_required.iter()) + .zip(parent_required.first().iter()) .all(|(c, p)| !c.compatible(p)) }) } @@ -384,20 +369,20 @@ fn pushdown_would_violate_requirements( /// - If parent requirements are more specific, push down parent requirements. /// - If they are not compatible, need to add a sort. fn determine_children_requirement( - parent_required: &RequiredInputOrdering, - child_requirement: &RequiredInputOrdering, + parent_required: &OrderingRequirements, + child_requirement: &OrderingRequirements, child_plan: &Arc, ) -> RequirementsCompatibility { let eqp = child_plan.equivalence_properties(); if eqp.requirements_compatible( - child_requirement.lex_requirement(), - parent_required.lex_requirement(), + child_requirement.first().clone(), + parent_required.first().clone(), ) { // Child requirements are more specific, no need to push down. RequirementsCompatibility::Satisfy } else if eqp.requirements_compatible( - parent_required.lex_requirement(), - child_requirement.lex_requirement(), + parent_required.first().clone(), + child_requirement.first().clone(), ) { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: @@ -409,10 +394,10 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: &LexRequirement, + parent_required: LexRequirement, sort_expr: &LexOrdering, push_side: JoinSide, -) -> Result>>> { +) -> Result>>> { let mut smj_required_orderings = smj.required_input_ordering(); let (new_left_ordering, new_right_ordering) = match push_side { @@ -426,7 +411,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); }; if !left_eq_properties - .ordering_satisfy_requirement(left_requirement.lex_requirement()) + .ordering_satisfy_requirement(left_requirement.into_single()) { return Ok(None); } @@ -444,7 +429,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); }; if !right_eq_properties - .ordering_satisfy_requirement(right_requirement.lex_requirement()) + .ordering_satisfy_requirement(right_requirement.into_single()) { return Ok(None); } @@ -472,7 +457,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(RequiredInputOrdering::from(sort_expr.clone())); + let new_req = Some(OrderingRequirements::from(sort_expr.clone())); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; @@ -577,17 +562,18 @@ fn shift_right_required( /// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. fn handle_custom_pushdown( plan: &Arc, - parent_required: &RequiredInputOrdering, + parent_required: OrderingRequirements, maintains_input_order: Vec, -) -> Result>>> { - // If there's no requirement from the parent or the plan has no children, return early - if parent_required.is_empty() || plan.children().is_empty() { +) -> Result>>> { + // If the plan has no children, return early: + if plan.children().is_empty() { return Ok(None); } - // Collect all unique column indices used in the parent-required sorting expression - let all_indices: HashSet = parent_required - .lex_requirement() + // Collect all unique column indices used in the parent-required sorting + // expression: + let requirement = parent_required.into_single(); + let all_indices: HashSet = requirement .iter() .flat_map(|order| { collect_columns(&order.expr) @@ -597,14 +583,14 @@ fn handle_custom_pushdown( }) .collect(); - // Get the number of fields in each child's schema - let len_of_child_schemas: Vec = plan + // Get the number of fields in each child's schema: + let children_schema_lengths: Vec = plan .children() .iter() .map(|c| c.schema().fields().len()) .collect(); - // Find the index of the child that maintains input order + // Find the index of the order-maintaining child: let Some(maintained_child_idx) = maintains_input_order .iter() .enumerate() @@ -614,27 +600,28 @@ fn handle_custom_pushdown( return Ok(None); }; - // Check if all required columns come from the child that maintains input order - let start_idx = len_of_child_schemas[..maintained_child_idx] + // Check if all required columns come from the order-maintaining child: + let start_idx = children_schema_lengths[..maintained_child_idx] .iter() .sum::(); - let end_idx = start_idx + len_of_child_schemas[maintained_child_idx]; + let end_idx = start_idx + children_schema_lengths[maintained_child_idx]; let all_from_maintained_child = all_indices.iter().all(|i| i >= &start_idx && i < &end_idx); - // If all columns are from the maintained child, update the parent requirements + // If all columns are from the maintained child, update the parent requirements: if all_from_maintained_child { - let sub_offset = len_of_child_schemas + let sub_offset = children_schema_lengths .iter() .take(maintained_child_idx) .sum::(); - // Transform the parent-required expression for the child schema by adjusting columns - let updated_parent_req = parent_required - .lex_requirement() - .iter() + // Transform the parent-required expression for the child schema by + // adjusting columns: + let updated_parent_req = requirement + .into_iter() .map(|req| { let child_schema = plan.children()[maintained_child_idx].schema(); - let updated_columns = Arc::clone(&req.expr) + let updated_columns = req + .expr .transform_up(|expr| { if let Some(col) = expr.as_any().downcast_ref::() { let new_index = col.index() - sub_offset; @@ -656,7 +643,7 @@ fn handle_custom_pushdown( .iter() .map(|&maintains_order| { maintains_order.then(|| { - RequiredInputOrdering::new(updated_parent_req.clone().into()) + OrderingRequirements::new_single(updated_parent_req.clone().into()) }) }) .collect(); @@ -671,17 +658,17 @@ fn handle_custom_pushdown( // for join type: Inner, Right, RightSemi, RightAnti fn handle_hash_join( plan: &HashJoinExec, - parent_required: &RequiredInputOrdering, -) -> Result>>> { - // If there's no requirement from the parent or the plan has no children - // or the join type is not Inner, Right, RightSemi, RightAnti, return early - if parent_required.is_empty() || !plan.maintains_input_order()[1] { + parent_required: OrderingRequirements, +) -> Result>>> { + // If the plan has no children or does not maintain the right side ordering, + // return early: + if !plan.maintains_input_order()[1] { return Ok(None); } // Collect all unique column indices used in the parent-required sorting expression - let all_indices: HashSet<_> = parent_required - .lex_requirement() + let requirement = parent_required.into_single(); + let all_indices: HashSet<_> = requirement .iter() .flat_map(|order| { collect_columns(&order.expr) @@ -707,12 +694,12 @@ fn handle_hash_join( // If all columns are from the right child, update the parent requirements if all_from_right_child { // Transform the parent-required expression for the child schema by adjusting columns - let updated_parent_req = parent_required - .lex_requirement() - .iter() + let updated_parent_req = requirement + .into_iter() .map(|req| { let child_schema = plan.children()[1].schema(); - let updated_columns = Arc::clone(&req.expr) + let updated_columns = req + .expr .transform_up(|expr| { if let Some(col) = expr.as_any().downcast_ref::() { let index = projected_indices[col.index()].index; @@ -732,7 +719,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - Some(RequiredInputOrdering::new(updated_parent_req.into())), + Some(OrderingRequirements::new_single(updated_parent_req.into())), ])) } else { Ok(None) @@ -771,7 +758,7 @@ enum RequirementsCompatibility { /// Requirements satisfy Satisfy, /// Requirements compatible - Compatible(Option), + Compatible(Option), /// Requirements not compatible NonCompatible, } diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index d5701289673ae..4f6a58fb25cbf 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -30,8 +30,10 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{Distribution, PhysicalSortRequirement}; -use datafusion_physical_plan::execution_plan::RequiredInputOrdering; +use datafusion_physical_expr::Distribution; +use datafusion_physical_expr_common::sort_expr::{ + OrderingRequirements, PhysicalSortRequirement, +}; use datafusion_physical_plan::projection::{ make_with_child, update_expr, ProjectionExec, }; @@ -95,7 +97,7 @@ enum RuleMode { #[derive(Debug)] pub struct OutputRequirementExec { input: Arc, - order_requirement: Option, + order_requirement: Option, dist_requirement: Distribution, cache: PlanProperties, } @@ -103,7 +105,7 @@ pub struct OutputRequirementExec { impl OutputRequirementExec { pub fn new( input: Arc, - requirements: Option, + requirements: Option, dist_requirement: Distribution, ) -> Self { let cache = Self::compute_properties(&input); @@ -177,7 +179,7 @@ impl ExecutionPlan for OutputRequirementExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.order_requirement.clone()] } @@ -215,8 +217,8 @@ impl ExecutionPlan for OutputRequirementExec { let mut updated_sort_reqs = vec![]; // None or empty_vec can be treated in the same way. - if let Some(reqs) = &self.required_input_ordering()[0] { - for req in reqs.lex_requirement().iter() { + if let Some(reqs) = self.required_input_ordering().swap_remove(0) { + for req in reqs.into_single().iter() { let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { return Ok(None); @@ -247,7 +249,7 @@ impl ExecutionPlan for OutputRequirementExec { .map(|input| { OutputRequirementExec::new( input, - Some(RequiredInputOrdering::new(updated_sort_reqs.into())), + Some(OrderingRequirements::new_single(updated_sort_reqs.into())), dist_req, ) }) @@ -318,13 +320,13 @@ fn require_top_ordering_helper( // Therefore; we check the sort expression field of the SortExec to assign the requirements. let req_ordering = sort_exec.expr(); let req_dist = sort_exec.required_input_distribution()[0].clone(); - let reqs = RequiredInputOrdering::from(req_ordering.clone()); + let reqs = OrderingRequirements::from(req_ordering.clone()); Ok(( Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, true, )) } else if let Some(spm) = plan.as_any().downcast_ref::() { - let reqs = RequiredInputOrdering::from(spm.expr().clone()); + let reqs = OrderingRequirements::from(spm.expr().clone()); Ok(( Arc::new(OutputRequirementExec::new( plan, @@ -337,7 +339,7 @@ fn require_top_ordering_helper( && (plan.required_input_ordering()[0].is_none() || matches!( plan.required_input_ordering()[0].clone().unwrap(), - RequiredInputOrdering::Soft(_) + OrderingRequirements::Soft(_) )) { // Keep searching for a `SortExec` as long as ordering is maintained, diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index 85b0f7f2f4cd6..10b023d80e171 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -137,12 +137,13 @@ pub fn check_plan_sanity( ) { let child_eq_props = child.equivalence_properties(); if let Some(sort_req) = sort_req { - if !child_eq_props.ordering_satisfy_requirement(sort_req.lex_requirement()) { + let sort_req = sort_req.into_single(); + if !child_eq_props.ordering_satisfy_requirement(sort_req.clone()) { let plan_str = get_plan_string(&plan); return plan_err!( "Plan: {:?} does not satisfy order requirements: {}. Child-{} order: {}", plan_str, - format_physical_sort_requirement_list(sort_req.lex_requirement()), + format_physical_sort_requirement_list(&sort_req), idx, child_eq_props.oeq_class() ); diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 7a08be6a9f55e..03ec87773febe 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -164,7 +164,7 @@ fn try_convert_aggregate_if_better( if !aggr_expr.order_sensitivity().is_beneficial() { Ok(aggr_expr) } else if !order_bys.is_empty() { - if eq_properties.ordering_satisfy_requirement(&concat_slices( + if eq_properties.ordering_satisfy_requirement(concat_slices( prefix_requirement, &order_bys .iter() @@ -173,7 +173,7 @@ fn try_convert_aggregate_if_better( )) { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + } else if eq_properties.ordering_satisfy_requirement(concat_slices( prefix_requirement, &order_bys .iter() diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 46ad84c64a455..6c480e4c67165 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -68,7 +68,7 @@ pub fn add_sort_above_with_check( if !node .plan .equivalence_properties() - .ordering_satisfy_requirement(&sort_requirements) + .ordering_satisfy_requirement(sort_requirements.clone()) { add_sort_above(node, sort_requirements, fetch) } else { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5ecfe8a359731..4a01da2e0d720 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -25,7 +25,7 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; -use crate::execution_plan::{CardinalityEffect, EmissionType, RequiredInputOrdering}; +use crate::execution_plan::{CardinalityEffect, EmissionType}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::get_field_metadata; use crate::windows::get_ordered_partition_by_indices; @@ -42,13 +42,16 @@ use datafusion_common::{internal_err, not_impl_err, Constraint, Constraints, Res use datafusion_execution::TaskContext; use datafusion_expr::{Accumulator, Aggregate}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::equivalence::ProjectionMapping; +use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - equivalence::ProjectionMapping, expressions::Column, physical_exprs_contains, - ConstExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, - PhysicalSortRequirement, + physical_exprs_contains, ConstExpr, EquivalenceProperties, +}; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortRequirement, }; -use datafusion_physical_expr_common::physical_expr::fmt_sql; use itertools::Itertools; pub(crate) mod group_values; @@ -390,7 +393,7 @@ pub struct AggregateExec { pub input_schema: SchemaRef, /// Execution metrics metrics: ExecutionPlanMetricsSet, - required_input_ordering: Option, + required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, cache: PlanProperties, @@ -498,7 +501,7 @@ impl AggregateExec { None } else { let reqs = LexRequirement::from(new_requirements).collapse(); - RequiredInputOrdering::new_with_alternatives(vec![reqs], true) + OrderingRequirements::new(vec![reqs], true) }; // If our aggregation has grouping sets then our base grouping exprs will @@ -657,7 +660,7 @@ impl AggregateExec { } // ensure no ordering is required on the input if let Some(requirement) = self.required_input_ordering()[0].clone() { - return matches!(requirement, RequiredInputOrdering::Hard(_)); + return matches!(requirement, OrderingRequirements::Hard(_)); } true } @@ -889,7 +892,7 @@ impl ExecutionPlan for AggregateExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.required_input_ordering.clone()] } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 947f161273eda..084724657b1f3 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -48,8 +48,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{exec_err, Constraints, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; use futures::stream::{StreamExt, TryStreamExt}; @@ -136,7 +136,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// NOTE that checking `!is_empty()` does **not** check for a /// required input ordering. Instead, the correct check is that at /// least one entry must be `Some` - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![None; self.children().len()] } @@ -1070,63 +1070,6 @@ pub enum CardinalityEffect { GreaterEqual, } -/// Represents the plan's input ordering requirements, -/// the elements of the vectors' represent alternative requirements -#[derive(Debug, Clone, PartialEq)] -pub enum RequiredInputOrdering { - /// The operator is not able to work without one of these requirements - Hard(Vec), - /// The operator can benefit from the ordering alternatives if provided - /// but if not provided it can also work. - Soft(Vec), -} - -impl RequiredInputOrdering { - /// Creates a new RequiredInputOrdering instance, - /// empty requirements are not allowed inside this type, - /// if given [`None`] will be returned - pub fn new_with_alternatives( - alternatives: Vec, - soft: bool, - ) -> Option { - (!(alternatives.is_empty() || alternatives[0].is_empty())).then(|| { - if soft { - Self::Soft(alternatives) - } else { - Self::Hard(alternatives) - } - }) - } - - pub fn new(requirement: LexRequirement) -> Self { - debug_assert!(!requirement.is_empty()); - Self::Hard(vec![requirement]) - } - - pub fn from(ordering: LexOrdering) -> Self { - Self::new(LexRequirement::from(ordering)) - } - - pub fn add_alternative(&mut self, requirement: LexRequirement) { - match self { - Self::Hard(alts) => alts.push(requirement), - Self::Soft(alts) => alts.push(requirement), - } - } - - /// Returns the first (i.e. most preferred) among alternative requirements. - pub fn lex_requirement(&self) -> &LexRequirement { - match self { - Self::Hard(alts) => &alts[0], - Self::Soft(alts) => &alts[0], - } - } - - pub fn is_empty(&self) -> bool { - self.lex_requirement().is_empty() - } -} - #[cfg(test)] mod tests { use std::any::Any; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 25fded8d55898..fab25d67aa519 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,9 +34,7 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::execution_plan::{ - boundedness_from_children, EmissionType, RequiredInputOrdering, -}; +use crate::execution_plan::{boundedness_from_children, EmissionType}; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, @@ -73,9 +71,8 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_expr_common::physical_expr::fmt_sql; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; use futures::{Stream, StreamExt}; @@ -418,10 +415,10 @@ impl ExecutionPlan for SortMergeJoinExec { ] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![ - Some(RequiredInputOrdering::from(self.left_sort_exprs.clone())), - Some(RequiredInputOrdering::from(self.right_sort_exprs.clone())), + Some(OrderingRequirements::from(self.left_sort_exprs.clone())), + Some(OrderingRequirements::from(self.right_sort_exprs.clone())), ] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index a6de0090a28b4..5f4eace3c3c17 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -33,9 +33,7 @@ use std::task::{Context, Poll}; use std::vec; use crate::common::SharedMemoryReservation; -use crate::execution_plan::{ - boundedness_from_children, emission_type_from_children, RequiredInputOrdering, -}; +use crate::execution_plan::{boundedness_from_children, emission_type_from_children}; use crate::joins::hash_join::{equal_rows_arr, update_hash}; use crate::joins::stream_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, @@ -75,9 +73,8 @@ use datafusion_execution::TaskContext; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; -use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_expr_common::physical_expr::fmt_sql; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; use ahash::RandomState; use futures::{ready, Stream, StreamExt}; @@ -435,14 +432,14 @@ impl ExecutionPlan for SymmetricHashJoinExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![ self.left_sort_exprs .as_ref() - .map(|e| RequiredInputOrdering::from(e.clone())), + .map(|e| OrderingRequirements::from(e.clone())), self.right_sort_exprs .as_ref() - .map(|e| RequiredInputOrdering::from(e.clone())), + .map(|e| OrderingRequirements::from(e.clone())), ] } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 62077ae2ca29f..8f60b726cb834 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -21,7 +21,6 @@ use std::any::Any; use std::sync::Arc; use crate::common::spawn_buffered; -use crate::execution_plan::RequiredInputOrdering; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{make_with_child, update_expr, ProjectionExec}; @@ -34,8 +33,9 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, OrderingRequirements, PhysicalSortExpr, +}; use log::{debug, trace}; @@ -241,8 +241,8 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![false] } - fn required_input_ordering(&self) -> Vec> { - vec![Some(RequiredInputOrdering::from(self.expr.clone()))] + fn required_input_ordering(&self) -> Vec> { + vec![Some(OrderingRequirements::from(self.expr.clone()))] } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 1461a937a2b38..3898d2af70c5a 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -28,7 +28,6 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::execution_plan::RequiredInputOrdering; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -61,7 +60,10 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::{ + OrderingRequirements, PhysicalSortExpr, +}; use ahash::RandomState; use futures::stream::Stream; @@ -287,7 +289,7 @@ impl ExecutionPlan for BoundedWindowAggExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); let partition_bys = self diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0b52c36d76e26..619f0059bc541 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -25,7 +25,6 @@ use std::borrow::Borrow; use std::iter; use std::sync::Arc; -use crate::execution_plan::RequiredInputOrdering; use crate::{ expressions::PhysicalSortExpr, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, @@ -46,8 +45,9 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::window::{ SlidingAggregateWindowExpr, StandardWindowFunctionExpr, }; -use datafusion_physical_expr::{ - ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, +use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, OrderingRequirements, PhysicalSortRequirement, }; use itertools::Itertools; @@ -280,7 +280,7 @@ pub(crate) fn calc_requirements< >( partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, -) -> Option { +) -> Option { let mut sort_reqs_with_partition = partition_by_exprs .into_iter() .map(|partition_by| { @@ -310,7 +310,7 @@ pub(crate) fn calc_requirements< alternatives.push(sort_reqs.into()); } - RequiredInputOrdering::new_with_alternatives(alternatives, false) + OrderingRequirements::new(alternatives, false) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -611,7 +611,7 @@ pub fn get_window_mode( { let mut req = partition_by_reqs.clone(); req.extend(orderbys.iter().cloned().map(Into::into)); - if req.is_empty() || partition_by_eqs.ordering_satisfy_requirement(&req) { + if req.is_empty() || partition_by_eqs.ordering_satisfy_requirement(req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { InputOrderMode::Sorted @@ -644,11 +644,11 @@ mod tests { use arrow::compute::SortOptions; use datafusion_execution::TaskContext; - use datafusion_functions_aggregate::count::count_udaf; - use futures::FutureExt; use InputOrderMode::{Linear, PartiallySorted, Sorted}; + use futures::FutureExt; + fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); @@ -771,7 +771,7 @@ mod tests { orderbys.push(PhysicalSortExpr { expr, options }); } - let mut expected: Option = None; + let mut expected: Option = None; for expected_param in expected_params.clone() { let mut requirements = vec![]; for (col_name, reqs) in expected_param { @@ -786,14 +786,12 @@ mod tests { if let Some(alts) = expected.as_mut() { alts.add_alternative(requirements.into()); } else { - expected = Some(RequiredInputOrdering::new(requirements.into())); + expected = + Some(OrderingRequirements::new_single(requirements.into())); } } } - assert_eq!( - calc_requirements(partitionbys.clone(), orderbys.clone()), - expected - ); + assert_eq!(calc_requirements(partitionbys, orderbys), expected); } Ok(()) } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index c2765319483a9..23b18ce856b6a 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::execution_plan::{EmissionType, RequiredInputOrdering}; +use crate::execution_plan::EmissionType; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -44,7 +44,9 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{ + OrderingRequirements, PhysicalSortExpr, +}; use futures::{ready, Stream, StreamExt}; @@ -216,7 +218,7 @@ impl ExecutionPlan for WindowAggExec { vec![true] } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { From c4d70a4d4066785f6d4e3f3abd04831db848bb22 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 14 Apr 2025 17:50:30 +0300 Subject: [PATCH 083/167] Simplify new_with_orderings --- .../datasource/physical_plan/arrow_file.rs | 4 ++-- datafusion/datasource-avro/src/source.rs | 4 ++-- datafusion/datasource-csv/src/source.rs | 4 ++-- datafusion/datasource-json/src/source.rs | 4 ++-- datafusion/datasource-parquet/src/mod.rs | 2 +- datafusion/datasource/src/file_scan_config.rs | 2 +- datafusion/datasource/src/memory.rs | 6 ++--- datafusion/ffi/src/plan_properties.rs | 5 +--- .../physical-expr/src/equivalence/ordering.rs | 24 +++++++++---------- .../src/equivalence/properties/mod.rs | 14 ++++++++--- .../src/equivalence/properties/union.rs | 2 +- datafusion/physical-plan/src/streaming.rs | 4 ++-- datafusion/physical-plan/src/test.rs | 6 ++--- 13 files changed, 42 insertions(+), 39 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 5dcf4df73f57a..959bbadc04e5e 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -66,7 +66,7 @@ impl ArrowExec { ) = base_config.project(); let cache = Self::compute_properties( Arc::clone(&projected_schema), - &projected_output_ordering, + projected_output_ordering, projected_constraints, &base_config, ); @@ -107,7 +107,7 @@ impl ArrowExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - output_ordering: &[LexOrdering], + output_ordering: Vec, constraints: Constraints, file_scan_config: &FileScanConfig, ) -> PlanProperties { diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index ce3722e7b11ee..c4efbaf04570f 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -62,7 +62,7 @@ impl AvroExec { ) = base_config.project(); let cache = Self::compute_properties( Arc::clone(&projected_schema), - &projected_output_ordering, + projected_output_ordering, projected_constraints, &base_config, ); @@ -81,7 +81,7 @@ impl AvroExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, constraints: Constraints, file_scan_config: &FileScanConfig, ) -> PlanProperties { diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 6db4d18703204..0e5936631033c 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -196,7 +196,7 @@ impl CsvExecBuilder { ) = base_config.project(); let cache = CsvExec::compute_properties( projected_schema, - &projected_output_ordering, + projected_output_ordering, projected_constraints, &base_config, ); @@ -298,7 +298,7 @@ impl CsvExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, constraints: Constraints, file_scan_config: &FileScanConfig, ) -> PlanProperties { diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index f1adccf9ded7d..f236cf1c17701 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -76,7 +76,7 @@ impl NdJsonExec { ) = base_config.project(); let cache = Self::compute_properties( projected_schema, - &projected_output_ordering, + projected_output_ordering, projected_constraints, &base_config, ); @@ -129,7 +129,7 @@ impl NdJsonExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, constraints: Constraints, file_scan_config: &FileScanConfig, ) -> PlanProperties { diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 516b13792189b..37d7462cb0809 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -444,7 +444,7 @@ impl ParquetExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, constraints: Constraints, file_config: &FileScanConfig, ) -> PlanProperties { diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 5b15f3d34f618..f0d6802fa1ce8 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -531,7 +531,7 @@ impl DataSource for FileScanConfig { fn eq_properties(&self) -> EquivalenceProperties { let (schema, constraints, _, orderings) = self.project(); - EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) + EquivalenceProperties::new_with_orderings(schema, orderings) .with_constraints(constraints) } diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 92dafa93dd759..2d584f7e710cf 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -336,7 +336,7 @@ impl MemoryExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, constraints: Constraints, partitions: &[Vec], ) -> PlanProperties { @@ -452,7 +452,7 @@ impl DataSource for MemorySourceConfig { fn eq_properties(&self) -> EquivalenceProperties { EquivalenceProperties::new_with_orderings( Arc::clone(&self.projected_schema), - self.sort_information.as_slice(), + self.sort_information.clone(), ) } @@ -697,7 +697,7 @@ impl MemorySourceConfig { if let Some(projection) = &self.projection { let base_eqp = EquivalenceProperties::new_with_orderings( self.original_schema(), - &sort_information, + sort_information, ); let proj_exprs = projection .iter() diff --git a/datafusion/ffi/src/plan_properties.rs b/datafusion/ffi/src/plan_properties.rs index 568cacc5a2694..d6901be3b9537 100644 --- a/datafusion/ffi/src/plan_properties.rs +++ b/datafusion/ffi/src/plan_properties.rs @@ -214,10 +214,7 @@ impl TryFrom for PlanProperties { let eq_properties = if sort_exprs.is_empty() { EquivalenceProperties::new(Arc::new(schema)) } else { - EquivalenceProperties::new_with_orderings( - Arc::new(schema), - &[sort_exprs.into()], - ) + EquivalenceProperties::new_with_orderings(Arc::new(schema), [sort_exprs]) }; let emission_type: EmissionType = diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 3ec8459e5b469..a5d2b2d92a581 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -176,14 +176,14 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - (!self.orderings.is_empty()).then(|| { - self.orderings - .iter() - .flatten() - .cloned() - .collect::() - .collapse() - }) + self.orderings + .iter() + .cloned() + .reduce(|mut cat, o| { + cat.extend(o); + cat + }) + .map(|o| o.collapse()) } // Append orderings in `other` to all existing orderings in this equivalence @@ -381,15 +381,13 @@ mod tests { // finer ordering satisfies, crude ordering should return true let eq_properties_finer = EquivalenceProperties::new_with_orderings( Arc::clone(&input_schema), - &[finer.clone()], + [finer.clone()], ); assert!(eq_properties_finer.ordering_satisfy(crude.clone())); // Crude ordering doesn't satisfy finer ordering. should return false - let eq_properties_crude = EquivalenceProperties::new_with_orderings( - Arc::clone(&input_schema), - &[crude], - ); + let eq_properties_crude = + EquivalenceProperties::new_with_orderings(Arc::clone(&input_schema), [crude]); assert!(!eq_properties_crude.ordering_satisfy(finer)); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 2373244c2a0f5..0cd800faf336c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -169,10 +169,18 @@ impl EquivalenceProperties { } /// Creates a new `EquivalenceProperties` object with the given orderings. - pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { + pub fn new_with_orderings( + schema: SchemaRef, + orderings: impl IntoIterator>, + ) -> Self { + let orderings = orderings + .into_iter() + .map(|o| o.into_iter().collect::>()) + .filter_map(|v| (!v.is_empty()).then(|| LexOrdering::new(v))) + .collect(); Self { eq_group: EquivalenceGroup::empty(), - oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), + oeq_class: OrderingEquivalenceClass::new(orderings), constants: vec![], constraints: Constraints::empty(), schema, @@ -270,7 +278,7 @@ impl EquivalenceProperties { &mut self, ordering: impl IntoIterator, ) { - self.add_new_orderings([ordering]); + self.add_new_orderings(std::iter::once(ordering)); } /// Incorporates the given equivalence group to into the existing diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 9e1c50c2f44b7..5da1f421531af 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -841,7 +841,7 @@ mod tests { .map(|col_name| ConstExpr::new(col(col_name, schema).unwrap())) .collect::>(); - EquivalenceProperties::new_with_orderings(Arc::clone(schema), &orderings) + EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings) .with_constants(constants) } } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 455d4ecfc846d..2cba30a98275c 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -99,7 +99,7 @@ impl StreamingTableExec { projected_output_ordering.into_iter().collect::>(); let cache = Self::compute_properties( Arc::clone(&projected_schema), - &projected_output_ordering, + projected_output_ordering.clone(), &partitions, infinite, ); @@ -146,7 +146,7 @@ impl StreamingTableExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( schema: SchemaRef, - orderings: &[LexOrdering], + orderings: Vec, partitions: &[Arc], infinite: bool, ) -> PlanProperties { diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index a2dc1d778436a..e937b4419ad4d 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -210,7 +210,7 @@ impl TestMemoryExec { fn eq_properties(&self) -> EquivalenceProperties { EquivalenceProperties::new_with_orderings( Arc::clone(&self.projected_schema), - self.sort_information.as_slice(), + self.sort_information.clone(), ) } @@ -234,7 +234,7 @@ impl TestMemoryExec { cache: PlanProperties::new( EquivalenceProperties::new_with_orderings( Arc::clone(&projected_schema), - vec![].as_slice(), + Vec::::new(), ), Partitioning::UnknownPartitioning(partitions.len()), EmissionType::Incremental, @@ -320,7 +320,7 @@ impl TestMemoryExec { if let Some(projection) = &self.projection { let base_eqp = EquivalenceProperties::new_with_orderings( self.original_schema(), - &sort_information, + sort_information, ); let proj_exprs = projection .iter() From 38ff1eb587095ba934d576e2228fef2b3594cbe7 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 00:56:44 +0300 Subject: [PATCH 084/167] Transition to fallible LexOrdering constructor --- benchmarks/src/sort.rs | 30 +- datafusion/core/benches/spm.rs | 6 +- .../core/src/datasource/listing/table.rs | 23 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 6 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 21 +- .../core/tests/fuzz_cases/window_fuzz.rs | 5 +- datafusion/core/tests/memory_limit/mod.rs | 5 +- .../enforce_distribution.rs | 244 ++-- .../physical_optimizer/enforce_sorting.rs | 1082 ++++++++--------- .../limited_distinct_aggregation.rs | 12 +- .../physical_optimizer/projection_pushdown.rs | 57 +- .../replace_with_order_preserving_variants.rs | 31 +- .../physical_optimizer/sanity_checker.rs | 45 +- .../tests/physical_optimizer/test_utils.rs | 19 +- datafusion/datasource/src/file_scan_config.rs | 7 +- datafusion/datasource/src/memory.rs | 10 +- .../functions-aggregate/src/array_agg.rs | 2 +- .../functions-aggregate/src/first_last.rs | 18 +- .../physical-expr-common/src/sort_expr.rs | 36 +- .../physical-expr/src/equivalence/ordering.rs | 11 +- .../src/equivalence/properties/dependency.rs | 77 +- .../src/equivalence/properties/mod.rs | 25 +- .../src/equivalence/properties/union.rs | 7 +- datafusion/physical-expr/src/physical_expr.rs | 4 +- .../src/enforce_sorting/mod.rs | 17 +- .../physical-optimizer/src/join_selection.rs | 2 +- .../src/topk_aggregation.rs | 3 +- .../physical-plan/src/aggregates/mod.rs | 31 +- .../src/joins/sort_merge_join.rs | 14 +- .../src/joins/symmetric_hash_join.rs | 131 +- .../physical-plan/src/joins/test_utils.rs | 6 +- datafusion/physical-plan/src/joins/utils.rs | 12 +- .../physical-plan/src/repartition/mod.rs | 8 +- .../physical-plan/src/sorts/partial_sort.rs | 83 +- datafusion/physical-plan/src/sorts/sort.rs | 72 +- .../src/sorts/sort_preserving_merge.rs | 115 +- .../src/windows/bounded_window_agg_exec.rs | 5 +- datafusion/physical-plan/src/windows/mod.rs | 6 +- .../tests/cases/roundtrip_physical_plan.rs | 46 +- 39 files changed, 1211 insertions(+), 1123 deletions(-) diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index 9cf09c57205a7..124c2e0b29d92 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -70,28 +70,31 @@ impl RunOpt { let sort_cases = vec![ ( "sort utf8", - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("request_method", &schema)?, options: Default::default(), - }]), + }] + .into(), ), ( "sort int", - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("response_bytes", &schema)?, options: Default::default(), - }]), + }] + .into(), ), ( "sort decimal", - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("decimal_price", &schema)?, options: Default::default(), - }]), + }] + .into(), ), ( "sort integer tuple", - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("request_bytes", &schema)?, options: Default::default(), @@ -100,11 +103,12 @@ impl RunOpt { expr: col("response_bytes", &schema)?, options: Default::default(), }, - ]), + ] + .into(), ), ( "sort utf8 tuple", - LexOrdering::new(vec![ + [ // sort utf8 tuple PhysicalSortExpr { expr: col("service", &schema)?, @@ -122,11 +126,12 @@ impl RunOpt { expr: col("image", &schema)?, options: Default::default(), }, - ]), + ] + .into(), ), ( "sort mixed tuple", - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("service", &schema)?, options: Default::default(), @@ -139,7 +144,8 @@ impl RunOpt { expr: col("decimal_price", &schema)?, options: Default::default(), }, - ]), + ] + .into(), ), ]; for (title, expr) in sort_cases { diff --git a/datafusion/core/benches/spm.rs b/datafusion/core/benches/spm.rs index 63b06f20cd86a..353b58cb3c1fa 100644 --- a/datafusion/core/benches/spm.rs +++ b/datafusion/core/benches/spm.rs @@ -21,7 +21,6 @@ use arrow::array::{ArrayRef, Int32Array, Int64Array, RecordBatch, StringArray}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{collect, ExecutionPlan}; @@ -70,7 +69,7 @@ fn generate_spm_for_round_robin_tie_breaker( let partitiones = vec![rbs.clone(); partition_count]; let schema = rb.schema(); - let sort = LexOrdering::new(vec![ + let sort = [ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -79,7 +78,8 @@ fn generate_spm_for_round_robin_tie_breaker( expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]); + ] + .into(); let exec = MemorySourceConfig::try_new_exec(&partitiones, schema, None).unwrap(); SortPreservingMergeExec::new(sort, exec) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 65abafd910cdc..47c4c0447aaeb 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1244,7 +1244,10 @@ mod tests { use crate::datasource::{provider_as_source, DefaultTableSource, MemTable}; use crate::execution::options::ArrowReadOptions; use crate::prelude::*; - use crate::test::{columns, object_store::register_test_store}; + use crate::test::object_store::{ + ensure_head_concurrency, make_test_store_and_state, register_test_store + }; + use crate::test::columns; use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; @@ -1253,10 +1256,8 @@ mod tests { use datafusion_common::{assert_contains, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; - use datafusion_physical_plan::collect; - use datafusion_physical_plan::ExecutionPlanProperties; + use datafusion_physical_plan::{collect, ExecutionPlanProperties}; - use crate::test::object_store::{ensure_head_concurrency, make_test_store_and_state}; use tempfile::TempDir; use url::Url; @@ -1353,7 +1354,7 @@ mod tests { // (file_sort_order, expected_result) let cases = vec![ - (vec![], Ok(vec![])), + (vec![], Ok(Vec::::new())), // sort expr, but non column ( vec![vec![ @@ -1364,15 +1365,13 @@ mod tests { // ok with one column ( vec![vec![col("string_col").sort(true, false)]], - Ok(vec![LexOrdering::new( - vec![PhysicalSortExpr { + Ok(vec![[PhysicalSortExpr { expr: physical_col("string_col", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: false, }, - }], - ) + }].into(), ]) ), // ok with two columns, different options @@ -1381,16 +1380,14 @@ mod tests { col("string_col").sort(true, false), col("int_col").sort(false, true), ]], - Ok(vec![LexOrdering::new( - vec![ + Ok(vec![[ PhysicalSortExpr::new_default(physical_col("string_col", &schema).unwrap()) .asc() .nulls_last(), PhysicalSortExpr::new_default(physical_col("int_col", &schema).unwrap()) .desc() .nulls_first() - ], - ) + ].into(), ]) ), ]; diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 92f3755250663..b92dec64e3f19 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -31,7 +31,6 @@ use datafusion::physical_plan::{ sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; @@ -109,13 +108,14 @@ async fn run_merge_test(input: Vec>) { .expect("at least one batch"); let schema = first_batch.schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort = [PhysicalSortExpr { expr: col("x", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let exec = MemorySourceConfig::try_new_exec(&input, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 0b0f0aa2f105a..acdc87197c5fc 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -232,18 +232,15 @@ impl SortTest { .expect("at least one batch"); let schema = first_batch.schema(); - let sort_ordering = LexOrdering::new( - self.sort_columns - .iter() - .map(|c| PhysicalSortExpr { - expr: col(c, &schema).unwrap(), - options: SortOptions { - descending: false, - nulls_first: true, - }, - }) - .collect(), - ); + let sort_ordering = + LexOrdering::new(self.sort_columns.iter().map(|c| PhysicalSortExpr { + expr: col(c, &schema).unwrap(), + options: SortOptions { + descending: false, + nulls_first: true, + }, + })) + .unwrap(); let exec = MemorySourceConfig::try_new_exec(&input, schema, None).unwrap(); let sort = Arc::new(SortExec::new(sort_ordering, exec)); diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 7765185edeb29..772773c91f287 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -621,7 +621,7 @@ async fn run_window_test( } let concat_input_record = concat_batches(&schema, &input1)?; - let source_sort_keys = LexOrdering::new(vec![ + let source_sort_keys: LexOrdering = [ PhysicalSortExpr { expr: col("a", &schema)?, options: Default::default(), @@ -634,7 +634,8 @@ async fn run_window_test( expr: col("c", &schema)?, options: Default::default(), }, - ]); + ] + .into(); let mut exec1 = DataSourceExec::from_data_source( MemorySourceConfig::try_new(&[vec![concat_input_record]], schema.clone(), None)? .try_with_sort_information(vec![source_sort_keys.clone()])?, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 01342d1604fca..a34279e1770a5 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -874,7 +874,7 @@ impl Scenario { descending: false, nulls_first: false, }; - let sort_information = vec![LexOrdering::new(vec![ + let sort_information = vec![[ PhysicalSortExpr { expr: col("a", &schema).unwrap(), options, @@ -883,7 +883,8 @@ impl Scenario { expr: col("b", &schema).unwrap(), options, }, - ])]; + ] + .into()]; let table = SortedTableProvider::new(batches, sort_information); Arc::new(table) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index b967497fb246d..56161b71e8e9b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1730,10 +1730,11 @@ fn smj_join_key_ordering() -> Result<()> { fn merge_does_not_need_sort() -> Result<()> { // see https://github.com/apache/datafusion/issues/4331 let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); // Scan some sorted parquet files let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); @@ -1930,10 +1931,11 @@ fn repartition_unsorted_limit() -> Result<()> { #[test] fn repartition_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); let expected = &[ @@ -1954,10 +1956,11 @@ fn repartition_sorted_limit() -> Result<()> { #[test] fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_required_exec_with_req( filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), sort_key, @@ -2037,10 +2040,11 @@ fn repartition_ignores_union() -> Result<()> { fn repartition_through_sort_preserving_merge() -> Result<()> { // sort preserving merge with non-sorted input let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); // need resort as the data was not sorted correctly @@ -2060,10 +2064,11 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_preserving_merge_exec( sort_key.clone(), parquet_exec_multiple_sorted(vec![sort_key]), @@ -2095,10 +2100,11 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let plan = sort_preserving_merge_exec(sort_key, input); @@ -2133,10 +2139,11 @@ fn repartition_does_not_destroy_sort() -> Result<()> { // SortRequired // Parquet(sorted) let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("d", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_required_exec_with_req( filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), sort_key, @@ -2171,10 +2178,11 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { // Parquet(unsorted) let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input1 = sort_required_exec_with_req( parquet_exec_with_sort(vec![sort_key.clone()]), sort_key, @@ -2207,18 +2215,19 @@ fn repartition_transitively_with_projection() -> Result<()> { let schema = schema(); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), + col("a", &schema)?, Operator::Plus, - col("b", &schema).unwrap(), - )) as Arc, + col("b", &schema)?, + )) as _, "sum".to_string(), )]; // non sorted input let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("sum", &proj.schema()).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("sum", &proj.schema())?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_preserving_merge_exec(sort_key, proj); // Test: run EnforceDistribution, then EnforceSort. @@ -2250,10 +2259,11 @@ fn repartition_transitively_with_projection() -> Result<()> { #[test] fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -2285,10 +2295,11 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { #[test] fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -2320,10 +2331,11 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { #[test] fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); // Test: run EnforceDistribution, then EnforceSort. @@ -2356,10 +2368,11 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = sort_exec( sort_key, projection_exec_with_alias( @@ -2441,10 +2454,11 @@ fn parallelization_single_partition() -> Result<()> { #[test] fn parallelization_multiple_files() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); @@ -2631,10 +2645,11 @@ fn parallelization_two_partitions_into_four() -> Result<()> { #[test] fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); @@ -2674,10 +2689,11 @@ fn parallelization_sorted_limit() -> Result<()> { #[test] fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let plan_parquet = limit_exec(filter_exec(sort_exec( sort_key.clone(), parquet_exec(), @@ -2828,10 +2844,11 @@ fn parallelization_union_inputs() -> Result<()> { #[test] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); // sort preserving merge already sorted input, let plan_parquet = sort_preserving_merge_exec( sort_key.clone(), @@ -2869,10 +2886,11 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { #[test] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let input_parquet = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); @@ -2942,10 +2960,11 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { #[test] fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); // SortRequired // Parquet(sorted) let plan_parquet = sort_required_exec_with_req( @@ -2987,10 +3006,11 @@ fn parallelization_does_not_benefit() -> Result<()> { fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { // sorted input let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -2999,10 +3019,11 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> ]; let proj_parquet = projection_exec_with_alias(parquet_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_parquet.schema()).unwrap(), + let sort_key_after_projection = [PhysicalSortExpr { + expr: col("c2", &proj_parquet.schema())?, options: SortOptions::default(), - }]); + }] + .into(); let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); let expected = &[ @@ -3033,10 +3054,11 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { // sorted input let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -3046,10 +3068,11 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { let proj_csv = projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_csv.schema()).unwrap(), + let sort_key_after_projection = [PhysicalSortExpr { + expr: col("c2", &proj_csv.schema())?, options: SortOptions::default(), - }]); + }] + .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", @@ -3102,10 +3125,11 @@ fn remove_redundant_roundrobins() -> Result<()> { #[test] fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -3132,10 +3156,11 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { #[test] fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("d", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -3157,10 +3182,11 @@ fn preserve_ordering_through_repartition() -> Result<()> { #[test] fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -3196,10 +3222,11 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -3221,16 +3248,18 @@ fn no_need_for_sort_after_filter() -> Result<()> { #[test] fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key]); - let sort_req = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_req = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); let test_config = TestConfig::default(); @@ -3266,10 +3295,11 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { #[test] fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key]); let physical_plan = filter_exec(input); @@ -3288,10 +3318,11 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { #[test] fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec(); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); let expected = &[ @@ -3325,10 +3356,11 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { #[test] fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), + let sort_key: LexOrdering = [PhysicalSortExpr { + expr: col("a", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); @@ -3362,10 +3394,11 @@ fn put_sort_when_input_is_valid() -> Result<()> { #[test] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_with_sort(vec![sort_key]); let physical_plan = aggregate_exec_with_alias(input, alias); @@ -3388,10 +3421,11 @@ fn do_not_add_unnecessary_hash() -> Result<()> { #[test] fn do_not_add_unnecessary_hash2() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_multiple_sorted(vec![sort_key]); let aggregate = aggregate_exec_with_alias(input, alias.clone()); diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index b2c3fa4b4d9c6..a6ebc3f3ab920 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -43,7 +43,7 @@ use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexRequirement, PhysicalSortExpr, OrderingRequirements + LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, OrderingRequirements }; use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; @@ -197,24 +197,26 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let left_schema = create_test_schema2()?; let right_schema = create_test_schema3()?; let left_input = memory_exec(&left_schema); - let parquet_sort_exprs = vec![sort_expr("a", &right_schema)]; + let parquet_sort_exprs = [sort_expr("a", &right_schema)]; let right_input = parquet_exec_sorted(&right_schema, parquet_sort_exprs); - let on = vec![( Arc::new(Column::new_with_schema("col_a", &left_schema)?) as _, Arc::new(Column::new_with_schema("c", &right_schema)?) as _, )]; let join = hash_join_exec(left_input, right_input, on, None, &JoinType::Inner)?; - let physical_plan = sort_exec(vec![sort_expr("a", &join.schema())], join); + let physical_plan = sort_exec([sort_expr("a", &join.schema())].into(), join); - let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", " DataSourceExec: partitions=1, partition_sizes=[0]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; - - let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; + let expected_optimized = [ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", " DataSourceExec: partitions=1, partition_sizes=[0]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -223,33 +225,33 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { #[tokio::test] async fn test_do_not_remove_sort_with_limit() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs = vec![ + let ordering: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort = sort_exec(sort_exprs.clone(), source1); + ] + .into(); + let sort = sort_exec(ordering.clone(), source1); let limit = limit_exec(sort); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - let union = union_exec(vec![source2, limit]); let repartition = repartition_exec(union); - let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); + let physical_plan = sort_preserving_merge_exec(ordering, repartition); - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; // We should keep the bottom `SortExec`. - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", @@ -257,7 +259,8 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -266,18 +269,15 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { #[tokio::test] async fn test_union_inputs_sorted() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source1); - - let source2 = parquet_exec_sorted(&schema, sort_exprs.clone()); - + let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let sort = sort_exec(ordering.clone(), source1); + let source2 = parquet_exec_sorted(&schema, ordering.clone()); let union = union_exec(vec![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs, union); + let physical_plan = sort_preserving_merge_exec(ordering, union); // one input to the union is already sorted, one is not. - let expected_input = vec![ + let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", @@ -285,8 +285,7 @@ async fn test_union_inputs_sorted() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed - let expected_optimized = expected_input.clone(); - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!(expected_input, expected_input, physical_plan, true); Ok(()) } @@ -294,22 +293,19 @@ async fn test_union_inputs_sorted() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source1); - - let parquet_sort_exprs = vec![ + let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let sort = sort_exec(ordering.clone(), source1); + let parquet_sort_exprs = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - let union = union_exec(vec![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs, union); + let physical_plan = sort_preserving_merge_exec(ordering, union); // one input to the union is already sorted, one is not. - let expected_input = vec![ + let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", @@ -317,8 +313,7 @@ async fn test_union_inputs_different_sorted() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed - let expected_optimized = expected_input.clone(); - assert_optimized!(expected_input, expected_optimized, physical_plan, true); + assert_optimized!(expected_input, expected_input, physical_plan, true); Ok(()) } @@ -326,35 +321,36 @@ async fn test_union_inputs_different_sorted() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted2() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs = vec![ + let sort_exprs: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; + ] + .into(); let sort = sort_exec(sort_exprs.clone(), source1); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - let union = union_exec(vec![source2, sort]); let physical_plan = sort_preserving_merge_exec(sort_exprs, union); // Input is an invalid plan. In this case rule should add required sorting in appropriate places. // First DataSourceExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -363,40 +359,42 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted3() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let sort2 = sort_exec(sort_exprs2, source1); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); - + ] + .into(); + let sort1 = sort_exec(ordering1, source1.clone()); + let ordering2 = [sort_expr("nullable_col", &schema)].into(); + let sort2 = sort_exec(ordering2, source1); + let parquet_ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let source2 = parquet_exec_sorted(&schema, parquet_ordering.clone()); let union = union_exec(vec![sort1, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + let physical_plan = sort_preserving_merge_exec(parquet_ordering, union); // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -405,40 +403,42 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted4() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); - let sort2 = sort_exec(sort_exprs2.clone(), source1); - - let source2 = parquet_exec_sorted(&schema, sort_exprs2); - + ] + .into(); + let ordering2: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let sort1 = sort_exec(ordering2.clone(), source1.clone()); + let sort2 = sort_exec(ordering2.clone(), source1); + let source2 = parquet_exec_sorted(&schema, ordering2); let union = union_exec(vec![sort1, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); + let physical_plan = sort_preserving_merge_exec(ordering1, union); // Ordering requirement of the `SortPreservingMergeExec` is not met. // Should modify the plan to ensure that all three inputs to the // `UnionExec` satisfy the ordering, OR add a single sort after // the `UnionExec` (both of which are equally good for this example). - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -447,13 +447,13 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted5() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ + ] + .into(); + let ordering2 = [ sort_expr("nullable_col", &schema), sort_expr_options( "non_nullable_col", @@ -463,29 +463,33 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { nulls_first: false, }, ), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort2 = sort_exec(sort_exprs2, source1); - + ] + .into(); + let ordering3 = [sort_expr("nullable_col", &schema)].into(); + let sort1 = sort_exec(ordering1, source1.clone()); + let sort2 = sort_exec(ordering2, source1); let union = union_exec(vec![sort1, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + let physical_plan = sort_preserving_merge_exec(ordering3, union); // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. However, we should be able to change the unnecessarily // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -494,22 +498,20 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted6() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort_exprs2 = vec![ + let ordering1 = [sort_expr("nullable_col", &schema)].into(); + let sort1 = sort_exec(ordering1, source1.clone()); + let ordering2 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; + ] + .into(); let repartition = repartition_exec(source1); - let spm = sort_preserving_merge_exec(sort_exprs2, repartition); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); - + let spm = sort_preserving_merge_exec(ordering2, repartition); + let parquet_ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let source2 = parquet_exec_sorted(&schema, parquet_ordering.clone()); let union = union_exec(vec![sort1, source2, spm]); - let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + let physical_plan = sort_preserving_merge_exec(parquet_ordering, union); // The plan is not valid as it is -- the input ordering requirement // of the `SortPreservingMergeExec` under the third child of the @@ -517,24 +519,28 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { // At the same time, this ordering requirement is unnecessarily fine. // The final plan should be valid AND the ordering of the third child // shouldn't be finer than necessary. - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -545,31 +551,36 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { let schema = create_test_schema()?; let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1.clone(), source1.clone()); - let sort2 = sort_exec(sort_exprs1, source1); + ] + .into(); + let sort1 = sort_exec(ordering1.clone(), source1.clone()); + let sort2 = sort_exec(ordering1, source1); let union = union_exec(vec![sort1, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + let ordering2 = [sort_expr("nullable_col", &schema)].into(); + let physical_plan = sort_preserving_merge_exec(ordering2, union); // Union has unnecessarily fine ordering below it. We should be able to replace them with absolutely necessary ordering. - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec - let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_output = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -580,11 +591,12 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { let schema = create_test_schema()?; let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ + ] + .into(); + let ordering2 = [ sort_expr_options( "nullable_col", &schema, @@ -601,24 +613,28 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { nulls_first: false, }, ), - ]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort2 = sort_exec(sort_exprs2, source1); - + ] + .into(); + let sort1 = sort_exec(ordering1, source1.clone()); + let sort2 = sort_exec(ordering2, source1); let physical_plan = union_exec(vec![sort1, sort2]); // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. - let expected_input = ["UnionExec", + let expected_input = [ + "UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. - let expected_optimized = ["UnionExec", + let expected_optimized = [ + "UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -628,22 +644,20 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { async fn test_soft_hard_requirements_remove_soft_requirement() -> Result<()> { let schema = create_test_schema()?; let source = parquet_exec_sorted(&schema, vec![]); - - let sort_exprs = vec![sort_expr_options( + let sort_exprs = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), Arc::clone(&source)); + )] + .into(); + let sort = sort_exec(sort_exprs, source); let partition_bys = &[col("nullable_col", &schema)?]; - let bounded_window = + let physical_plan = bounded_window_exec_with_partition("nullable_col", vec![], partition_bys, sort); - let physical_plan = bounded_window; - let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", @@ -658,8 +672,7 @@ async fn test_soft_hard_requirements_remove_soft_requirement() -> Result<()> { "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - ] - ; + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -669,29 +682,28 @@ async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( ) -> Result<()> { let schema = create_test_schema()?; let source = parquet_exec_sorted(&schema, vec![]); - - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source.clone()); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("nullable_col", &schema).unwrap(), + col("nullable_col", &schema)?, Operator::Plus, - col("non_nullable_col", &schema).unwrap(), - )) as Arc, + col("non_nullable_col", &schema)?, + )) as _, "count".to_string(), )]; let partition_bys = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition("nullable_col", vec![], partition_bys, sort); - let projection = projection_exec(proj_exprs, bounded_window)?; - let physical_plan = projection; + let physical_plan = projection_exec(proj_exprs, bounded_window)?; let expected_input = [ "ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as count]", @@ -713,32 +725,32 @@ async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("nullable_col", &schema).unwrap(), + col("nullable_col", &schema)?, Operator::Plus, - col("non_nullable_col", &schema).unwrap(), - )) as Arc, + col("non_nullable_col", &schema)?, + )) as _, "nullable_col".to_string(), )]; let partition_bys = &[col("nullable_col", &schema)?]; let projection = projection_exec(proj_exprs, sort)?; - let bounded_window = bounded_window_exec_with_partition( + let physical_plan = bounded_window_exec_with_partition( "nullable_col", vec![], partition_bys, projection, ); - let physical_plan = bounded_window; let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -767,22 +779,22 @@ async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> { let schema = create_test_schema()?; let source = parquet_exec_sorted(&schema, vec![]); - - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source.clone()); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("nullable_col", &schema).unwrap(), + col("nullable_col", &schema)?, Operator::Plus, - col("non_nullable_col", &schema).unwrap(), - )) as Arc, + col("non_nullable_col", &schema)?, + )) as _, "nullable_col".to_string(), )]; let partition_bys = &[col("nullable_col", &schema)?]; @@ -793,13 +805,12 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> partition_bys, projection, ); - let bounded_window2 = bounded_window_exec_with_partition( + let physical_plan = bounded_window_exec_with_partition( "count", vec![], partition_bys, bounded_window, ); - let physical_plan = bounded_window2; let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -825,21 +836,22 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("nullable_col", &schema).unwrap(), + col("nullable_col", &schema)?, Operator::Plus, - col("non_nullable_col", &schema).unwrap(), - )) as Arc, + col("non_nullable_col", &schema)?, + )) as _, "nullable_col".to_string(), )]; let partition_bys = &[col("nullable_col", &schema)?]; @@ -851,19 +863,19 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> projection, ); - let sort_exprs2 = vec![sort_expr_options( + let ordering2: LexOrdering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort2 = sort_exec(sort_exprs2.clone(), bounded_window.clone()); - let sort3 = sort_exec(sort_exprs2.clone(), sort2); - let bounded_window2 = + )] + .into(); + let sort2 = sort_exec(ordering2.clone(), bounded_window); + let sort3 = sort_exec(ordering2, sort2); + let physical_plan = bounded_window_exec_with_partition("count", vec![], partition_bys, sort3); - let physical_plan = bounded_window2; let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -888,8 +900,7 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> " ProjectionExec: expr=[nullable_col@0 + non_nullable_col@1 as nullable_col]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - ] - ; + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) } @@ -897,22 +908,22 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> async fn test_soft_hard_requirements_multiple_sorts() -> Result<()> { let schema = create_test_schema()?; let source = parquet_exec_sorted(&schema, vec![]); - - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source); let proj_exprs = vec![( Arc::new(BinaryExpr::new( - col("nullable_col", &schema).unwrap(), + col("nullable_col", &schema)?, Operator::Plus, - col("non_nullable_col", &schema).unwrap(), - )) as Arc, + col("non_nullable_col", &schema)?, + )) as _, "nullable_col".to_string(), )]; let partition_bys = &[col("nullable_col", &schema)?]; @@ -923,18 +934,17 @@ async fn test_soft_hard_requirements_multiple_sorts() -> Result<()> { partition_bys, projection, ); - - let sort_exprs2 = vec![sort_expr_options( + let ordering2: LexOrdering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort2 = sort_exec(sort_exprs2.clone(), bounded_window.clone()); - let sort3 = sort_exec(sort_exprs2.clone(), sort2); - let physical_plan = sort3; + )] + .into(); + let sort2 = sort_exec(ordering2.clone(), bounded_window); + let physical_plan = sort_exec(ordering2, sort2); let expected_input = [ "SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", @@ -967,28 +977,19 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ ) -> Result<()> { let schema = create_test_schema()?; let source = parquet_exec_sorted(&schema, vec![]); - - let sort_exprs1 = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs1.clone(), source.clone()); + )] + .into(); + let sort = sort_exec(ordering, source); let partition_bys1 = &[col("nullable_col", &schema)?]; let bounded_window = bounded_window_exec_with_partition("nullable_col", vec![], partition_bys1, sort); - - let sort_exprs2 = vec![sort_expr_options( - "non_nullable_col", - &schema, - SortOptions { - descending: false, - nulls_first: true, - }, - )]; let partition_bys2 = &[col("non_nullable_col", &schema)?]; let bounded_window2 = bounded_window_exec_with_partition( "non_nullable_col", @@ -996,15 +997,19 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ partition_bys2, bounded_window, ); - let output_requirements: Arc = - Arc::new(OutputRequirementExec::new( - bounded_window2, - Some(OrderingRequirements::new_single(LexRequirement::from( - LexOrdering::from(sort_exprs2), - ))), - Distribution::SinglePartition, - )); - let physical_plan = output_requirements; + let requirement = [PhysicalSortRequirement { + expr: col("non_nullable_col", &schema)?, + options: Some(SortOptions { + descending: false, + nulls_first: true, + }), + }] + .into(); + let physical_plan = Arc::new(OutputRequirementExec::new( + bounded_window2, + Some(OrderingRequirements::new_single(requirement)), + Distribution::SinglePartition, + )); let expected_input = [ "OutputRequirementExec", @@ -1036,29 +1041,28 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ #[tokio::test] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; - - let sort_exprs1 = vec![ + let sort_exprs1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - // reverse sorting of sort_exprs2 - let sort_exprs3 = vec![sort_expr_options( + let sort_exprs2 = [sort_expr("nullable_col", &schema)]; + // Reverse of the above + let ordering: LexOrdering = [sort_expr_options( "nullable_col", &schema, SortOptions { descending: true, nulls_first: false, }, - )]; + )] + .into(); let source1 = parquet_exec_sorted(&schema, sort_exprs1); let source2 = parquet_exec_sorted(&schema, sort_exprs2); - let sort1 = sort_exec(sort_exprs3.clone(), source1); - let sort2 = sort_exec(sort_exprs3.clone(), source2); - + let sort1 = sort_exec(ordering.clone(), source1); + let sort2 = sort_exec(ordering.clone(), source2); let union = union_exec(vec![sort1, sort2]); - let spm = sort_preserving_merge_exec(sort_exprs3.clone(), union); - let physical_plan = bounded_window_exec("nullable_col", sort_exprs3, spm); + let spm = sort_preserving_merge_exec(ordering.clone(), union); + let physical_plan = bounded_window_exec("nullable_col", ordering, spm); // The `WindowAggExec` gets its sorting from multiple children jointly. // During the removal of `SortExec`s, it should be able to remove the @@ -1071,13 +1075,15 @@ async fn test_window_multi_path_sort() -> Result<()> { " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + ]; let expected_optimized = [ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1086,35 +1092,38 @@ async fn test_window_multi_path_sort() -> Result<()> { #[tokio::test] async fn test_window_multi_path_sort2() -> Result<()> { let schema = create_test_schema()?; - - let sort_exprs1 = LexOrdering::new(vec![ + let ordering1: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]); - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let source1 = parquet_exec_sorted(&schema, sort_exprs2.clone()); - let source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); - let sort1 = sort_exec(sort_exprs1.clone(), source1); - let sort2 = sort_exec(sort_exprs1.clone(), source2); - + ] + .into(); + let ordering2: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let source1 = parquet_exec_sorted(&schema, ordering2.clone()); + let source2 = parquet_exec_sorted(&schema, ordering2.clone()); + let sort1 = sort_exec(ordering1.clone(), source1); + let sort2 = sort_exec(ordering1.clone(), source2); let union = union_exec(vec![sort1, sort2]); - let spm = Arc::new(SortPreservingMergeExec::new(sort_exprs1, union)) as _; - let physical_plan = bounded_window_exec("nullable_col", sort_exprs2, spm); + let spm = Arc::new(SortPreservingMergeExec::new(ordering1, union)) as _; + let physical_plan = bounded_window_exec("nullable_col", ordering2, spm); // The `WindowAggExec` can get its required sorting from the leaf nodes directly. // The unnecessary SortExecs should be removed - let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; - let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1123,13 +1132,13 @@ async fn test_window_multi_path_sort2() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ + ] + .into(); + let ordering2 = [ sort_expr("nullable_col", &schema), sort_expr_options( "non_nullable_col", @@ -1139,34 +1148,37 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { nulls_first: false, }, ), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - - let sort2 = sort_exec(sort_exprs2, source1); + ] + .into(); + let sort1 = sort_exec(ordering1, source1.clone()); + let sort2 = sort_exec(ordering2, source1); let limit = local_limit_exec(sort2); let limit = global_limit_exec(limit); - let union = union_exec(vec![sort1, limit]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + let ordering3 = [sort_expr("nullable_col", &schema)].into(); + let physical_plan = sort_preserving_merge_exec(ordering3, union); // Should not change the unnecessarily fine `SortExec`s because there is `LimitExec` - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1182,8 +1194,8 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { // Join on (nullable_col == col_a) let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("nullable_col", &left.schema())?) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema())?) as _, )]; let join_types = vec![ @@ -1197,11 +1209,12 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { for join_type in join_types { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let sort_exprs = vec![ + let ordering = [ sort_expr("nullable_col", &join.schema()), sort_expr("non_nullable_col", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs.clone(), join); + ] + .into(); + let physical_plan = sort_preserving_merge_exec(ordering, join); let join_plan = format!( "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" @@ -1254,8 +1267,8 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { // Join on (nullable_col == col_a) let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("nullable_col", &left.schema())?) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema())?) as _, )]; let join_types = vec![ @@ -1268,11 +1281,12 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { for join_type in join_types { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let sort_exprs = vec![ + let ordering = [ sort_expr("col_a", &join.schema()), sort_expr("col_b", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs, join); + ] + .into(); + let physical_plan = sort_preserving_merge_exec(ordering, join); let join_plan = format!( "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" @@ -1326,53 +1340,61 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { // Join on (nullable_col == col_a) let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("nullable_col", &left.schema())?) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema())?) as _, )]; let join = sort_merge_join_exec(left, right, &join_on, &JoinType::Inner); // order by (col_b, col_a) - let sort_exprs1 = vec![ + let ordering = [ sort_expr("col_b", &join.schema()), sort_expr("col_a", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs1, join.clone()); + ] + .into(); + let physical_plan = sort_preserving_merge_exec(ordering, join.clone()); - let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; - + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + ]; // can not push down the sort requirements, need to add SortExec - let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", + let expected_optimized = [ + "SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) - let sort_exprs2 = vec![ + let ordering2 = [ sort_expr("nullable_col", &join.schema()), sort_expr("col_b", &join.schema()), sort_expr("col_a", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs2, join); + ] + .into(); + let physical_plan = sort_preserving_merge_exec(ordering2, join); - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; - + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + ]; // can not push down the sort requirements, need to add SortExec - let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1381,33 +1403,34 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { #[tokio::test] async fn test_multilayer_coalesce_partitions() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); let repartition = repartition_exec(source1); let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before let filter = filter_exec( - Arc::new(NotExpr::new( - col("non_nullable_col", schema.as_ref()).unwrap(), - )), + Arc::new(NotExpr::new(col("non_nullable_col", schema.as_ref())?)), coalesce, ); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let physical_plan = sort_exec(sort_exprs, filter); + let ordering = [sort_expr("nullable_col", &schema)].into(); + let physical_plan = sort_exec(ordering, filter); // CoalescePartitionsExec and SortExec are not directly consecutive. In this case // we should be able to parallelize Sorting also (given that executors in between don't require) // single partition. - let expected_input = ["SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1416,26 +1439,30 @@ async fn test_multilayer_coalesce_partitions() -> Result<()> { #[tokio::test] async fn test_with_lost_ordering_bounded() -> Result<()> { let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = csv_exec_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, - Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), + Partitioning::Hash(vec![col("c", &schema)?], 10), )?) as _; let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); + let physical_plan = sort_exec([sort_expr("a", &schema)].into(), coalesce_partitions); - let expected_input = ["SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1447,20 +1474,20 @@ async fn test_with_lost_ordering_unbounded_bounded( #[values(false, true)] source_unbounded: bool, ) -> Result<()> { let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; // create either bounded or unbounded source let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered(&schema, sort_exprs.clone()) } else { - csv_exec_sorted(&schema, sort_exprs) + csv_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, - Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), + Partitioning::Hash(vec![col("c", &schema)?], 10), )?) as _; let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); + let physical_plan = sort_exec(sort_exprs.into(), coalesce_partitions); // Expected inputs unbounded and bounded let expected_input_unbounded = vec![ @@ -1534,20 +1561,24 @@ async fn test_with_lost_ordering_unbounded_bounded( #[tokio::test] async fn test_do_not_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let sort_exprs = [sort_expr("a", &schema), sort_expr("b", &schema)]; let source = csv_exec_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); - let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); - let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); + let spm = sort_preserving_merge_exec(sort_exprs.into(), repartition_rr); + let physical_plan = sort_exec([sort_expr("b", &schema)].into(), spm); - let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; - let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false", + ]; + let expected_optimized = [ + "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1556,27 +1587,31 @@ async fn test_do_not_pushdown_through_spm() -> Result<()> { #[tokio::test] async fn test_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let sort_exprs = [sort_expr("a", &schema), sort_expr("b", &schema)]; let source = csv_exec_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); - let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); + let spm = sort_preserving_merge_exec(sort_exprs.into(), repartition_rr); let physical_plan = sort_exec( - vec![ + [ sort_expr("a", &schema), sort_expr("b", &schema), sort_expr("c", &schema), - ], + ] + .into(), spm, ); - let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false", + ]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1585,13 +1620,12 @@ async fn test_pushdown_through_spm() -> Result<()> { #[tokio::test] async fn test_window_multi_layer_requirement() -> Result<()> { let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let sort_exprs = [sort_expr("a", &schema), sort_expr("b", &schema)]; let source = csv_exec_sorted(&schema, vec![]); - let sort = sort_exec(sort_exprs.clone(), source); + let sort = sort_exec(sort_exprs.clone().into(), source); let repartition = repartition_exec(sort); let repartition = spr_repartition_exec(repartition); - let spm = sort_preserving_merge_exec(sort_exprs.clone(), repartition); - + let spm = sort_preserving_merge_exec(sort_exprs.clone().into(), repartition); let physical_plan = bounded_window_exec("a", sort_exprs, spm); let expected_input = [ @@ -1617,15 +1651,15 @@ async fn test_window_multi_layer_requirement() -> Result<()> { #[tokio::test] async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; let parquet_input = parquet_exec_sorted(&schema, input_sort_exprs); - let physical_plan = sort_exec( - vec![ + [ sort_expr("a", &schema), sort_expr("b", &schema), sort_expr("c", &schema), - ], + ] + .into(), parquet_input, ); let expected_input = [ @@ -1729,8 +1763,8 @@ macro_rules! assert_optimized { async fn test_remove_unnecessary_sort() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); + let input = sort_exec([sort_expr("non_nullable_col", &schema)].into(), source); + let physical_plan = sort_exec([sort_expr("nullable_col", &schema)].into(), input); let expected_input = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -1750,57 +1784,54 @@ async fn test_remove_unnecessary_sort() -> Result<()> { async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr_options( + let ordering: LexOrdering = [sort_expr_options( "non_nullable_col", &source.schema(), SortOptions { descending: true, nulls_first: true, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source); + )] + .into(); + let sort = sort_exec(ordering.clone(), source); // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before let coalesce_batches = coalesce_batches_exec(sort); - - let window_agg = - bounded_window_exec("non_nullable_col", sort_exprs, coalesce_batches); - - let sort_exprs = vec![sort_expr_options( + let window_agg = bounded_window_exec("non_nullable_col", ordering, coalesce_batches); + let ordering2: LexOrdering = [sort_expr_options( "non_nullable_col", &window_agg.schema(), SortOptions { descending: false, nulls_first: false, }, - )]; - - let sort = sort_exec(sort_exprs.clone(), window_agg); - + )] + .into(); + let sort = sort_exec(ordering2.clone(), window_agg); // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before let filter = filter_exec( - Arc::new(NotExpr::new( - col("non_nullable_col", schema.as_ref()).unwrap(), - )), + Arc::new(NotExpr::new(col("non_nullable_col", schema.as_ref())?)), sort, ); + let physical_plan = bounded_window_exec("non_nullable_col", ordering2, filter); - let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs, filter); - - let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " FilterExec: NOT non_nullable_col@1", " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]" + ]; - let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", + let expected_optimized = [ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " FilterExec: NOT non_nullable_col@1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]" + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1810,10 +1841,8 @@ async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { async fn test_add_required_sort() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - - let physical_plan = sort_preserving_merge_exec(sort_exprs, source); + let ordering = [sort_expr("nullable_col", &schema)].into(); + let physical_plan = sort_preserving_merge_exec(ordering, source); let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -1832,13 +1861,12 @@ async fn test_add_required_sort() -> Result<()> { async fn test_remove_unnecessary_sort1() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); + let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); + let sort = sort_exec(ordering.clone(), source); + let spm = sort_preserving_merge_exec(ordering.clone(), sort); + let sort = sort_exec(ordering.clone(), spm); + let physical_plan = sort_preserving_merge_exec(ordering, sort); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), spm); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -1859,19 +1887,18 @@ async fn test_remove_unnecessary_sort1() -> Result<()> { async fn test_remove_unnecessary_sort2() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let sort_exprs = vec![ + let ordering: LexOrdering = [sort_expr("non_nullable_col", &schema)].into(); + let sort = sort_exec(ordering.clone(), source); + let spm = sort_preserving_merge_exec(ordering, sort); + let ordering2: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort2 = sort_exec(sort_exprs.clone(), spm); - let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); - - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort3 = sort_exec(sort_exprs, spm2); + ] + .into(); + let sort2 = sort_exec(ordering2.clone(), spm); + let spm2 = sort_preserving_merge_exec(ordering2, sort2); + let ordering3 = [sort_expr("nullable_col", &schema)].into(); + let sort3 = sort_exec(ordering3, spm2); let physical_plan = repartition_exec(repartition_exec(sort3)); let expected_input = [ @@ -1884,7 +1911,6 @@ async fn test_remove_unnecessary_sort2() -> Result<()> { " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: partitions=1, partition_sizes=[0]", ]; - let expected_optimized = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1899,21 +1925,20 @@ async fn test_remove_unnecessary_sort2() -> Result<()> { async fn test_remove_unnecessary_sort3() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let sort_exprs = LexOrdering::new(vec![ + let ordering: LexOrdering = [sort_expr("non_nullable_col", &schema)].into(); + let sort = sort_exec(ordering.clone(), source); + let spm = sort_preserving_merge_exec(ordering, sort); + let repartition_exec = repartition_exec(spm); + let ordering2: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]); - let repartition_exec = repartition_exec(spm); + ] + .into(); let sort2 = Arc::new( - SortExec::new(sort_exprs.clone(), repartition_exec) + SortExec::new(ordering2.clone(), repartition_exec) .with_preserve_partitioning(true), ) as _; - let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); - + let spm2 = sort_preserving_merge_exec(ordering2, sort2); let physical_plan = aggregate_exec(spm2); // When removing a `SortPreservingMergeExec`, make sure that partitioning @@ -1928,7 +1953,6 @@ async fn test_remove_unnecessary_sort3() -> Result<()> { " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " DataSourceExec: partitions=1, partition_sizes=[0]", ]; - let expected_optimized = [ "AggregateExec: mode=Final, gby=[], aggr=[]", " CoalescePartitionsExec", @@ -1944,34 +1968,29 @@ async fn test_remove_unnecessary_sort3() -> Result<()> { async fn test_remove_unnecessary_sort4() -> Result<()> { let schema = create_test_schema()?; let source1 = repartition_exec(memory_exec(&schema)); - let source2 = repartition_exec(memory_exec(&schema)); let union = union_exec(vec![source1, source2]); - - let sort_exprs = LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]); - // let sort = sort_exec(sort_exprs.clone(), union); - let sort = Arc::new( - SortExec::new(sort_exprs.clone(), union).with_preserve_partitioning(true), - ) as _; - let spm = sort_preserving_merge_exec(sort_exprs, sort); - + let ordering: LexOrdering = [sort_expr("non_nullable_col", &schema)].into(); + let sort = + Arc::new(SortExec::new(ordering.clone(), union).with_preserve_partitioning(true)) + as _; + let spm = sort_preserving_merge_exec(ordering, sort); let filter = filter_exec( - Arc::new(NotExpr::new( - col("non_nullable_col", schema.as_ref()).unwrap(), - )), + Arc::new(NotExpr::new(col("non_nullable_col", schema.as_ref())?)), spm, ); - - let sort_exprs = vec![ + let ordering2 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let physical_plan = sort_exec(sort_exprs, filter); + ] + .into(); + let physical_plan = sort_exec(ordering2, filter); // When removing a `SortPreservingMergeExec`, make sure that partitioning // requirements are not violated. In some cases, we may need to replace // it with a `CoalescePartitionsExec` instead of directly removing it. - let expected_input = ["SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " FilterExec: NOT non_nullable_col@1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", @@ -1979,16 +1998,18 @@ async fn test_remove_unnecessary_sort4() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; - - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1998,18 +2019,17 @@ async fn test_remove_unnecessary_sort4() -> Result<()> { async fn test_remove_unnecessary_sort6() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - ) - .with_fetch(Some(2)), + let input = sort_exec_with_fetch( + [sort_expr("non_nullable_col", &schema)].into(), + Some(2), + source, ); let physical_plan = sort_exec( - vec![ + [ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ], + ] + .into(), input, ); @@ -2031,21 +2051,19 @@ async fn test_remove_unnecessary_sort6() -> Result<()> { async fn test_remove_unnecessary_sort7() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![ + let input = sort_exec( + [ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ]), + ] + .into(), source, - )); - - let physical_plan = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - input, - ) - .with_fetch(Some(2)), - ) as Arc; + ); + let physical_plan = sort_exec_with_fetch( + [sort_expr("non_nullable_col", &schema)].into(), + Some(2), + input, + ); let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false], sort_prefix=[non_nullable_col@1 ASC]", @@ -2066,16 +2084,14 @@ async fn test_remove_unnecessary_sort7() -> Result<()> { async fn test_remove_unnecessary_sort8() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); + let input = sort_exec([sort_expr("non_nullable_col", &schema)].into(), source); let limit = Arc::new(LocalLimitExec::new(input, 2)); let physical_plan = sort_exec( - vec![ + [ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ], + ] + .into(), limit, ); @@ -2099,13 +2115,9 @@ async fn test_remove_unnecessary_sort8() -> Result<()> { async fn test_do_not_pushdown_through_limit() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); + let input = sort_exec([sort_expr("non_nullable_col", &schema)].into(), source); let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); + let physical_plan = sort_exec([sort_expr("nullable_col", &schema)].into(), limit); let expected_input = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -2128,12 +2140,11 @@ async fn test_do_not_pushdown_through_limit() -> Result<()> { async fn test_remove_unnecessary_spm1() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input2 = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], input); + let ordering: LexOrdering = [sort_expr("non_nullable_col", &schema)].into(); + let input = sort_preserving_merge_exec(ordering.clone(), source); + let input2 = sort_preserving_merge_exec(ordering, input); let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("nullable_col", &schema)], input2); + sort_preserving_merge_exec([sort_expr("nullable_col", &schema)].into(), input2); let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -2155,7 +2166,7 @@ async fn test_remove_unnecessary_spm2() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = sort_preserving_merge_exec_with_fetch( - vec![sort_expr("non_nullable_col", &schema)], + [sort_expr("non_nullable_col", &schema)].into(), source, 100, ); @@ -2178,12 +2189,13 @@ async fn test_remove_unnecessary_spm2() -> Result<()> { async fn test_change_wrong_sorting() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let sort_exprs = vec![ + let sort_exprs = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; - let sort = sort_exec(vec![sort_exprs[0].clone()], source); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); + let sort = sort_exec([sort_exprs[0].clone()].into(), source); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -2202,13 +2214,13 @@ async fn test_change_wrong_sorting() -> Result<()> { async fn test_change_wrong_sorting2() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let sort_exprs = vec![ + let sort_exprs = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; - let spm1 = sort_preserving_merge_exec(sort_exprs.clone(), source); - let sort2 = sort_exec(vec![sort_exprs[0].clone()], spm1); - let physical_plan = sort_preserving_merge_exec(vec![sort_exprs[1].clone()], sort2); + let spm1 = sort_preserving_merge_exec(sort_exprs.clone().into(), source); + let sort2 = sort_exec([sort_exprs[0].clone()].into(), spm1); + let physical_plan = sort_preserving_merge_exec([sort_exprs[1].clone()].into(), sort2); let expected_input = [ "SortPreservingMergeExec: [non_nullable_col@1 ASC]", @@ -2229,31 +2241,31 @@ async fn test_change_wrong_sorting2() -> Result<()> { async fn test_multiple_sort_window_exec() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort_exprs2 = vec![ + let ordering1 = [sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(ordering1.clone().into(), source); + let window_agg1 = bounded_window_exec("non_nullable_col", ordering1.clone(), sort1); + let ordering2 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; + let window_agg2 = bounded_window_exec("non_nullable_col", ordering2, window_agg1); + let physical_plan = bounded_window_exec("non_nullable_col", ordering1, window_agg2); - let sort1 = sort_exec(sort_exprs1.clone(), source); - let window_agg1 = bounded_window_exec("non_nullable_col", sort_exprs1.clone(), sort1); - let window_agg2 = bounded_window_exec("non_nullable_col", sort_exprs2, window_agg1); - // let filter_exec = sort_exec; - let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs1, window_agg2); - - let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + let expected_input = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; - - let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " DataSourceExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -2267,15 +2279,12 @@ async fn test_multiple_sort_window_exec() -> Result<()> { // EnforceDistribution may invalidate ordering invariant. async fn test_commutativity() -> Result<()> { let schema = create_test_schema()?; - let config = ConfigOptions::new(); - let memory_exec = memory_exec(&schema); - let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + let sort_exprs = [sort_expr("nullable_col", &schema)]; let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); let repartition = repartition_exec(window); + let orig_plan = sort_exec(sort_exprs.into(), repartition); - let orig_plan = - Arc::new(SortExec::new(sort_exprs, repartition)) as Arc; let actual = get_plan_string(&orig_plan); let expected_input = vec![ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -2288,26 +2297,25 @@ async fn test_commutativity() -> Result<()> { "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_input:#?}\nactual:\n\n{actual:#?}\n\n" ); - let mut plan = orig_plan.clone(); + let config = ConfigOptions::new(); let rules = vec![ Arc::new(EnforceDistribution::new()) as Arc, Arc::new(EnforceSorting::new()) as Arc, ]; + let mut first_plan = orig_plan.clone(); for rule in rules { - plan = rule.optimize(plan, &config)?; + first_plan = rule.optimize(first_plan, &config)?; } - let first_plan = plan.clone(); - let mut plan = orig_plan.clone(); let rules = vec![ Arc::new(EnforceSorting::new()) as Arc, Arc::new(EnforceDistribution::new()) as Arc, Arc::new(EnforceSorting::new()) as Arc, ]; + let mut second_plan = orig_plan.clone(); for rule in rules { - plan = rule.optimize(plan, &config)?; + second_plan = rule.optimize(second_plan, &config)?; } - let second_plan = plan.clone(); assert_eq!(get_plan_string(&first_plan), get_plan_string(&second_plan)); Ok(()) @@ -2318,15 +2326,15 @@ async fn test_coalesce_propagate() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); let repartition = repartition_exec(source); - let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); + let coalesce_partitions = coalesce_partitions_exec(repartition); let repartition = repartition_exec(coalesce_partitions); - let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); // Add local sort let sort = Arc::new( - SortExec::new(sort_exprs.clone(), repartition).with_preserve_partitioning(true), + SortExec::new(ordering.clone(), repartition).with_preserve_partitioning(true), ) as _; - let spm = sort_preserving_merge_exec(sort_exprs.clone(), sort); - let sort = sort_exec(sort_exprs, spm); + let spm = sort_preserving_merge_exec(ordering.clone(), sort); + let sort = sort_exec(ordering, spm); let physical_plan = sort.clone(); // Sort Parallelize rule should end Coalesce + Sort linkage when Sort is Global Sort @@ -2354,15 +2362,15 @@ async fn test_coalesce_propagate() -> Result<()> { #[tokio::test] async fn test_replace_with_partial_sort2() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("a", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("a", &schema), sort_expr("c", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - let physical_plan = sort_exec( - vec![ + [ sort_expr("a", &schema), sort_expr("c", &schema), sort_expr("d", &schema), - ], + ] + .into(), unbounded_input, ); @@ -2381,65 +2389,55 @@ async fn test_replace_with_partial_sort2() -> Result<()> { #[tokio::test] async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { - // SortExec: expr=[b] <-- can't push this down - // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order - // SortExec: expr=[a] - // DataSourceExec let schema = create_test_schema3()?; - let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); - let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); + let ordering_a: LexOrdering = [sort_expr("a", &schema)].into(); + let ordering_b: LexOrdering = [sort_expr("b", &schema)].into(); let plan = memory_exec(&schema); - let plan = sort_exec(sort_exprs_a.clone(), plan); + let plan = sort_exec(ordering_a.clone(), plan); let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(Some(sort_exprs_a)) + .with_required_input_ordering(Some(ordering_a)) .with_maintains_input_order(true) .into_arc(); - let plan = sort_exec(sort_exprs_b, plan); + let plan = sort_exec(ordering_b, plan); let expected_input = [ - "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " RequiredInputOrderingExec", + "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", // <-- can't push this down + " RequiredInputOrderingExec", // <-- this requires input sorted by a, and preserves the input order " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " DataSourceExec: partitions=1, partition_sizes=[0]", ]; // should not be able to push shorts - let expected_no_change = expected_input; - assert_optimized!(expected_input, expected_no_change, plan, true); + assert_optimized!(expected_input, expected_input, plan, true); Ok(()) } // test when the required input ordering is satisfied so could push through #[tokio::test] async fn test_push_with_required_input_ordering_allowed() -> Result<()> { - // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) - // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order - // SortExec: expr=[a] - // DataSourceExec let schema = create_test_schema3()?; - let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); - let sort_exprs_ab = - LexOrdering::new(vec![sort_expr("a", &schema), sort_expr("b", &schema)]); + let ordering_a: LexOrdering = [sort_expr("a", &schema)].into(); + let ordering_ab = [sort_expr("a", &schema), sort_expr("b", &schema)].into(); let plan = memory_exec(&schema); - let plan = sort_exec(sort_exprs_a.clone(), plan); + let plan = sort_exec(ordering_a.clone(), plan); let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(Some(sort_exprs_a)) + .with_required_input_ordering(Some(ordering_a)) .with_maintains_input_order(true) .into_arc(); - let plan = sort_exec(sort_exprs_ab, plan); + let plan = sort_exec(ordering_ab, plan); let expected_input = [ - "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " RequiredInputOrderingExec", + "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", // <-- can push this down (as it is compatible with the required input ordering) + " RequiredInputOrderingExec", // <-- this requires input sorted by a, and preserves the input order " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " DataSourceExec: partitions=1, partition_sizes=[0]", ]; - // should able to push shorts - let expected = [ + // Should be able to push down + let expected_optimized = [ "RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " DataSourceExec: partitions=1, partition_sizes=[0]", ]; - assert_optimized!(expected_input, expected, plan, true); + assert_optimized!(expected_input, expected_optimized, plan, true); Ok(()) } @@ -2448,9 +2446,8 @@ async fn test_replace_with_partial_sort() -> Result<()> { let schema = create_test_schema3()?; let input_sort_exprs = vec![sort_expr("a", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - let physical_plan = sort_exec( - vec![sort_expr("a", &schema), sort_expr("c", &schema)], + [sort_expr("a", &schema), sort_expr("c", &schema)].into(), unbounded_input, ); @@ -2469,23 +2466,22 @@ async fn test_replace_with_partial_sort() -> Result<()> { #[tokio::test] async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - let physical_plan = sort_exec( - vec![ + [ sort_expr("a", &schema), sort_expr("b", &schema), sort_expr("c", &schema), - ], + ] + .into(), unbounded_input, ); let expected_input = [ "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" ]; - let expected_no_change = expected_input; - assert_optimized!(expected_input, expected_no_change, physical_plan, true); + assert_optimized!(expected_input, expected_input, physical_plan, true); Ok(()) } @@ -3737,7 +3733,8 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { ) }) .collect::>(); - let physical_plan = sort_exec(sort_expr, window_exec); + let ordering = LexOrdering::new(sort_expr).unwrap(); + let physical_plan = sort_exec(ordering, window_exec); assert_optimized!( case.initial_plan, @@ -3754,11 +3751,10 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { #[test] fn test_removes_unused_orthogonal_sort() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); - - let orthogonal_sort = sort_exec(vec![sort_expr("a", &schema)], unbounded_input); - let output_sort = sort_exec(input_sort_exprs, orthogonal_sort); // same sort as data source + let orthogonal_sort = sort_exec([sort_expr("a", &schema)].into(), unbounded_input); + let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort); // same sort as data source // Test scenario/input has an orthogonal sort: let expected_input = [ @@ -3766,7 +3762,7 @@ fn test_removes_unused_orthogonal_sort() -> Result<()> { " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" ]; - assert_eq!(get_plan_string(&output_sort), expected_input,); + assert_eq!(get_plan_string(&output_sort), expected_input); // Test: should remove orthogonal sort, and the uppermost (unneeded) sort: let expected_optimized = [ @@ -3780,12 +3776,11 @@ fn test_removes_unused_orthogonal_sort() -> Result<()> { #[test] fn test_keeps_used_orthogonal_sort() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); - let orthogonal_sort = - sort_exec_with_fetch(vec![sort_expr("a", &schema)], Some(3), unbounded_input); // has fetch, so this orthogonal sort changes the output - let output_sort = sort_exec(input_sort_exprs, orthogonal_sort); + sort_exec_with_fetch([sort_expr("a", &schema)].into(), Some(3), unbounded_input); // has fetch, so this orthogonal sort changes the output + let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort); // Test scenario/input has an orthogonal sort: let expected_input = [ @@ -3793,7 +3788,7 @@ fn test_keeps_used_orthogonal_sort() -> Result<()> { " SortExec: TopK(fetch=3), expr=[a@0 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" ]; - assert_eq!(get_plan_string(&output_sort), expected_input,); + assert_eq!(get_plan_string(&output_sort), expected_input); // Test: should keep the orthogonal sort, since it modifies the output: let expected_optimized = expected_input; @@ -3805,15 +3800,16 @@ fn test_keeps_used_orthogonal_sort() -> Result<()> { #[test] fn test_handles_multiple_orthogonal_sorts() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); - - let orthogonal_sort_0 = sort_exec(vec![sort_expr("c", &schema)], unbounded_input); // has no fetch, so can be removed + let ordering0: LexOrdering = [sort_expr("c", &schema)].into(); + let orthogonal_sort_0 = sort_exec(ordering0.clone(), unbounded_input); // has no fetch, so can be removed + let ordering1: LexOrdering = [sort_expr("a", &schema)].into(); let orthogonal_sort_1 = - sort_exec_with_fetch(vec![sort_expr("a", &schema)], Some(3), orthogonal_sort_0); // has fetch, so this orthogonal sort changes the output - let orthogonal_sort_2 = sort_exec(vec![sort_expr("c", &schema)], orthogonal_sort_1); // has no fetch, so can be removed - let orthogonal_sort_3 = sort_exec(vec![sort_expr("a", &schema)], orthogonal_sort_2); // has no fetch, so can be removed - let output_sort = sort_exec(input_sort_exprs, orthogonal_sort_3); // final sort + sort_exec_with_fetch(ordering1.clone(), Some(3), orthogonal_sort_0); // has fetch, so this orthogonal sort changes the output + let orthogonal_sort_2 = sort_exec(ordering0, orthogonal_sort_1); // has no fetch, so can be removed + let orthogonal_sort_3 = sort_exec(ordering1, orthogonal_sort_2); // has no fetch, so can be removed + let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort_3); // final sort // Test scenario/input has an orthogonal sort: let expected_input = [ @@ -3824,7 +3820,7 @@ fn test_handles_multiple_orthogonal_sorts() -> Result<()> { " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]", ]; - assert_eq!(get_plan_string(&output_sort), expected_input,); + assert_eq!(get_plan_string(&output_sort), expected_input); // Test: should keep only the needed orthogonal sort, and remove the unneeded ones: let expected_optimized = [ diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index f9810eab8f594..c04a75e00b232 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -30,9 +30,8 @@ use datafusion::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::cast; -use datafusion_physical_expr::{expressions, expressions::col, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr::expressions::{self, cast, col}; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode}, collect, @@ -236,10 +235,11 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { #[test] fn test_has_order_by() -> Result<()> { - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema()).unwrap(), + let sort_key = [PhysicalSortExpr { + expr: col("a", &schema())?, options: SortOptions::default(), - }]); + }] + .into(); let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index ce9f6dc0614ce..a967b6df5713f 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -39,8 +39,7 @@ use datafusion_physical_expr::expressions::{ use datafusion_physical_expr::{Distribution, Partitioning, ScalarFunctionExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, - PhysicalSortRequirement, + LexRequirement, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; @@ -518,7 +517,7 @@ fn test_streaming_table_after_projection() -> Result<()> { }) as _], Some(&vec![0_usize, 2, 4, 3]), vec![ - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: Arc::new(Column::new("e", 2)), options: SortOptions::default(), @@ -527,11 +526,13 @@ fn test_streaming_table_after_projection() -> Result<()> { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), }, - ]), - LexOrdering::new(vec![PhysicalSortExpr { + ] + .into(), + [PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), options: SortOptions::default(), - }]), + }] + .into(), ] .into_iter(), true, @@ -578,7 +579,7 @@ fn test_streaming_table_after_projection() -> Result<()> { assert_eq!( result.projected_output_ordering().into_iter().collect_vec(), vec![ - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: Arc::new(Column::new("e", 1)), options: SortOptions::default(), @@ -587,11 +588,13 @@ fn test_streaming_table_after_projection() -> Result<()> { expr: Arc::new(Column::new("a", 2)), options: SortOptions::default(), }, - ]), - LexOrdering::new(vec![PhysicalSortExpr { + ] + .into(), + [PhysicalSortExpr { expr: Arc::new(Column::new("d", 0)), options: SortOptions::default(), - }]), + }] + .into(), ] ); assert!(result.is_infinite()); @@ -1256,8 +1259,8 @@ fn test_repartition_after_projection() -> Result<()> { #[test] fn test_sort_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortExec::new( - LexOrdering::new(vec![ + let sort_exec = SortExec::new( + [ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -1270,17 +1273,18 @@ fn test_sort_after_projection() -> Result<()> { )), options: SortOptions::default(), }, - ]), - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( + ] + .into(), + csv, + ); + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c".to_string()), (Arc::new(Column::new("a", 0)), "new_a".to_string()), (Arc::new(Column::new("b", 1)), "b".to_string()), ], - sort_req.clone(), - )?); + Arc::new(sort_exec), + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ @@ -1306,8 +1310,8 @@ fn test_sort_after_projection() -> Result<()> { #[test] fn test_sort_preserving_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - LexOrdering::new(vec![ + let sort_exec = SortPreservingMergeExec::new( + [ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -1320,17 +1324,18 @@ fn test_sort_preserving_after_projection() -> Result<()> { )), options: SortOptions::default(), }, - ]), - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( + ] + .into(), + csv, + ); + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c".to_string()), (Arc::new(Column::new("a", 0)), "new_a".to_string()), (Arc::new(Column::new("b", 1)), "b".to_string()), ], - sort_req.clone(), - )?); + Arc::new(sort_exec), + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index f69299d5701eb..5900e0a41ac86 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -197,7 +197,8 @@ async fn test_replace_multiple_input_repartition_1( let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); - let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + let physical_plan = + sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -276,8 +277,10 @@ async fn test_with_inter_children_change_only( let filter = filter_exec(repartition_hash2); let sort2 = sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &sort2.schema())], sort2); + let physical_plan = sort_preserving_merge_exec( + [sort_expr_default("a", &sort2.schema())].into(), + sort2, + ); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -371,7 +374,8 @@ async fn test_replace_multiple_input_repartition_2( let repartition_hash = repartition_exec_hash(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); - let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + let physical_plan = + sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -448,7 +452,8 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); - let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + let physical_plan = + sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -531,7 +536,8 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( let coalesce_batches_exec_2 = coalesce_batches_exec(filter); let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); - let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + let physical_plan = + sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -692,7 +698,8 @@ async fn test_with_multiple_replacable_repartitions( let repartition_hash_2 = repartition_exec_hash(coalesce_batches); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); - let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + let physical_plan = + sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -782,7 +789,7 @@ async fn test_not_replace_with_different_orderings( ); let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &sort.schema())], sort); + sort_preserving_merge_exec([sort_expr_default("c", &sort.schema())].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -928,8 +935,10 @@ async fn test_with_lost_and_kept_ordering( let filter = filter_exec(repartition_hash2); let sort2 = sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("c", &sort2.schema())], sort2); + let physical_plan = sort_preserving_merge_exec( + [sort_expr_default("c", &sort2.schema())].into(), + sort2, + ); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -1046,7 +1055,7 @@ async fn test_with_multiple_child_trees( ); let physical_plan = - sort_preserving_merge_exec(vec![sort_expr_default("a", &sort.schema())], sort); + sort_preserving_merge_exec([sort_expr_default("a", &sort.schema())].into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index a73d084a081f3..1ea944e8dba86 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -30,6 +30,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinType, Result}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::sanity_checker::SanityCheckPlan; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::repartition::RepartitionExec; @@ -410,16 +411,17 @@ fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { async fn test_bounded_window_agg_sort_requirement() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr_options( + let ordering: LexOrdering = [sort_expr_options( "c9", &source.schema(), SortOptions { descending: false, nulls_first: false, }, - )]; - let sort = sort_exec(sort_exprs.clone(), source); - let bw = bounded_window_exec("c9", sort_exprs, sort); + )] + .into(); + let sort = sort_exec(ordering.clone(), source); + let bw = bounded_window_exec("c9", ordering, sort); assert_plan(bw.as_ref(), vec![ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", @@ -518,12 +520,12 @@ async fn test_sort_merge_join_satisfied() -> Result<()> { let source1 = memory_exec(&schema1); let source2 = memory_exec(&schema2); let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; - let left = sort_exec(sort_exprs1, source1); - let right = sort_exec(sort_exprs2, source2); - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); + let ordering1 = [sort_expr_options("c9", &source1.schema(), sort_opts)].into(); + let ordering2 = [sort_expr_options("a", &source2.schema(), sort_opts)].into(); + let left = sort_exec(ordering1, source1); + let right = sort_exec(ordering2, source2); + let left_jcol = col("c9", &left.schema())?; + let right_jcol = col("a", &right.schema())?; let left = Arc::new(RepartitionExec::try_new( left, Partitioning::Hash(vec![left_jcol.clone()], 10), @@ -562,15 +564,16 @@ async fn test_sort_merge_join_order_missing() -> Result<()> { let schema2 = create_test_schema2(); let source1 = memory_exec(&schema1); let right = memory_exec(&schema2); - let sort_exprs1 = vec![sort_expr_options( + let ordering1 = [sort_expr_options( "c9", &source1.schema(), SortOptions::default(), - )]; - let left = sort_exec(sort_exprs1, source1); + )] + .into(); + let left = sort_exec(ordering1, source1); // Missing sort of the right child here.. - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); + let left_jcol = col("c9", &left.schema())?; + let right_jcol = col("a", &right.schema())?; let left = Arc::new(RepartitionExec::try_new( left, Partitioning::Hash(vec![left_jcol.clone()], 10), @@ -610,16 +613,16 @@ async fn test_sort_merge_join_dist_missing() -> Result<()> { let source1 = memory_exec(&schema1); let source2 = memory_exec(&schema2); let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; - let left = sort_exec(sort_exprs1, source1); - let right = sort_exec(sort_exprs2, source2); + let ordering1 = [sort_expr_options("c9", &source1.schema(), sort_opts)].into(); + let ordering2 = [sort_expr_options("a", &source2.schema(), sort_opts)].into(); + let left = sort_exec(ordering1, source1); + let right = sort_exec(ordering2, source2); let right = Arc::new(RepartitionExec::try_new( right, Partitioning::RoundRobinBatch(10), )?); - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); + let left_jcol = col("c9", &left.schema())?; + let right_jcol = col("a", &right.schema())?; let left = Arc::new(RepartitionExec::try_new( left, Partitioning::Hash(vec![left_jcol.clone()], 10), diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 56ad628c354af..93e02a336adb4 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -243,20 +243,18 @@ pub fn filter_exec( } pub fn sort_preserving_merge_exec( - sort_exprs: impl IntoIterator, + ordering: LexOrdering, input: Arc, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) + Arc::new(SortPreservingMergeExec::new(ordering, input)) } pub fn sort_preserving_merge_exec_with_fetch( - sort_exprs: impl IntoIterator, + ordering: LexOrdering, input: Arc, fetch: usize, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortPreservingMergeExec::new(sort_exprs, input).with_fetch(Some(fetch))) + Arc::new(SortPreservingMergeExec::new(ordering, input).with_fetch(Some(fetch))) } pub fn union_exec(input: Vec>) -> Arc { @@ -307,19 +305,18 @@ pub fn coalesce_batches_exec(input: Arc) -> Arc, + ordering: LexOrdering, input: Arc, ) -> Arc { - sort_exec_with_fetch(sort_exprs, None, input) + sort_exec_with_fetch(ordering, None, input) } pub fn sort_exec_with_fetch( - sort_exprs: impl IntoIterator, + ordering: LexOrdering, fetch: Option, input: Arc, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortExec::new(sort_exprs, input).with_fetch(fetch)) + Arc::new(SortExec::new(ordering, input).with_fetch(fetch)) } pub fn projection_exec( diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index f0d6802fa1ce8..444bdd15b7e34 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -2181,9 +2181,10 @@ mod tests { "test.parquet".to_string(), 1024, )])]) - .with_output_ordering(vec![LexOrdering::new(vec![ - PhysicalSortExpr::new_default(Arc::new(Column::new("date", 0))), - ])]) + .with_output_ordering(vec![[PhysicalSortExpr::new_default(Arc::new( + Column::new("date", 0), + ))] + .into()]) .with_file_compression_type(FileCompressionType::UNCOMPRESSED) .with_newlines_in_values(true) .build(); diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 2d584f7e710cf..36d9ad8027b92 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -831,7 +831,7 @@ mod memory_source_tests { Field::new("b", DataType::Int64, false), Field::new("c", DataType::Int64, false), ])); - let sort1 = LexOrdering::new(vec![ + let sort1: LexOrdering = [ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), @@ -840,11 +840,13 @@ mod memory_source_tests { expr: col("b", &schema)?, options: SortOptions::default(), }, - ]); - let sort2 = LexOrdering::new(vec![PhysicalSortExpr { + ] + .into(); + let sort2: LexOrdering = [PhysicalSortExpr { expr: col("c", &schema)?, options: SortOptions::default(), - }]); + }] + .into(); let mut expected_output_order = vec![]; expected_output_order.extend(sort1.clone()); expected_output_order.extend(sort2.clone()); diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index bee034b4a1752..a2fd1a5351f5a 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -181,7 +181,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - LexOrdering::new(order_bys.to_vec()), + LexOrdering::new(order_bys.to_vec()).unwrap(), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 51140da50be66..3bf09134e739a 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -1602,13 +1602,13 @@ mod tests { Field::new("e", DataType::Boolean, true), ])); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + let sort_keys = [PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]); + }]; let mut group_acc = FirstPrimitiveGroupsAccumulator::::try_new( - sort_key, + sort_keys.into(), true, &DataType::Int64, &[DataType::Int64], @@ -1696,13 +1696,13 @@ mod tests { Field::new("e", DataType::Boolean, true), ])); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + let sort_keys = [PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]); + }]; let mut group_acc = FirstPrimitiveGroupsAccumulator::::try_new( - sort_key, + sort_keys.into(), true, &DataType::Int64, &[DataType::Int64], @@ -1777,13 +1777,13 @@ mod tests { Field::new("e", DataType::Boolean, true), ])); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + let sort_keys = [PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]); + }]; let mut group_acc = FirstPrimitiveGroupsAccumulator::::try_new( - sort_key, + sort_keys.into(), true, &DataType::Int64, &[DataType::Int64], diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 2c79c9bba8cfe..070ecd91c61ac 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -339,10 +339,10 @@ pub struct LexOrdering { impl LexOrdering { /// Creates a new [`LexOrdering`] from the given vector of sort expressions. - /// The vector must not be empty. - pub fn new(inner: Vec) -> Self { - debug_assert!(!inner.is_empty()); - Self { inner } + /// If the vector is empty, returns `None`. + pub fn new(inner: impl IntoIterator) -> Option { + let inner = inner.into_iter().collect::>(); + (!inner.is_empty()).then(|| Self { inner }) } /// Appends an element to the back of the `LexOrdering`. @@ -416,7 +416,18 @@ impl PartialOrd for LexOrdering { impl From> for LexOrdering { fn from(value: Vec) -> Self { - Self::new(value) + Self::new(value).unwrap() + } +} + +impl From<[PhysicalSortExpr; N]> for LexOrdering { + fn from(value: [PhysicalSortExpr; N]) -> Self { + // TODO: Replace this with a condition on the generic parameter when + // Rust supports it. + assert!(N > 0); + Self { + inner: value.to_vec(), + } } } @@ -451,7 +462,7 @@ impl Display for LexOrdering { impl FromIterator for LexOrdering { fn from_iter>(iter: T) -> Self { - Self::new(iter.into_iter().collect()) + Self::new(iter).unwrap() } } @@ -532,6 +543,17 @@ impl From> for LexRequirement { } } +impl From<[PhysicalSortRequirement; N]> for LexRequirement { + fn from(value: [PhysicalSortRequirement; N]) -> Self { + // TODO: Replace this with a condition on the generic parameter when + // Rust supports it. + assert!(N > 0); + Self { + inner: value.to_vec(), + } + } +} + impl Deref for LexRequirement { type Target = [PhysicalSortRequirement]; @@ -597,7 +619,7 @@ impl From for LexOrdering { impl From> for LexOrdering { fn from(value: Vec) -> Self { - Self::new(value.into_iter().map(Into::into).collect()) + Self::new(value.into_iter().map(Into::into)).unwrap() } } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index a5d2b2d92a581..a073345f6e147 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -107,7 +107,7 @@ impl OrderingEquivalenceClass { ) { self.orderings.extend(orderings.into_iter().filter_map(|o| { let sort_exprs = o.into_iter().collect::>(); - (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs)) + LexOrdering::new(sort_exprs) })); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); @@ -356,7 +356,6 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; - use datafusion_physical_expr_common::sort_expr::LexOrdering; #[test] fn test_ordering_satisfy() -> Result<()> { @@ -364,11 +363,11 @@ mod tests { Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), ])); - let crude = LexOrdering::new(vec![PhysicalSortExpr { + let crude = vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), - }]); - let finer = LexOrdering::new(vec![ + }]; + let finer = vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -377,7 +376,7 @@ mod tests { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }, - ]); + ]; // finer ordering satisfies, crude ordering should return true let eq_properties_finer = EquivalenceProperties::new_with_orderings( Arc::clone(&input_schema), diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index dd8ac7baee310..d3f1a89a140a0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -139,7 +139,7 @@ impl<'a> DependencyEnumerator<'a> { // An empty dependency means the referred_sort_expr represents a global ordering. // Return its projected version, which is the target_expression. if node.dependencies.is_empty() { - return vec![LexOrdering::new(vec![target_sort_expr.clone()])]; + return vec![[target_sort_expr.clone()].into()]; }; node.dependencies @@ -1035,20 +1035,13 @@ mod tests { Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) - .with_reorder(LexOrdering::new( - ["a", "b", "c"] - .into_iter() - .map(|c| { - col(c, schema.as_ref()).map(|expr| PhysicalSortExpr { - expr, - options: SortOptions { - descending: false, - nulls_first: true, - }, - }) - }) - .collect::>>()?, - )); + .with_reorder(["a", "b", "c"].into_iter().map(|c| PhysicalSortExpr { + expr: col(c, schema.as_ref()).unwrap(), + options: SortOptions { + descending: false, + nulls_first: true, + }, + })); struct TestCase { name: &'static str, @@ -1341,7 +1334,7 @@ mod tests { let col_b = col("b", &schema)?; eq_properties = eq_properties.with_constants([ConstExpr::from(&col_a)]); - let sort_exprs = LexOrdering::new(vec![ + let sort_exprs = vec![ PhysicalSortExpr { expr: Arc::clone(&col_a), options: SortOptions::default(), @@ -1350,7 +1343,7 @@ mod tests { expr: Arc::clone(&col_b), options: SortOptions::default(), }, - ]); + ]; let result = eq_properties.with_reorder(sort_exprs); @@ -1395,10 +1388,10 @@ mod tests { ]); // New ordering: [a ASC] - let new_order = LexOrdering::new(vec![PhysicalSortExpr { + let new_order = vec![PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, - }]); + }]; let result = eq_properties.with_reorder(new_order); @@ -1443,10 +1436,10 @@ mod tests { ]); // New ordering: [b ASC] - let new_order = LexOrdering::new(vec![PhysicalSortExpr { + let new_order = vec![PhysicalSortExpr { expr: Arc::clone(&col_b), options: asc, - }]); + }]; let result = eq_properties.with_reorder(new_order); @@ -1491,17 +1484,17 @@ mod tests { ]); // New ordering: [a DESC] - let new_order = LexOrdering::new(vec![PhysicalSortExpr { + let new_order = vec![PhysicalSortExpr { expr: Arc::clone(&col_a), options: desc, - }]); + }]; let result = eq_properties.with_reorder(new_order.clone()); // Should only contain the new ordering since options don't match assert_eq!(result.oeq_class().len(), 1); let ordering = result.oeq_class().iter().next().unwrap(); - assert_eq!(ordering, &new_order); + assert_eq!(ordering.to_vec(), new_order); Ok(()) } @@ -1544,7 +1537,7 @@ mod tests { ]); // Initial ordering: [b ASC, c ASC] - let new_order = LexOrdering::new(vec![ + let new_order = vec![ PhysicalSortExpr { expr: Arc::clone(&col_b), options: asc, @@ -1553,7 +1546,7 @@ mod tests { expr: Arc::clone(&col_c), options: asc, }, - ]); + ]; let result = eq_properties.with_reorder(new_order); @@ -1674,31 +1667,27 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::clone(schema)); // Convert string column names to orderings - let satisfied_orderings: Vec = satisfied_orders + let satisfied_orderings: Vec<_> = satisfied_orders .iter() .map(|cols| { - LexOrdering::new( - cols.iter() - .map(|col_name| PhysicalSortExpr { - expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), - }) - .collect(), - ) + cols.iter() + .map(|col_name| PhysicalSortExpr { + expr: col(col_name, schema).unwrap(), + options: SortOptions::default(), + }) + .collect::>() }) .collect(); - let unsatisfied_orderings: Vec = unsatisfied_orders + let unsatisfied_orderings: Vec<_> = unsatisfied_orders .iter() .map(|cols| { - LexOrdering::new( - cols.iter() - .map(|col_name| PhysicalSortExpr { - expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), - }) - .collect(), - ) + cols.iter() + .map(|col_name| PhysicalSortExpr { + expr: col(col_name, schema).unwrap(), + options: SortOptions::default(), + }) + .collect::>() }) .collect(); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 0cd800faf336c..04b480579553e 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -173,11 +173,7 @@ impl EquivalenceProperties { schema: SchemaRef, orderings: impl IntoIterator>, ) -> Self { - let orderings = orderings - .into_iter() - .map(|o| o.into_iter().collect::>()) - .filter_map(|v| (!v.is_empty()).then(|| LexOrdering::new(v))) - .collect(); + let orderings = orderings.into_iter().filter_map(LexOrdering::new).collect(); Self { eq_group: EquivalenceGroup::empty(), oeq_class: OrderingEquivalenceClass::new(orderings), @@ -222,7 +218,7 @@ impl EquivalenceProperties { // Prune out constant expressions: let mut sort_exprs = self.oeq_class().output_ordering()?.take(); sort_exprs.retain(|item| !const_exprs_contains(&self.constants, &item.expr)); - (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs)) + LexOrdering::new(sort_exprs) } /// Returns the normalized version of the ordering equivalence class within. @@ -418,15 +414,17 @@ impl EquivalenceProperties { /// is re-sorted according to the argument `ordering`. Note that constants /// and equivalence classes are unchanged as they are unaffected by a re-sort. /// If the given ordering is already satisfied, the function does nothing. - pub fn with_reorder(mut self, ordering: LexOrdering) -> Self { + pub fn with_reorder( + mut self, + ordering: impl IntoIterator, + ) -> Self { // Filter out constant expressions as they don't affect ordering let filtered_exprs = ordering .into_iter() .filter(|expr| !self.is_expr_constant(&expr.expr)) .collect::>(); - if !filtered_exprs.is_empty() { - let filtered_exprs = LexOrdering::new(filtered_exprs); + if let Some(filtered_exprs) = LexOrdering::new(filtered_exprs) { // Preserve valid suffixes from existing orderings: let oeq_class = mem::take(&mut self.oeq_class); let mut new_orderings = oeq_class @@ -480,7 +478,7 @@ impl EquivalenceProperties { .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)) .collect::>(); - (!sort_exprs.is_empty()).then(|| LexOrdering::new(sort_exprs).collapse()) + LexOrdering::new(sort_exprs).map(|o| o.collapse()) } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the @@ -825,7 +823,7 @@ impl EquivalenceProperties { let result = new_orderings .into_iter() .multi_cartesian_product() - .map(LexOrdering::new) + .map(Into::into) .collect(); Ok(result) } @@ -1009,8 +1007,7 @@ impl EquivalenceProperties { generate_dependency_orderings(&relevant_deps, &dependency_map); // Append `sort_expr` to the dependent orderings: if dependency_orderings.is_empty() { - dependency_orderings - .push(LexOrdering::new(vec![sort_expr.clone()])); + dependency_orderings.push([sort_expr.clone()].into()); } else { for ordering in dependency_orderings.iter_mut() { ordering.push(sort_expr.clone()); @@ -1031,7 +1028,7 @@ impl EquivalenceProperties { // If prefix is empty, there is no dependency. Insert // empty ordering: if let Some(target) = &node.target_sort_expr { - prefixes.push(LexOrdering::new(vec![target.clone()])); + prefixes.push([target.clone()].into()); } } else { // Append current ordering on top its dependencies: diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 5da1f421531af..3ec0a604671b0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -156,10 +156,11 @@ impl UnionEquivalentOrderingBuilder { AddedOrdering::No(ordering) => { let mut sort_exprs = ordering.take(); sort_exprs.pop(); - if sort_exprs.is_empty() { + if let Some(ordering) = LexOrdering::new(sort_exprs) { + ordering + } else { break; } - LexOrdering::new(sort_exprs) } } } @@ -255,7 +256,7 @@ impl UnionEquivalentOrderingBuilder { } } - (!augmented_ordering.is_empty()).then(|| LexOrdering::new(augmented_ordering)) + LexOrdering::new(augmented_ordering) } fn build(self) -> Vec { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 9ca22c7f7714d..3162db0bbe3c2 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -141,9 +141,7 @@ pub fn create_ordering( } } } - if !sort_exprs.is_empty() { - all_sort_orders.push(LexOrdering::new(sort_exprs)); - } + all_sort_orders.extend(LexOrdering::new(sort_exprs)); } Ok(all_sort_orders) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 33cca0b358dc4..6cae3f78c59a5 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -279,18 +279,18 @@ fn replace_with_partial_sort( // Here we're trying to find the common prefix for sorted columns that is required for the // sort and already satisfied by the given ordering let child_eq_properties = child.equivalence_properties(); - let sort_req = LexRequirement::from(sort_plan.expr().clone()); + let sort_exprs = sort_plan.expr().clone(); let mut common_prefix_length = 0; while child_eq_properties - .ordering_satisfy_requirement(sort_req[0..common_prefix_length + 1].to_vec()) + .ordering_satisfy(sort_exprs[0..common_prefix_length + 1].to_vec()) { common_prefix_length += 1; } if common_prefix_length > 0 { return Ok(Arc::new( PartialSortExec::new( - LexOrdering::new(sort_plan.expr().to_vec()), + sort_exprs, Arc::clone(sort_plan.input()), common_prefix_length, ) @@ -573,10 +573,12 @@ fn analyze_immediate_sort_removal( && sort_input.output_partitioning().partition_count() > 1 { // Replace the sort with a sort-preserving merge: - let expr = LexOrdering::new(sort_exec.expr().to_vec()); Arc::new( - SortPreservingMergeExec::new(expr, Arc::clone(sort_input)) - .with_fetch(sort_exec.fetch()), + SortPreservingMergeExec::new( + sort_exec.expr().clone(), + Arc::clone(sort_input), + ) + .with_fetch(sort_exec.fetch()), ) as _ } else { // Remove the sort: @@ -795,8 +797,7 @@ fn remove_corresponding_sort_from_sub_plan( let fetch = plan.fetch(); let plan = if let Some(ordering) = plan.output_ordering() { Arc::new( - SortPreservingMergeExec::new(LexOrdering::new(ordering.to_vec()), plan) - .with_fetch(fetch), + SortPreservingMergeExec::new(ordering.clone(), plan).with_fetch(fetch), ) as _ } else { Arc::new(CoalescePartitionsExec::new(plan)) as _ diff --git a/datafusion/physical-optimizer/src/join_selection.rs b/datafusion/physical-optimizer/src/join_selection.rs index 5a772ccdd249f..1e62fa6f2639d 100644 --- a/datafusion/physical-optimizer/src/join_selection.rs +++ b/datafusion/physical-optimizer/src/join_selection.rs @@ -459,7 +459,7 @@ fn hash_join_convert_symmetric_subrule( JoinSide::Right => hash_join.right().output_ordering(), JoinSide::None => unreachable!(), } - .map(|p| LexOrdering::new(p.to_vec())) + .cloned() }) .flatten() }; diff --git a/datafusion/physical-optimizer/src/topk_aggregation.rs b/datafusion/physical-optimizer/src/topk_aggregation.rs index faedea55ca150..bff0b1e49684f 100644 --- a/datafusion/physical-optimizer/src/topk_aggregation.rs +++ b/datafusion/physical-optimizer/src/topk_aggregation.rs @@ -25,7 +25,6 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::execution_plan::CardinalityEffect; use datafusion_physical_plan::projection::ProjectionExec; @@ -131,7 +130,7 @@ impl TopKAggregation { Ok(Transformed::no(plan)) }; let child = Arc::clone(child).transform_down(closure).data().ok()?; - let sort = SortExec::new(LexOrdering::new(sort.expr().to_vec()), child) + let sort = SortExec::new(sort.expr().clone(), child) .with_fetch(sort.fetch()) .with_preserve_partitioning(sort.preserve_partitioning()); Some(Arc::new(sort)) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4a01da2e0d720..12f299a9fc6b5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2348,7 +2348,7 @@ mod tests { async fn test_get_finest_requirements() -> Result<()> { let test_schema = create_test_schema()?; - let options1 = SortOptions { + let options = SortOptions { descending: false, nulls_first: false, }; @@ -2364,44 +2364,44 @@ mod tests { vec![], vec![PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options, }], vec![ PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options, }, PhysicalSortExpr { expr: Arc::clone(col_b), - options: options1, + options, }, PhysicalSortExpr { expr: Arc::clone(col_c), - options: options1, + options, }, ], vec![ PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options, }, PhysicalSortExpr { expr: Arc::clone(col_b), - options: options1, + options, }, ], ]; - let common_requirement = LexOrdering::new(vec![ - PhysicalSortExpr { + let common_requirement = vec![ + PhysicalSortRequirement { expr: Arc::clone(col_a), - options: options1, + options: Some(options), }, - PhysicalSortExpr { + PhysicalSortRequirement { expr: Arc::clone(col_c), - options: options1, + options: Some(options), }, - ]); + ]; let mut aggr_exprs = order_by_exprs .into_iter() .map(|order_by_expr| { @@ -2415,14 +2415,13 @@ mod tests { }) .collect::>(); let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_finer_aggregate_exprs_requirement( + let result = get_finer_aggregate_exprs_requirement( &mut aggr_exprs, &group_by, &eq_properties, &AggregateMode::Partial, )?; - let res = LexOrdering::from_iter(res.into_iter().map(Into::into)); - assert_eq!(res, common_requirement); + assert_eq!(result, common_requirement); Ok(()) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index fab25d67aa519..2154a8440d8e5 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -199,6 +199,16 @@ impl SortMergeJoinExec { (left, right) }) .unzip(); + let Some(left_sort_exprs) = LexOrdering::new(left_sort_exprs) else { + return plan_err!( + "SortMergeJoinExec requires valid sort expressions for its left side" + ); + }; + let Some(right_sort_exprs) = LexOrdering::new(right_sort_exprs) else { + return plan_err!( + "SortMergeJoinExec requires valid sort expressions for its right side" + ); + }; let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); @@ -212,8 +222,8 @@ impl SortMergeJoinExec { join_type, schema, metrics: ExecutionPlanMetricsSet::new(), - left_sort_exprs: LexOrdering::new(left_sort_exprs), - right_sort_exprs: LexOrdering::new(right_sort_exprs), + left_sort_exprs, + right_sort_exprs, sort_options, null_equals_null, cache, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 5f4eace3c3c17..bfcf6c9237e6e 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -632,21 +632,18 @@ impl ExecutionPlan for SymmetricHashJoinExec { self.right(), )?; - Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( + SymmetricHashJoinExec::try_new( Arc::new(new_left), Arc::new(new_right), new_on, new_filter, self.join_type(), self.null_equals_null(), - self.right() - .output_ordering() - .map(|p| LexOrdering::new(p.to_vec())), - self.left() - .output_ordering() - .map(|p| LexOrdering::new(p.to_vec())), + self.right().output_ordering().cloned(), + self.left().output_ordering().cloned(), self.partition_mode(), - )?))) + ) + .map(|e| Some(Arc::new(e) as _)) } } @@ -1740,7 +1737,7 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, col, lit, Column}; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use rstest::*; @@ -1840,7 +1837,7 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: binary( col("la1", left_schema)?, Operator::Plus, @@ -1848,11 +1845,13 @@ mod tests { left_schema, )?, options: SortOptions::default(), - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -1920,14 +1919,16 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2065,20 +2066,22 @@ mod tests { let (left_partition, right_partition) = get_or_create_table((11, 21), 8)?; let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("la1_des", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("ra1_des", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2124,20 +2127,22 @@ mod tests { let (left_partition, right_partition) = get_or_create_table((10, 11), 8)?; let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("l_asc_null_first", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("r_asc_null_first", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2183,20 +2188,22 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("l_asc_null_last", left_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("r_asc_null_last", right_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2244,20 +2251,22 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("l_desc_null_first", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("r_desc_null_first", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2306,15 +2315,16 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]); - - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2365,20 +2375,23 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let left_sorted = vec![ - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]), - LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(), + [PhysicalSortExpr { expr: col("la2", left_schema)?, options: SortOptions::default(), - }]), + }] + .into(), ]; - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let right_sorted = [PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, @@ -2446,20 +2459,22 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("lt1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("rt1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2529,20 +2544,22 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("li1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("ri1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2605,14 +2622,16 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { + let left_sorted = [PhysicalSortExpr { expr: col("l_float", left_schema)?, options: SortOptions::default(), - }]); - let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { + }] + .into(); + let right_sorted = [PhysicalSortExpr { expr: col("r_float", right_schema)?, options: SortOptions::default(), - }]); + }] + .into(); let (left, right) = create_memory_table( left_partition, right_partition, diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index d38637dae0282..5f87a218540cd 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -102,10 +102,8 @@ pub async fn partitioned_sym_join_with_filter( filter, join_type, null_equals_null, - left.output_ordering().map(|p| LexOrdering::new(p.to_vec())), - right - .output_ordering() - .map(|p| LexOrdering::new(p.to_vec())), + left.output_ordering().cloned(), + right.output_ordering().cloned(), StreamJoinPartitionMode::Partitioned, )?; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 5df95a8db1c15..eeda2c19435eb 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -2357,7 +2357,7 @@ mod tests { let probe_sides = [Some(JoinSide::Left), Some(JoinSide::Right)]; let expected = [ - Some(LexOrdering::new(vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options, @@ -2378,8 +2378,8 @@ mod tests { expr: Arc::new(Column::new("y", 6)), options, }, - ])), - Some(LexOrdering::new(vec![ + ]), + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 7)), options, @@ -2400,7 +2400,7 @@ mod tests { expr: Arc::new(Column::new("d", 3)), options, }, - ])), + ]), ]; for (i, (maintains_input_order, probe_side)) in @@ -2408,8 +2408,8 @@ mod tests { { assert_eq!( calculate_join_output_ordering( - Some(&left_ordering), - Some(&right_ordering), + left_ordering.as_ref(), + right_ordering.as_ref(), join_type, &on_columns, left_columns_len, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 63f7c5ca5a095..1dbcc88a3945e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1715,11 +1715,11 @@ mod test { } fn sort_exprs(schema: &Schema) -> LexOrdering { - let options = SortOptions::default(); - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("c0", schema).unwrap(), - options, - }]) + options: SortOptions::default(), + }] + .into() } fn memory_exec(schema: &SchemaRef) -> Arc { diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 58e210b53eaa1..34d66c85563cb 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -503,7 +503,7 @@ mod tests { }; let partial_sort_exec = Arc::new(PartialSortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -516,10 +516,11 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(&source), 2, - )) as Arc; + )); let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; @@ -568,7 +569,7 @@ mod tests { for common_prefix_length in [1, 2] { let partial_sort_exec = Arc::new( PartialSortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -581,12 +582,13 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(&source), common_prefix_length, ) .with_fetch(Some(4)), - ) as Arc; + ); let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; @@ -641,7 +643,7 @@ mod tests { [(1, &source_tables[0]), (2, &source_tables[1])] { let partial_sort_exec = Arc::new(PartialSortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -654,7 +656,8 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(source), common_prefix_length, )); @@ -730,8 +733,8 @@ mod tests { nulls_first: false, }; let schema = mem_exec.schema(); - let partial_sort_executor = PartialSortExec::new( - LexOrdering::new(vec![ + let partial_sort_exec = PartialSortExec::new( + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -744,17 +747,16 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(&mem_exec), 1, ); - let partial_sort_exec = - Arc::new(partial_sort_executor.clone()) as Arc; let sort_exec = Arc::new(SortExec::new( - partial_sort_executor.expr, - partial_sort_executor.input, - )) as Arc; - let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; + partial_sort_exec.expr.clone(), + Arc::clone(&partial_sort_exec.input), + )); + let result = collect(Arc::new(partial_sort_exec), Arc::clone(&task_ctx)).await?; assert_eq!( result.iter().map(|r| r.num_rows()).collect_vec(), [125, 125, 150] @@ -791,8 +793,8 @@ mod tests { (Some(150), vec![125, 25]), (Some(250), vec![125, 125]), ] { - let partial_sort_executor = PartialSortExec::new( - LexOrdering::new(vec![ + let partial_sort_exec = PartialSortExec::new( + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -805,19 +807,22 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(&mem_exec), 1, ) .with_fetch(fetch_size); - let partial_sort_exec = - Arc::new(partial_sort_executor.clone()) as Arc; let sort_exec = Arc::new( - SortExec::new(partial_sort_executor.expr, partial_sort_executor.input) - .with_fetch(fetch_size), - ) as Arc; - let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; + SortExec::new( + partial_sort_exec.expr.clone(), + Arc::clone(&partial_sort_exec.input), + ) + .with_fetch(fetch_size), + ); + let result = + collect(Arc::new(partial_sort_exec), Arc::clone(&task_ctx)).await?; assert_eq!( result.iter().map(|r| r.num_rows()).collect_vec(), expected_batch_num_rows @@ -846,8 +851,8 @@ mod tests { nulls_first: false, }; let fetch_size = Some(250); - let partial_sort_executor = PartialSortExec::new( - LexOrdering::new(vec![ + let partial_sort_exec = PartialSortExec::new( + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -856,15 +861,14 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ]), + ] + .into(), Arc::clone(&mem_exec), 1, ) .with_fetch(fetch_size); - let partial_sort_exec = - Arc::new(partial_sort_executor.clone()) as Arc; - let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; + let result = collect(Arc::new(partial_sort_exec), Arc::clone(&task_ctx)).await?; for rb in result { assert!(rb.num_rows() > 0); } @@ -897,10 +901,11 @@ mod tests { TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; let partial_sort_exec = Arc::new(PartialSortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), input, 1, )); @@ -986,7 +991,7 @@ mod tests { )?; let partial_sort_exec = Arc::new(PartialSortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -999,7 +1004,8 @@ mod tests { expr: col("c", &schema)?, options: option_desc, }, - ]), + ] + .into(), TestMemoryExec::try_new_exec(&[vec![batch]], schema, None)?, 2, )); @@ -1061,10 +1067,11 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); let sort_exec = Arc::new(PartialSortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), blocking_exec, 1, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a9b70233f12ba..3feaa1ed0fd51 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1466,10 +1466,11 @@ mod tests { let schema = csv.schema(); let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), Arc::new(CoalescePartitionsExec::new(csv)), )); @@ -1477,7 +1478,6 @@ mod tests { assert_eq!(result.len(), 1); assert_eq!(result[0].num_rows(), 400); - assert_eq!( task_ctx.runtime_env().memory_pool.reserved(), 0, @@ -1512,10 +1512,11 @@ mod tests { let schema = input.schema(); let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), Arc::new(CoalescePartitionsExec::new(input)), )); @@ -1548,7 +1549,6 @@ mod tests { let i = as_primitive_array::(&columns[0])?; assert_eq!(i.value(0), 0); assert_eq!(i.value(i.len() - 1), 81); - assert_eq!( task_ctx.runtime_env().memory_pool.reserved(), 0, @@ -1580,18 +1580,15 @@ mod tests { let schema = input.schema(); let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), Arc::new(CoalescePartitionsExec::new(input)), )); - let result = collect( - Arc::clone(&sort_exec) as Arc, - Arc::clone(&task_ctx), - ) - .await?; + let result = collect(Arc::clone(&sort_exec) as _, Arc::clone(&task_ctx)).await?; let num_rows = result.iter().map(|batch| batch.num_rows()).sum::(); assert_eq!(num_rows, 20000); @@ -1680,20 +1677,18 @@ mod tests { let sort_exec = Arc::new( SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), Arc::new(CoalescePartitionsExec::new(csv)), ) .with_fetch(fetch), ); - let result = collect( - Arc::clone(&sort_exec) as Arc, - Arc::clone(&task_ctx), - ) - .await?; + let result = + collect(Arc::clone(&sort_exec) as _, Arc::clone(&task_ctx)).await?; assert_eq!(result.len(), 1); let metrics = sort_exec.metrics().unwrap(); @@ -1723,16 +1718,16 @@ mod tests { let data: ArrayRef = Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::()); - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; let input = - TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) - .unwrap(); + TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), input, )); @@ -1741,7 +1736,7 @@ mod tests { let expected_data: ArrayRef = Arc::new(vec![1, 2, 3].into_iter().map(Some).collect::()); let expected_batch = - RecordBatch::try_new(Arc::clone(&schema), vec![expected_data]).unwrap(); + RecordBatch::try_new(Arc::clone(&schema), vec![expected_data])?; // Data is correct assert_eq!(&vec![expected_batch], &result); @@ -1780,7 +1775,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1795,7 +1790,8 @@ mod tests { nulls_first: false, }, }, - ]), + ] + .into(), TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, )); @@ -1866,7 +1862,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![ + [ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1881,7 +1877,8 @@ mod tests { nulls_first: false, }, }, - ]), + ] + .into(), TestMemoryExec::try_new_exec(&[vec![batch]], schema, None)?, )); @@ -1945,10 +1942,11 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); let sort_exec = Arc::new(SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), blocking_exec, )); @@ -1976,10 +1974,11 @@ mod tests { RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options) .unwrap(); - let expressions = LexOrdering::new(vec![PhysicalSortExpr { + let expressions = [PhysicalSortExpr { expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), options: SortOptions::default(), - }]); + }] + .into(); let result = sort_batch(&batch, &expressions, None).unwrap(); assert_eq!(result.num_rows(), 1); @@ -1995,9 +1994,10 @@ mod tests { cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), }; let mut plan = SortExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + [PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))]), + )))] + .into(), Arc::new(source), ); plan = plan.with_fetch(Some(9)); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 8f60b726cb834..bcfc880f054a8 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -414,7 +414,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::test_util::batches_to_string; - use datafusion_common::{assert_batches_eq, DataFusionError}; + use datafusion_common::{assert_batches_eq, exec_err}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; @@ -422,8 +422,8 @@ mod tests { use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; - use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + use futures::{FutureExt, Stream, StreamExt}; use insta::assert_snapshot; use tokio::time::timeout; @@ -450,24 +450,25 @@ mod tests { let a: ArrayRef = Arc::new(Int32Array::from(vec![1; row_size])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"); row_size])); let c: ArrayRef = Arc::new(Int64Array::from_iter(vec![0; row_size])); - let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); + let rb = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)])?; let rbs = (0..1024).map(|_| rb.clone()).collect::>(); let schema = rb.schema(); - let sort = LexOrdering::new(vec![ + let sort = [ PhysicalSortExpr { - expr: col("b", &schema).unwrap(), + expr: col("b", &schema)?, options: Default::default(), }, PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("c", &schema)?, options: Default::default(), }, - ]); + ] + .into(); let repartition_exec = RepartitionExec::try_new( - TestMemoryExec::try_new_exec(&[rbs], schema, None).unwrap(), + TestMemoryExec::try_new_exec(&[rbs], schema, None)?, Partitioning::RoundRobinBatch(2), )?; let coalesce_batches_exec = @@ -486,7 +487,7 @@ mod tests { async fn test_round_robin_tie_breaker_success() -> Result<()> { let task_ctx = generate_task_ctx_for_round_robin_tie_breaker()?; let spm = generate_spm_for_round_robin_tie_breaker(true)?; - let _collected = collect(spm, task_ctx).await.unwrap(); + let _collected = collect(spm, task_ctx).await?; Ok(()) } @@ -718,7 +719,7 @@ mod tests { context: Arc, ) { let schema = partitions[0][0].schema(); - let sort = LexOrdering::new(vec![ + let sort = [ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -727,7 +728,8 @@ mod tests { expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]); + ] + .into(); let exec = TestMemoryExec::try_new_exec(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -775,13 +777,14 @@ mod tests { let csv = test::scan_partitioned(partitions); let schema = csv.schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("i", &schema).unwrap(), + let sort: LexOrdering = [PhysicalSortExpr { + expr: col("i", &schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]); + }] + .into(); let basic = basic_sort(Arc::clone(&csv), sort.clone(), Arc::clone(&task_ctx)).await; @@ -836,17 +839,18 @@ mod tests { let sorted = basic_sort(csv, sort, context).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); - Ok(TestMemoryExec::try_new_exec(&split, sorted.schema(), None).unwrap()) + TestMemoryExec::try_new_exec(&split, sorted.schema(), None).map(|e| e as _) } #[tokio::test] async fn test_partition_sort_streaming_input() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("i", &schema).unwrap(), + let sort: LexOrdering = [PhysicalSortExpr { + expr: col("i", &schema)?, options: Default::default(), - }]); + }] + .into(); let input = sorted_partitioned_input(sort.clone(), &[10, 3, 11], Arc::clone(&task_ctx)) @@ -858,12 +862,9 @@ mod tests { assert_eq!(basic.num_rows(), 1200); assert_eq!(partition.num_rows(), 1200); - let basic = arrow::util::pretty::pretty_format_batches(&[basic]) - .unwrap() - .to_string(); - let partition = arrow::util::pretty::pretty_format_batches(&[partition]) - .unwrap() - .to_string(); + let basic = arrow::util::pretty::pretty_format_batches(&[basic])?.to_string(); + let partition = + arrow::util::pretty::pretty_format_batches(&[partition])?.to_string(); assert_eq!(basic, partition); @@ -873,10 +874,11 @@ mod tests { #[tokio::test] async fn test_partition_sort_streaming_input_output() -> Result<()> { let schema = make_partition(11).schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("i", &schema).unwrap(), + let sort: LexOrdering = [PhysicalSortExpr { + expr: col("i", &schema)?, options: Default::default(), - }]); + }] + .into(); // Test streaming with default batch size let task_ctx = Arc::new(TaskContext::default()); @@ -891,19 +893,14 @@ mod tests { let task_ctx = Arc::new(task_ctx); let merge = Arc::new(SortPreservingMergeExec::new(sort, input)); - let merged = collect(merge, task_ctx).await.unwrap(); + let merged = collect(merge, task_ctx).await?; assert_eq!(merged.len(), 53); - assert_eq!(basic.num_rows(), 1200); assert_eq!(merged.iter().map(|x| x.num_rows()).sum::(), 1200); - let basic = arrow::util::pretty::pretty_format_batches(&[basic]) - .unwrap() - .to_string(); - let partition = arrow::util::pretty::pretty_format_batches(merged.as_slice()) - .unwrap() - .to_string(); + let basic = arrow::util::pretty::pretty_format_batches(&[basic])?.to_string(); + let partition = arrow::util::pretty::pretty_format_batches(&merged)?.to_string(); assert_eq!(basic, partition); @@ -948,7 +945,7 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); let schema = b1.schema(); - let sort = LexOrdering::new(vec![ + let sort = [ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { @@ -963,7 +960,8 @@ mod tests { nulls_first: false, }, }, - ]); + ] + .into(); let exec = TestMemoryExec::try_new_exec(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -997,13 +995,14 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort = [PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let exec = TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec).with_fetch(Some(2))); @@ -1029,13 +1028,14 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort = [PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let exec = TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -1059,10 +1059,11 @@ mod tests { async fn test_async() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort: LexOrdering = [PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: SortOptions::default(), - }]); + }] + .into(); let batches = sorted_partitioned_input(sort.clone(), &[5, 7, 3], Arc::clone(&task_ctx)) @@ -1138,10 +1139,11 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = b1.schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort = [PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), - }]); + }] + .into(); let exec = TestMemoryExec::try_new_exec(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -1197,10 +1199,11 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2)); let refs = blocking_exec.refs(); let sort_preserving_merge_exec = Arc::new(SortPreservingMergeExec::new( - LexOrdering::new(vec![PhysicalSortExpr { + [PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }]), + }] + .into(), blocking_exec, )); @@ -1245,13 +1248,14 @@ mod tests { let schema = partitions[0][0].schema(); - let sort = LexOrdering::new(vec![PhysicalSortExpr { + let sort = [PhysicalSortExpr { expr: col("value", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]); + }] + .into(); let exec = TestMemoryExec::try_new_exec(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -1431,9 +1435,10 @@ mod tests { congestion_cleared: Arc::new(Mutex::new(false)), }; let spm = SortPreservingMergeExec::new( - LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + [PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))]), + )))] + .into(), Arc::new(source), ); let spm_task = SpawnedTask::spawn(collect(Arc::new(spm), task_ctx)); @@ -1442,12 +1447,8 @@ mod tests { match result { Ok(Ok(Ok(_batches))) => Ok(()), Ok(Ok(Err(e))) => Err(e), - Ok(Err(_)) => Err(DataFusionError::Execution( - "SortPreservingMerge task panicked or was cancelled".to_string(), - )), - Err(_) => Err(DataFusionError::Execution( - "SortPreservingMerge caused a deadlock".to_string(), - )), + Ok(Err(_)) => exec_err!("SortPreservingMerge task panicked or was cancelled"), + Err(_) => exec_err!("SortPreservingMerge caused a deadlock"), } } } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 3898d2af70c5a..c3ee99f08fa5f 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1459,13 +1459,14 @@ mod tests { } fn schema_orders(schema: &SchemaRef) -> Result> { - let orderings = vec![LexOrdering::new(vec![PhysicalSortExpr { + let orderings = vec![[PhysicalSortExpr { expr: col("sn", schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }])]; + }] + .into()]; Ok(orderings) } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 619f0059bc541..d8ea48b3a8c3a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -373,10 +373,8 @@ pub(crate) fn window_equivalence_properties( .iter() .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))) .multi_cartesian_product() - .filter(|lex| { - !lex.is_empty() && window_eq_properties.ordering_satisfy(lex.clone()) - }) - .map(LexOrdering::new) + .filter_map(LexOrdering::new) + .filter(|lex| window_eq_properties.ordering_satisfy(lex.clone())) .collect::>(); // If there is a partitioning, and no possible ordering cannot satisfy // the input plan's orderings, then we cannot further introduce any diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index feff25f91aa95..2c13b945bd1f2 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -73,6 +73,7 @@ use datafusion::physical_plan::expressions::{ use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, + SymmetricHashJoinExec, }; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; @@ -108,8 +109,7 @@ use datafusion_functions_aggregate::string_agg::string_agg_udaf; use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, }; -use datafusion_proto::protobuf; -use datafusion_proto::protobuf::PhysicalPlanNode; +use datafusion_proto::protobuf::{self, PhysicalPlanNode}; /// Perform a serde roundtrip and assert that the string representation of the before and after plans /// are identical. Note that this often isn't sufficient to guarantee that no information is @@ -653,7 +653,7 @@ fn roundtrip_sort() -> Result<()> { let field_a = Field::new("a", DataType::Boolean, false); let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let sort_exprs = LexOrdering::new(vec![ + let sort_exprs = [ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -668,7 +668,8 @@ fn roundtrip_sort() -> Result<()> { nulls_first: true, }, }, - ]); + ] + .into(); roundtrip_test(Arc::new(SortExec::new( sort_exprs, Arc::new(EmptyExec::new(schema)), @@ -680,7 +681,7 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { let field_a = Field::new("a", DataType::Boolean, false); let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let sort_exprs = LexOrdering::new(vec![ + let sort_exprs: LexOrdering = [ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -695,7 +696,8 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { nulls_first: true, }, }, - ]); + ] + .into(); roundtrip_test(Arc::new(SortExec::new( sort_exprs.clone(), @@ -1458,31 +1460,29 @@ fn roundtrip_sym_hash_join() -> Result<()> { ] { for left_order in &[ None, - Some(LexOrdering::new(vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_left.index_of("col")?)), options: Default::default(), - }])), + }]), ] { for right_order in &[ None, - Some(LexOrdering::new(vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_right.index_of("col")?)), options: Default::default(), - }])), + }]), ] { - roundtrip_test(Arc::new( - datafusion::physical_plan::joins::SymmetricHashJoinExec::try_new( - Arc::new(EmptyExec::new(schema_left.clone())), - Arc::new(EmptyExec::new(schema_right.clone())), - on.clone(), - None, - join_type, - false, - left_order.clone(), - right_order.clone(), - *partition_mode, - )?, - ))?; + roundtrip_test(Arc::new(SymmetricHashJoinExec::try_new( + Arc::new(EmptyExec::new(schema_left.clone())), + Arc::new(EmptyExec::new(schema_right.clone())), + on.clone(), + None, + join_type, + false, + left_order.clone(), + right_order.clone(), + *partition_mode, + )?))?; } } } From a3101bbe9f46901b637b7fa1408eec0b7565148c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 01:48:50 +0300 Subject: [PATCH 085/167] Transition to fallible LexOrdering constructor - 2 --- .../core/src/datasource/listing/table.rs | 4 +- .../enforce_distribution.rs | 40 ++-- .../physical_optimizer/limit_pushdown.rs | 50 ++--- .../replace_with_order_preserving_variants.rs | 186 +++++++----------- .../tests/physical_optimizer/test_utils.rs | 7 + 5 files changed, 111 insertions(+), 176 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 47c4c0447aaeb..fffe5d14e6fc4 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1244,10 +1244,10 @@ mod tests { use crate::datasource::{provider_as_source, DefaultTableSource, MemTable}; use crate::execution::options::ArrowReadOptions; use crate::prelude::*; + use crate::test::columns; use crate::test::object_store::{ - ensure_head_concurrency, make_test_store_and_state, register_test_store + ensure_head_concurrency, make_test_store_and_state, register_test_store, }; - use crate::test::columns; use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 56161b71e8e9b..db66c230dc998 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -19,10 +19,10 @@ use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; -use crate::physical_optimizer::test_utils::parquet_exec_with_sort; use crate::physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, schema, - sort_merge_join_exec, sort_preserving_merge_exec, + check_integrity, coalesce_partitions_exec, parquet_exec_with_sort, repartition_exec, + schema, sort_exec, sort_exec_with_preserve_partitioning, sort_merge_join_exec, + sort_preserving_merge_exec, }; use arrow::compute::SortOptions; @@ -57,7 +57,6 @@ use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::JoinOn; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ @@ -314,16 +313,6 @@ fn filter_exec(input: Arc) -> Arc { Arc::new(FilterExec::try_new(predicate, input).unwrap()) } -fn sort_exec( - sort_exprs: LexOrdering, - input: Arc, - preserve_partitioning: bool, -) -> Arc { - let new_sort = SortExec::new(sort_exprs, input) - .with_preserve_partitioning(preserve_partitioning); - Arc::new(new_sort) -} - fn limit_exec(input: Arc) -> Arc { Arc::new(GlobalLimitExec::new( Arc::new(LocalLimitExec::new(input, 100)), @@ -1936,7 +1925,7 @@ fn repartition_sorted_limit() -> Result<()> { options: SortOptions::default(), }] .into(); - let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); + let plan = limit_exec(sort_exec(sort_key, parquet_exec())); let expected = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -1962,7 +1951,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { }] .into(); let plan = sort_required_exec_with_req( - filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), + filter_exec(sort_exec(sort_key.clone(), parquet_exec())), sort_key, ); @@ -2307,10 +2296,9 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { ]; let plan = sort_preserving_merge_exec( sort_key.clone(), - sort_exec( + sort_exec_with_preserve_partitioning( sort_key, projection_exec_with_alias(parquet_exec(), alias), - true, ), ); @@ -2336,7 +2324,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { options: SortOptions::default(), }] .into(); - let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); + let plan = sort_exec(sort_key, filter_exec(parquet_exec())); // Test: run EnforceDistribution, then EnforceSort. let expected = &[ @@ -2383,7 +2371,6 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> ("c".to_string(), "c".to_string()), ], ), - false, ); // Test: run EnforceDistribution, then EnforceSort. @@ -2650,8 +2637,8 @@ fn parallelization_sorted_limit() -> Result<()> { options: SortOptions::default(), }] .into(); - let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); + let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec())); + let plan_csv = limit_exec(sort_exec(sort_key, csv_exec())); let test_config = TestConfig::default(); @@ -2694,12 +2681,9 @@ fn parallelization_limit_with_filter() -> Result<()> { options: SortOptions::default(), }] .into(); - let plan_parquet = limit_exec(filter_exec(sort_exec( - sort_key.clone(), - parquet_exec(), - false, - ))); - let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); + let plan_parquet = + limit_exec(filter_exec(sort_exec(sort_key.clone(), parquet_exec()))); + let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec()))); let test_config = TestConfig::default(); diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index dd2c1960a6580..f4708528f552d 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -17,28 +17,29 @@ use std::sync::Arc; +use crate::physical_optimizer::test_utils::{ + coalesce_partitions_exec, sort_exec, sort_preserving_merge_exec, +}; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::BinaryExpr; -use datafusion_physical_expr::expressions::{col, lit}; +use datafusion_physical_expr::expressions::{col, lit, BinaryExpr}; use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::limit_pushdown::LimitPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort::SortExec; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; -use datafusion_physical_plan::{get_plan_string, ExecutionPlan, ExecutionPlanProperties}; +use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; fn create_schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -74,22 +75,6 @@ fn local_limit_exec( Arc::new(LocalLimitExec::new(input, fetch)) } -fn sort_exec( - sort_exprs: impl IntoIterator, - input: Arc, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortExec::new(sort_exprs, input)) -} - -fn sort_preserving_merge_exec( - sort_exprs: impl IntoIterator, - input: Arc, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) -} - fn projection_exec( schema: SchemaRef, input: Arc, @@ -122,12 +107,6 @@ fn coalesce_batches_exec(input: Arc) -> Arc, -) -> Arc { - Arc::new(CoalescePartitionsExec::new(local_limit)) -} - fn repartition_exec( streaming_table: Arc, ) -> Result> { @@ -314,14 +293,13 @@ fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { let coalesce_batches = coalesce_batches_exec(streaming_table); let projection = projection_exec(Arc::clone(&schema), coalesce_batches)?; let repartition = repartition_exec(projection)?; - let sort = sort_exec( - vec![PhysicalSortExpr { - expr: col("c1", &schema)?, - options: SortOptions::default(), - }], - repartition, - ); - let spm = sort_preserving_merge_exec(sort.output_ordering().unwrap().to_vec(), sort); + let ordering: LexOrdering = [PhysicalSortExpr { + expr: col("c1", &schema)?, + options: SortOptions::default(), + }] + .into(); + let sort = sort_exec(ordering.clone(), repartition); + let spm = sort_preserving_merge_exec(ordering, sort); let global_limit = global_limit_exec(spm, 0, Some(5)); let initial = get_plan_string(&global_limit); diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 5900e0a41ac86..0684f723de758 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -18,7 +18,9 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - check_integrity, sort_preserving_merge_exec, stream_exec_ordered_with_projection, + check_integrity, coalesce_partitions_exec, sort_exec, + sort_exec_with_preserve_partitioning, sort_preserving_merge_exec, + stream_exec_ordered_with_projection, }; use datafusion::prelude::SessionContext; @@ -28,13 +30,11 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_execution::TaskContext; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; @@ -44,7 +44,7 @@ use datafusion_common::Result; use datafusion_common::config::ConfigOptions; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use object_store::memory::InMemory; @@ -188,17 +188,16 @@ async fn test_replace_multiple_input_repartition_1( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); - - let physical_plan = - sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); + let sort = + sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), repartition); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -258,29 +257,22 @@ async fn test_with_inter_children_change_only( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr_default("a", &schema)]; + let sort_exprs = [sort_expr_default("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let sort = sort_exec( - vec![sort_expr_default("a", &coalesce_partitions.schema())], - coalesce_partitions, - false, - ); + let sort = sort_exec(sort_exprs.clone().into(), coalesce_partitions); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); let filter = filter_exec(repartition_hash2); - let sort2 = sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); + let sort2 = sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), filter); - let physical_plan = sort_preserving_merge_exec( - [sort_expr_default("a", &sort2.schema())].into(), - sort2, - ); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort2); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -363,19 +355,18 @@ async fn test_replace_multiple_input_repartition_2( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); - - let physical_plan = - sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); + let sort = + sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), repartition_hash); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -440,20 +431,21 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); - let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); - let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); - - let physical_plan = - sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); + let coalesce_batches_exec = coalesce_batches_exec(filter); + let sort = sort_exec_with_preserve_partitioning( + sort_exprs.clone().into(), + coalesce_batches_exec, + ); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -523,21 +515,22 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); let filter = filter_exec(repartition_hash); let coalesce_batches_exec_2 = coalesce_batches_exec(filter); - let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); - - let physical_plan = - sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); + let sort = sort_exec_with_preserve_partitioning( + sort_exprs.clone().into(), + coalesce_batches_exec_2, + ); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -612,7 +605,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { stream_exec_ordered_with_projection(&schema, sort_exprs) } else { @@ -621,10 +614,8 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); - let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); - - let physical_plan: Arc = - coalesce_partitions_exec(coalesce_batches_exec); + let coalesce_batches_exec = coalesce_batches_exec(filter); + let physical_plan = coalesce_partitions_exec(coalesce_batches_exec); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -685,21 +676,22 @@ async fn test_with_multiple_replacable_repartitions( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); let coalesce_batches = coalesce_batches_exec(filter); let repartition_hash_2 = repartition_exec_hash(coalesce_batches); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); - - let physical_plan = - sort_preserving_merge_exec([sort_expr("a", &schema)].into(), sort); + let sort = sort_exec_with_preserve_partitioning( + sort_exprs.clone().into(), + repartition_hash_2, + ); + let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -773,8 +765,10 @@ async fn test_not_replace_with_different_orderings( #[values(false, true)] source_unbounded: bool, #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { + use datafusion_physical_expr::LexOrdering; + let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { stream_exec_ordered_with_projection(&schema, sort_exprs) } else { @@ -782,14 +776,10 @@ async fn test_not_replace_with_different_orderings( }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let sort = sort_exec( - vec![sort_expr_default("c", &repartition_hash.schema())], - repartition_hash, - true, - ); - - let physical_plan = - sort_preserving_merge_exec([sort_expr_default("c", &sort.schema())].into(), sort); + let ordering: LexOrdering = + [sort_expr_default("c", &repartition_hash.schema())].into(); + let sort = sort_exec_with_preserve_partitioning(ordering.clone(), repartition_hash); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -846,17 +836,16 @@ async fn test_with_lost_ordering( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = - sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions, false); + let physical_plan = sort_exec(sort_exprs.into(), coalesce_partitions); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -915,8 +904,10 @@ async fn test_with_lost_and_kept_ordering( #[values(false, true)] source_unbounded: bool, #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { + use datafusion_physical_expr::LexOrdering; + let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; + let sort_exprs = [sort_expr("a", &schema)]; let source = if source_unbounded { stream_exec_ordered_with_projection(&schema, sort_exprs) } else { @@ -925,20 +916,14 @@ async fn test_with_lost_and_kept_ordering( let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let sort = sort_exec( - vec![sort_expr_default("c", &coalesce_partitions.schema())], - coalesce_partitions, - false, - ); + let ordering: LexOrdering = + [sort_expr_default("c", &coalesce_partitions.schema())].into(); + let sort = sort_exec(ordering.clone(), coalesce_partitions); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); let filter = filter_exec(repartition_hash2); - let sort2 = sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); - - let physical_plan = sort_preserving_merge_exec( - [sort_expr_default("c", &sort2.schema())].into(), - sort2, - ); + let sort2 = sort_exec_with_preserve_partitioning(ordering.clone(), filter); + let physical_plan = sort_preserving_merge_exec(ordering, sort2); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -1022,9 +1007,11 @@ async fn test_with_multiple_child_trees( #[values(false, true)] source_unbounded: bool, #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { + use datafusion_physical_expr::LexOrdering; + let schema = create_test_schema()?; - let left_sort_exprs = vec![sort_expr("a", &schema)]; + let left_sort_exprs = [sort_expr("a", &schema)]; let left_source = if source_unbounded { stream_exec_ordered_with_projection(&schema, left_sort_exprs) } else { @@ -1035,7 +1022,7 @@ async fn test_with_multiple_child_trees( let left_coalesce_partitions = Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); - let right_sort_exprs = vec![sort_expr("a", &schema)]; + let right_sort_exprs = [sort_expr("a", &schema)]; let right_source = if source_unbounded { stream_exec_ordered_with_projection(&schema, right_sort_exprs) } else { @@ -1048,14 +1035,9 @@ async fn test_with_multiple_child_trees( let hash_join_exec = hash_join_exec(left_coalesce_partitions, right_coalesce_partitions); - let sort = sort_exec( - vec![sort_expr_default("a", &hash_join_exec.schema())], - hash_join_exec, - true, - ); - - let physical_plan = - sort_preserving_merge_exec([sort_expr_default("a", &sort.schema())].into(), sort); + let ordering: LexOrdering = [sort_expr_default("a", &hash_join_exec.schema())].into(); + let sort = sort_exec_with_preserve_partitioning(ordering.clone(), hash_join_exec); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -1154,18 +1136,6 @@ fn sort_expr_options( } } -fn sort_exec( - sort_exprs: impl IntoIterator, - input: Arc, - preserve_partitioning: bool, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - SortExec::new(sort_exprs, input) - .with_preserve_partitioning(preserve_partitioning), - ) -} - fn repartition_exec_round_robin(input: Arc) -> Arc { Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(8)).unwrap()) } @@ -1197,10 +1167,6 @@ fn coalesce_batches_exec(input: Arc) -> Arc) -> Arc { - Arc::new(CoalescePartitionsExec::new(input)) -} - fn hash_join_exec( left: Arc, right: Arc, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 93e02a336adb4..3ca4561362856 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -311,6 +311,13 @@ pub fn sort_exec( sort_exec_with_fetch(ordering, None, input) } +pub fn sort_exec_with_preserve_partitioning( + ordering: LexOrdering, + input: Arc, +) -> Arc { + Arc::new(SortExec::new(ordering, input).with_preserve_partitioning(true)) +} + pub fn sort_exec_with_fetch( ordering: LexOrdering, fetch: Option, From 8819acee7daaf5a0680210c8874594f2bbe28514 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 01:56:11 +0300 Subject: [PATCH 086/167] Transition to fallible LexOrdering constructor - 3 --- .../physical_optimizer/enforce_sorting.rs | 22 +++++++++---------- .../tests/physical_optimizer/test_utils.rs | 4 ---- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index a6ebc3f3ab920..f67aabfe9670d 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -21,9 +21,9 @@ use crate::physical_optimizer::test_utils::{ aggregate_exec, bounded_window_exec, bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, - hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_exec, - projection_exec, repartition_exec, sort_exec, sort_exec_with_fetch, sort_expr, - sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, + hash_join_exec, local_limit_exec, memory_exec, parquet_exec, projection_exec, + repartition_exec, sort_exec, sort_exec_with_fetch, sort_expr, sort_expr_options, + sort_merge_join_exec, sort_preserving_merge_exec, sort_preserving_merge_exec_with_fetch, spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, }; @@ -232,7 +232,7 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { ] .into(); let sort = sort_exec(ordering.clone(), source1); - let limit = limit_exec(sort); + let limit = local_limit_exec(sort); let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); let union = union_exec(vec![source2, limit]); @@ -244,10 +244,9 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // We should keep the bottom `SortExec`. let expected_optimized = [ @@ -256,10 +255,9 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 3ca4561362856..ade75026de843 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -261,10 +261,6 @@ pub fn union_exec(input: Vec>) -> Arc Arc::new(UnionExec::new(input)) } -pub fn limit_exec(input: Arc) -> Arc { - global_limit_exec(local_limit_exec(input)) -} - pub fn local_limit_exec(input: Arc) -> Arc { Arc::new(LocalLimitExec::new(input, 100)) } From 3660ce5bf7ad317d9a0f4c9a4baffc21f27c6dc2 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 02:02:20 +0300 Subject: [PATCH 087/167] Transition to fallible LexOrdering constructor - 4 --- .../physical_optimizer/enforce_sorting.rs | 6 ++--- .../physical_optimizer/limit_pushdown.rs | 23 ++++--------------- .../physical_optimizer/sanity_checker.rs | 6 ++--- .../tests/physical_optimizer/test_utils.rs | 15 ++++++++---- 4 files changed, 21 insertions(+), 29 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index f67aabfe9670d..a65d511029acb 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -232,7 +232,7 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { ] .into(); let sort = sort_exec(ordering.clone(), source1); - let limit = local_limit_exec(sort); + let limit = local_limit_exec(sort, 100); let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); let union = union_exec(vec![source2, limit]); @@ -1150,8 +1150,8 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { .into(); let sort1 = sort_exec(ordering1, source1.clone()); let sort2 = sort_exec(ordering2, source1); - let limit = local_limit_exec(sort2); - let limit = global_limit_exec(limit); + let limit = local_limit_exec(sort2, 100); + let limit = global_limit_exec(limit, 0, Some(100)); let union = union_exec(vec![sort1, limit]); let ordering3 = [sort_expr("nullable_col", &schema)].into(); let physical_plan = sort_preserving_merge_exec(ordering3, union); diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index f4708528f552d..9e94e16d9611d 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -18,7 +18,8 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - coalesce_partitions_exec, sort_exec, sort_preserving_merge_exec, + coalesce_partitions_exec, global_limit_exec, local_limit_exec, + sort_exec, sort_preserving_merge_exec, }; use arrow::compute::SortOptions; @@ -28,14 +29,13 @@ use datafusion_common::error::Result; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, lit, BinaryExpr}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::limit_pushdown::LimitPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; @@ -60,21 +60,6 @@ fn streaming_table_exec(schema: SchemaRef) -> Result> { )?)) } -fn global_limit_exec( - input: Arc, - skip: usize, - fetch: Option, -) -> Arc { - Arc::new(GlobalLimitExec::new(input, skip, fetch)) -} - -fn local_limit_exec( - input: Arc, - fetch: usize, -) -> Arc { - Arc::new(LocalLimitExec::new(input, fetch)) -} - fn projection_exec( schema: SchemaRef, input: Arc, diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 1ea944e8dba86..bf9de28d29058 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -460,7 +460,7 @@ async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { async fn test_global_limit_single_partition() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); - let limit = global_limit_exec(source); + let limit = global_limit_exec(source, 0, Some(100)); assert_plan( limit.as_ref(), @@ -479,7 +479,7 @@ async fn test_global_limit_single_partition() -> Result<()> { async fn test_global_limit_multi_partition() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); - let limit = global_limit_exec(repartition_exec(source)); + let limit = global_limit_exec(repartition_exec(source), 0, Some(100)); assert_plan( limit.as_ref(), @@ -499,7 +499,7 @@ async fn test_global_limit_multi_partition() -> Result<()> { async fn test_local_limit() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); - let limit = local_limit_exec(source); + let limit = local_limit_exec(source, 100); assert_plan( limit.as_ref(), diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index ade75026de843..5142be31d1377 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -261,12 +261,19 @@ pub fn union_exec(input: Vec>) -> Arc Arc::new(UnionExec::new(input)) } -pub fn local_limit_exec(input: Arc) -> Arc { - Arc::new(LocalLimitExec::new(input, 100)) +pub fn local_limit_exec( + input: Arc, + fetch: usize, +) -> Arc { + Arc::new(LocalLimitExec::new(input, fetch)) } -pub fn global_limit_exec(input: Arc) -> Arc { - Arc::new(GlobalLimitExec::new(input, 0, Some(100))) +pub fn global_limit_exec( + input: Arc, + skip: usize, + fetch: Option, +) -> Arc { + Arc::new(GlobalLimitExec::new(input, skip, fetch)) } pub fn repartition_exec(input: Arc) -> Arc { From 4c9770ce07ce1a8c8d7bff8a1a98cb0dfc38fb95 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 02:06:47 +0300 Subject: [PATCH 088/167] Transition to fallible LexOrdering constructor - 5 --- .../tests/physical_optimizer/enforce_sorting.rs | 2 +- .../tests/physical_optimizer/limit_pushdown.rs | 17 ++++++----------- .../replace_with_order_preserving_variants.rs | 16 ++++++---------- .../core/tests/physical_optimizer/test_utils.rs | 7 +++++-- 4 files changed, 18 insertions(+), 24 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index a65d511029acb..71705375a4b68 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -1793,7 +1793,7 @@ async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { .into(); let sort = sort_exec(ordering.clone(), source); // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before - let coalesce_batches = coalesce_batches_exec(sort); + let coalesce_batches = coalesce_batches_exec(sort, 128); let window_agg = bounded_window_exec("non_nullable_col", ordering, coalesce_batches); let ordering2: LexOrdering = [sort_expr_options( "non_nullable_col", diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 9e94e16d9611d..38fc94655b53c 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - coalesce_partitions_exec, global_limit_exec, local_limit_exec, + coalesce_batches_exec, coalesce_partitions_exec, global_limit_exec, local_limit_exec, sort_exec, sort_preserving_merge_exec, }; @@ -33,7 +33,6 @@ use datafusion_physical_expr::Partitioning; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::limit_pushdown::LimitPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::projection::ProjectionExec; @@ -88,10 +87,6 @@ fn filter_exec( )?)) } -fn coalesce_batches_exec(input: Arc) -> Arc { - Arc::new(CoalesceBatchesExec::new(input, 8192)) -} - fn repartition_exec( streaming_table: Arc, ) -> Result> { @@ -176,7 +171,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi let streaming_table = streaming_table_exec(Arc::clone(&schema))?; let repartition = repartition_exec(streaming_table)?; let filter = filter_exec(schema, repartition)?; - let coalesce_batches = coalesce_batches_exec(filter); + let coalesce_batches = coalesce_batches_exec(filter, 8192); let local_limit = local_limit_exec(coalesce_batches, 5); let coalesce_partitions = coalesce_partitions_exec(local_limit); let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); @@ -243,8 +238,8 @@ fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batches_exec_into_fetching_version( ) -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); - let coalesce_batches = coalesce_batches_exec(streaming_table); + let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let coalesce_batches = coalesce_batches_exec(streaming_table, 8192); let projection = projection_exec(schema, coalesce_batches)?; let global_limit = global_limit_exec(projection, 0, Some(5)); @@ -274,8 +269,8 @@ fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batc #[test] fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); - let coalesce_batches = coalesce_batches_exec(streaming_table); + let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let coalesce_batches = coalesce_batches_exec(streaming_table, 8192); let projection = projection_exec(Arc::clone(&schema), coalesce_batches)?; let repartition = repartition_exec(projection)?; let ordering: LexOrdering = [PhysicalSortExpr { diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 0684f723de758..7fc0f3166ed2d 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, sort_exec, + check_integrity, coalesce_batches_exec, coalesce_partitions_exec, sort_exec, sort_exec_with_preserve_partitioning, sort_preserving_merge_exec, stream_exec_ordered_with_projection, }; @@ -440,7 +440,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); - let coalesce_batches_exec = coalesce_batches_exec(filter); + let coalesce_batches_exec = coalesce_batches_exec(filter, 8192); let sort = sort_exec_with_preserve_partitioning( sort_exprs.clone().into(), coalesce_batches_exec, @@ -522,10 +522,10 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition_rr = repartition_exec_round_robin(source); - let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); + let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr, 8192); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); let filter = filter_exec(repartition_hash); - let coalesce_batches_exec_2 = coalesce_batches_exec(filter); + let coalesce_batches_exec_2 = coalesce_batches_exec(filter, 8192); let sort = sort_exec_with_preserve_partitioning( sort_exprs.clone().into(), coalesce_batches_exec_2, @@ -614,7 +614,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); - let coalesce_batches_exec = coalesce_batches_exec(filter); + let coalesce_batches_exec = coalesce_batches_exec(filter, 8192); let physical_plan = coalesce_partitions_exec(coalesce_batches_exec); // Expected inputs unbounded and bounded @@ -685,7 +685,7 @@ async fn test_with_multiple_replacable_repartitions( let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); - let coalesce_batches = coalesce_batches_exec(filter); + let coalesce_batches = coalesce_batches_exec(filter, 8192); let repartition_hash_2 = repartition_exec_hash(coalesce_batches); let sort = sort_exec_with_preserve_partitioning( sort_exprs.clone().into(), @@ -1163,10 +1163,6 @@ fn filter_exec(input: Arc) -> Arc { Arc::new(FilterExec::try_new(predicate, input).unwrap()) } -fn coalesce_batches_exec(input: Arc) -> Arc { - Arc::new(CoalesceBatchesExec::new(input, 8192)) -} - fn hash_join_exec( left: Arc, right: Arc, diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 5142be31d1377..fe53c00e8603d 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -303,8 +303,11 @@ pub fn aggregate_exec(input: Arc) -> Arc { ) } -pub fn coalesce_batches_exec(input: Arc) -> Arc { - Arc::new(CoalesceBatchesExec::new(input, 128)) +pub fn coalesce_batches_exec( + input: Arc, + batch_size: usize, +) -> Arc { + Arc::new(CoalesceBatchesExec::new(input, batch_size)) } pub fn sort_exec( From 7f2fe976488aaa73e6f9219e99d1c02a5c335470 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 15 Apr 2025 22:02:15 +0300 Subject: [PATCH 089/167] Transition to fallible LexOrdering constructor - 6 --- datafusion/catalog/src/memory/table.rs | 25 ++- datafusion/core/benches/physical_plan.rs | 3 +- datafusion/core/benches/sort.rs | 11 +- datafusion/core/src/physical_planner.rs | 14 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 36 ++--- .../aggregation_fuzzer/data_generator.rs | 16 +- .../tests/fuzz_cases/equivalence/ordering.rs | 61 +++---- .../fuzz_cases/equivalence/projection.rs | 24 +-- .../fuzz_cases/equivalence/properties.rs | 11 +- .../tests/fuzz_cases/equivalence/utils.rs | 39 +---- .../sort_preserving_repartition_fuzz.rs | 27 ++-- .../core/tests/fuzz_cases/window_fuzz.rs | 8 +- .../physical_optimizer/enforce_sorting.rs | 43 ++--- .../physical_optimizer/limit_pushdown.rs | 42 +---- .../replace_with_order_preserving_variants.rs | 151 ++++++++---------- .../tests/physical_optimizer/test_utils.rs | 32 ++-- .../benches/split_groups_by_statistics.rs | 21 ++- datafusion/datasource/src/file_scan_config.rs | 29 ++-- datafusion/datasource/src/memory.rs | 5 +- datafusion/datasource/src/statistics.rs | 28 ++-- .../functions-aggregate/src/first_last.rs | 36 ++--- .../functions-aggregate/src/nth_value.rs | 14 +- .../physical-expr-common/src/sort_expr.rs | 18 --- .../physical-expr/src/equivalence/mod.rs | 51 +++--- .../physical-expr/src/equivalence/ordering.rs | 12 +- .../src/equivalence/projection.rs | 4 +- .../src/equivalence/properties/dependency.rs | 89 +++++------ .../src/equivalence/properties/joins.rs | 7 +- .../src/equivalence/properties/mod.rs | 17 +- .../src/equivalence/properties/union.rs | 2 +- datafusion/physical-expr/src/utils/mod.rs | 11 -- .../physical-plan/src/aggregates/mod.rs | 7 +- .../physical-plan/src/aggregates/row_hash.rs | 32 ++-- .../src/joins/nested_loop_join.rs | 35 ++-- datafusion/physical-plan/src/joins/utils.rs | 116 ++++---------- datafusion/physical-plan/src/projection.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 30 ++-- .../src/sorts/sort_preserving_merge.rs | 15 +- datafusion/physical-plan/src/streaming.rs | 18 +-- datafusion/physical-plan/src/topk/mod.rs | 2 +- datafusion/physical-plan/src/union.rs | 34 +--- datafusion/physical-plan/src/windows/mod.rs | 20 +-- .../proto/src/physical_plan/from_proto.rs | 6 +- datafusion/proto/src/physical_plan/mod.rs | 96 +++++------ 44 files changed, 547 insertions(+), 753 deletions(-) diff --git a/datafusion/catalog/src/memory/table.rs b/datafusion/catalog/src/memory/table.rs index 99bd71c0caa2b..86b8d7578fa20 100644 --- a/datafusion/catalog/src/memory/table.rs +++ b/datafusion/catalog/src/memory/table.rs @@ -23,25 +23,22 @@ use std::fmt::Debug; use std::sync::Arc; use crate::TableProvider; -use datafusion_common::error::Result; -use datafusion_expr::Expr; -use datafusion_expr::TableType; -use datafusion_physical_expr::create_physical_sort_exprs; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::{ - common, ExecutionPlan, ExecutionPlanProperties, Partitioning, -}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; +use datafusion_common::error::Result; use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; use datafusion_common_runtime::JoinSet; -use datafusion_datasource::memory::MemSink; -use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::memory::{MemSink, MemorySourceConfig}; use datafusion_datasource::sink::DataSinkExec; use datafusion_datasource::source::DataSourceExec; use datafusion_expr::dml::InsertOp; -use datafusion_expr::SortExpr; +use datafusion_expr::{Expr, SortExpr, TableType}; +use datafusion_physical_expr::{create_physical_sort_exprs, LexOrdering}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::{ + common, ExecutionPlan, ExecutionPlanProperties, Partitioning, +}; use datafusion_session::Session; use async_trait::async_trait; @@ -242,9 +239,9 @@ impl TableProvider for MemTable { let eqp = state.execution_props(); let mut file_sort_order = vec![]; for sort_exprs in sort_order.iter() { - file_sort_order.push( - create_physical_sort_exprs(sort_exprs, &df_schema, eqp)?.into(), - ); + let physical_exprs = + create_physical_sort_exprs(sort_exprs, &df_schema, eqp)?; + file_sort_order.extend(LexOrdering::new(physical_exprs)); } source = source.try_with_sort_information(file_sort_order)?; } diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 0a65c52f72def..aa65c0b5e98b2 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -54,7 +54,8 @@ fn sort_preserving_merge_operator( expr: col(name, &schema).unwrap(), options: Default::default(), }) - .collect::(); + .collect::>(); + let sort = LexOrdering::new(sort).unwrap(); let exec = MemorySourceConfig::try_new_exec( &batches.into_iter().map(|rb| vec![rb]).collect::>(), diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 85f456ce5dc22..aba1269ed268d 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -71,7 +71,6 @@ use std::sync::Arc; use arrow::array::StringViewArray; use arrow::{ array::{DictionaryArray, Float64Array, Int64Array, StringArray}, - compute::SortOptions, datatypes::{Int32Type, Schema}, record_batch::RecordBatch, }; @@ -272,14 +271,12 @@ impl BenchCase { /// Make sort exprs for each column in `schema` fn make_sort_exprs(schema: &Schema) -> LexOrdering { - schema + let sort_exprs = schema .fields() .iter() - .map(|f| PhysicalSortExpr { - expr: col(f.name(), schema).unwrap(), - options: SortOptions::default(), - }) - .collect() + .map(|f| PhysicalSortExpr::new_default(col(f.name(), schema).unwrap())) + .collect::>(); + LexOrdering::new(sort_exprs).unwrap() } /// Create streams of int64 (where approximately 1/3 values is repeated) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 7cd2c02897cef..f085086ab8c60 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -82,7 +82,9 @@ use datafusion_expr::{ }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{create_physical_sort_exprs, PhysicalSortExpr}; +use datafusion_physical_expr::{ + create_physical_sort_exprs, LexOrdering, PhysicalSortExpr, +}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::execution_plan::InvariantLevel; @@ -822,13 +824,17 @@ impl DefaultPhysicalPlanner { }) => { let physical_input = children.one()?; let input_dfschema = input.as_ref().schema(); - let sort_expr = create_physical_sort_exprs( + let sort_exprs = create_physical_sort_exprs( expr, input_dfschema, session_state.execution_props(), )?; - let new_sort = - SortExec::new(sort_expr.into(), physical_input).with_fetch(*fetch); + let Some(ordering) = LexOrdering::new(sort_exprs) else { + return internal_err!( + "SortExec requires at least one sort expression" + ); + }; + let new_sort = SortExec::new(ordering, physical_input).with_fetch(*fetch); Arc::new(new_sort) } LogicalPlan::Subquery(_) => todo!(), diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 4bc6514143383..cc62b65c93e8c 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -17,6 +17,7 @@ use std::sync::Arc; +use super::record_batch_generator::get_supported_types_columns; use crate::fuzz_cases::aggregation_fuzzer::{ AggregationFuzzerBuilder, DatasetGeneratorConfig, QueryBuilder, }; @@ -25,38 +26,35 @@ use arrow::array::{ types::Int64Type, Array, ArrayRef, AsArray, Int32Array, Int64Array, RecordBatch, StringArray, }; -use arrow::compute::{concat_batches, SortOptions}; +use arrow::compute::concat_batches; use arrow::datatypes::DataType; use arrow::util::pretty::pretty_format_batches; use arrow_schema::{Field, Schema, SchemaRef}; -use datafusion::common::Result; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::MemTable; -use datafusion::physical_expr::aggregate::AggregateExprBuilder; -use datafusion::physical_plan::aggregates::{ - AggregateExec, AggregateMode, PhysicalGroupBy, -}; -use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; -use datafusion_common::HashMap; +use datafusion_common::{HashMap, Result}; use datafusion_common_runtime::JoinSet; +use datafusion_execution::memory_pool::FairSpillPool; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; +use datafusion_execution::TaskContext; use datafusion_functions_aggregate::sum::sum_udaf; +use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::{col, lit, Column}; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::metrics::MetricValue; use datafusion_physical_plan::InputOrderMode; +use datafusion_physical_plan::{collect, displayable, ExecutionPlan}; use test_utils::{add_empty_batches, StringBatchGenerator}; -use datafusion_execution::memory_pool::FairSpillPool; -use datafusion_execution::runtime_env::RuntimeEnvBuilder; -use datafusion_execution::TaskContext; -use datafusion_physical_plan::metrics::MetricValue; use rand::rngs::StdRng; use rand::{random, thread_rng, Rng, SeedableRng}; -use super::record_batch_generator::get_supported_types_columns; - // ======================================================================== // The new aggregation fuzz tests based on [`AggregationFuzzer`] // ======================================================================== @@ -294,13 +292,9 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = vec![]; - for ordering_col in ["a", "b", "c"] { - sort_keys.push(PhysicalSortExpr { - expr: col(ordering_col, &schema).unwrap(), - options: SortOptions::default(), - }) - } + let sort_keys = ["a", "b", "c"].map(|ordering_col| { + PhysicalSortExpr::new_default(col(ordering_col, &schema).unwrap()) + }); let concat_input_record = concat_batches(&schema, &input1).unwrap(); diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index ac1cb0b3fe33f..753a74995d8ff 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -149,14 +149,14 @@ impl DatasetGenerator { for sort_keys in self.sort_keys_set.clone() { let sort_exprs = sort_keys .iter() - .map(|key| { - let col_expr = col(key, schema)?; - Ok(PhysicalSortExpr::new_default(col_expr)) - }) - .collect::>()?; - let sorted_batch = sort_batch(&base_batch, &sort_exprs, None)?; - - let batches = stagger_batch(sorted_batch); + .map(|key| col(key, schema).map(PhysicalSortExpr::new_default)) + .collect::>>()?; + let batch = if let Some(ordering) = LexOrdering::new(sort_exprs) { + sort_batch(&base_batch, &ordering, None)? + } else { + base_batch.clone() + }; + let batches = stagger_batch(batch); let dataset = Dataset::new(batches, sort_keys); datasets.push(dataset); } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index f54b1cb2ecce1..72a8ad7210122 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -16,13 +16,16 @@ // under the License. use crate::fuzz_cases::equivalence::utils::{ - convert_to_orderings, create_random_schema, create_test_params, create_test_schema_2, + create_random_schema, create_test_params, create_test_schema_2, generate_table_for_eq_properties, generate_table_for_orderings, is_table_same_after_sort, TestScalarUDF, }; use arrow::compute::SortOptions; use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; +use datafusion_physical_expr::equivalence::{ + convert_to_orderings, convert_to_sort_exprs, +}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; @@ -57,25 +60,28 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { for n_req in 1..=col_exprs.len() { for exprs in col_exprs.iter().combinations(n_req) { - let requirement = exprs + let sort_exprs = exprs .into_iter() .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::(); + .collect::>(); + let Some(ordering) = LexOrdering::new(sort_exprs) else { + unreachable!("Test should always produce non-degenerate orderings"); + }; let expected = is_table_same_after_sort( - requirement.clone(), - table_data_with_properties.clone(), + ordering.clone(), + &table_data_with_properties, )?; let err_msg = format!( "Error in test case requirement:{:?}, expected: {:?}, eq_properties {}", - requirement, expected, eq_properties + ordering, expected, eq_properties ); // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(requirement), + eq_properties.ordering_satisfy(ordering), expected, "{}", err_msg @@ -129,26 +135,29 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { for n_req in 1..=exprs.len() { for exprs in exprs.iter().combinations(n_req) { - let requirement = exprs + let sort_exprs = exprs .into_iter() .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::(); + .collect::>(); + let Some(ordering) = LexOrdering::new(sort_exprs) else { + unreachable!("Test should always produce non-degenerate orderings"); + }; let expected = is_table_same_after_sort( - requirement.clone(), - table_data_with_properties.clone(), + ordering.clone(), + &table_data_with_properties, )?; let err_msg = format!( "Error in test case requirement:{:?}, expected: {:?}, eq_properties: {}", - requirement, expected, eq_properties, + ordering, expected, eq_properties, ); // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(requirement), + eq_properties.ordering_satisfy(ordering), (expected | false), "{}", err_msg @@ -304,25 +313,19 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { ]; for (cols, expected) in requirements { - let err_msg = format!("Error in test case:{cols:?}"); - let required = cols - .into_iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr), - options, - }) - .collect::(); + let err_msg = format!("Error in test case: {cols:?}"); + let sort_exprs = convert_to_sort_exprs(&cols); + let Some(ordering) = LexOrdering::new(sort_exprs) else { + unreachable!("Test should always produce non-degenerate orderings"); + }; // Check expected result with experimental result. assert_eq!( - is_table_same_after_sort( - required.clone(), - table_data_with_properties.clone() - )?, + is_table_same_after_sort(ordering.clone(), &table_data_with_properties)?, expected ); assert_eq!( - eq_properties.ordering_satisfy(required), + eq_properties.ordering_satisfy(ordering), expected, "{err_msg}" ); @@ -375,7 +378,7 @@ fn test_ordering_satisfy_on_data() -> Result<()> { (col_d, option_asc), ]; let ordering = convert_to_orderings(&[ordering])[0].clone(); - assert!(!is_table_same_after_sort(ordering, batch.clone())?); + assert!(!is_table_same_after_sort(ordering, &batch)?); // [a ASC, b ASC, d ASC] cannot be deduced let ordering = vec![ @@ -384,12 +387,12 @@ fn test_ordering_satisfy_on_data() -> Result<()> { (col_d, option_asc), ]; let ordering = convert_to_orderings(&[ordering])[0].clone(); - assert!(!is_table_same_after_sort(ordering, batch.clone())?); + assert!(!is_table_same_after_sort(ordering, &batch)?); // [a ASC, b ASC] can be deduced let ordering = vec![(col_a, option_asc), (col_b, option_asc)]; let ordering = convert_to_orderings(&[ordering])[0].clone(); - assert!(is_table_same_after_sort(ordering, batch.clone())?); + assert!(is_table_same_after_sort(ordering, &batch)?); Ok(()) } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index c25d366ac9549..04823b986325b 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -88,10 +88,7 @@ fn project_orderings_random() -> Result<()> { // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). assert!( - is_table_same_after_sort( - ordering.clone(), - projected_batch.clone(), - )?, + is_table_same_after_sort(ordering.clone(), &projected_batch)?, "{}", err_msg ); @@ -166,25 +163,28 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { for n_req in 1..=projected_exprs.len() { for exprs in projected_exprs.iter().combinations(n_req) { - let requirement = exprs + let sort_exprs = exprs .into_iter() .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::(); - let expected = is_table_same_after_sort( - requirement.clone(), - projected_batch.clone(), - )?; + .collect::>(); + let Some(ordering) = LexOrdering::new(sort_exprs) else { + unreachable!( + "Test should always produce non-degenerate orderings" + ); + }; + let expected = + is_table_same_after_sort(ordering.clone(), &projected_batch)?; let err_msg = format!( "Error in test case requirement:{:?}, expected: {:?}, eq_properties: {}, projected_eq: {}, projection_mapping: {:?}", - requirement, expected, eq_properties, projected_eq, projection_mapping + ordering, expected, eq_properties, projected_eq, projection_mapping ); // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - projected_eq.ordering_satisfy(requirement), + projected_eq.ordering_satisfy(ordering), expected, "{}", err_msg diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 34d58456ac732..d3b0613de8d81 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -25,7 +25,7 @@ use crate::fuzz_cases::equivalence::utils::{ use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; -use datafusion_physical_expr::ScalarFunctionExpr; +use datafusion_physical_expr::{LexOrdering, ScalarFunctionExpr}; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use itertools::Itertools; @@ -92,12 +92,11 @@ fn test_find_longest_permutation_random() -> Result<()> { assert_eq!(ordering.len(), indices.len(), "{}", err_msg); // Since ordered section satisfies schema, we expect // that result will be same after sort (e.g sort was unnecessary). + let Some(ordering) = LexOrdering::new(ordering) else { + continue; + }; assert!( - ordering.is_empty() - || is_table_same_after_sort( - ordering.into(), - table_data_with_properties.clone(), - )?, + is_table_same_after_sort(ordering, &table_data_with_properties)?, "{}", err_msg ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 037e30e2d1ce4..0d63bfc3a2f60 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -15,16 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::physical_plan::expressions::col; -use datafusion::physical_plan::expressions::Column; -use datafusion_physical_expr::{ConstExpr, EquivalenceProperties, PhysicalSortExpr}; use std::any::Any; use std::cmp::Ordering; use std::sync::Arc; use arrow::array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Array}; -use arrow::compute::SortOptions; -use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn}; +use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn, SortOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, plan_datafusion_err, DataFusionError, Result}; @@ -32,9 +28,13 @@ use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use datafusion_physical_expr::equivalence::{EquivalenceClass, ProjectionMapping}; +use datafusion_physical_expr::equivalence::{ + convert_to_orderings, EquivalenceClass, ProjectionMapping, +}; +use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_plan::expressions::{col, Column}; use itertools::izip; use rand::prelude::*; @@ -226,7 +226,7 @@ fn add_equal_conditions_test() -> Result<()> { /// already sorted according to `required_ordering` to begin with. pub fn is_table_same_after_sort( mut required_ordering: LexOrdering, - batch: RecordBatch, + batch: &RecordBatch, ) -> Result { // Clone the original schema and columns let original_schema = batch.schema(); @@ -494,29 +494,6 @@ pub fn generate_table_for_orderings( Ok(batch) } -// Convert each tuple to PhysicalSortExpr -pub fn convert_to_sort_exprs( - in_data: &[(&Arc, SortOptions)], -) -> LexOrdering { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(*expr), - options: *options, - }) - .collect() -} - -// Convert each inner tuple to PhysicalSortExpr -pub fn convert_to_orderings( - orderings: &[Vec<(&Arc, SortOptions)>], -) -> Vec { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) - .collect() -} - // Utility function to generate random f64 array fn generate_random_f64_array( n_elems: usize, diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 3f8a5f03a28f5..23848433b069d 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -37,11 +37,13 @@ mod sp_repartition_fuzz_tests { use datafusion::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::{config::SessionConfig, memory_pool::MemoryConsumer}; - use datafusion_physical_expr::{ - equivalence::{EquivalenceClass, EquivalenceProperties}, - expressions::{col, Column}, - ConstExpr, PhysicalExpr, PhysicalSortExpr, + use datafusion_physical_expr::equivalence::{ + EquivalenceClass, EquivalenceProperties, }; + use datafusion_physical_expr::expressions::{col, Column}; + use datafusion_physical_expr::ConstExpr; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use test_utils::add_empty_batches; use itertools::izip; @@ -344,18 +346,14 @@ mod sp_repartition_fuzz_tests { let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = vec![]; - for ordering_col in ["a", "b", "c"] { - sort_keys.push(PhysicalSortExpr { - expr: col(ordering_col, &schema).unwrap(), - options: SortOptions::default(), - }) - } + let sort_keys = ["a", "b", "c"].map(|ordering_col| { + PhysicalSortExpr::new_default(col(ordering_col, &schema).unwrap()) + }); let concat_input_record = concat_batches(&schema, &input1).unwrap(); let running_source = Arc::new( - MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None) + MemorySourceConfig::try_new(&[input1], schema.clone(), None) .unwrap() .try_with_sort_information(vec![sort_keys.clone().into()]) .unwrap(), @@ -378,7 +376,7 @@ mod sp_repartition_fuzz_tests { sort_preserving_repartition_exec_hash(intermediate, hash_exprs.clone()) }; - let final_plan = sort_preserving_merge_exec(sort_keys, intermediate); + let final_plan = sort_preserving_merge_exec(sort_keys.into(), intermediate); let task_ctx = ctx.task_ctx(); let collected_running = collect(final_plan, task_ctx.clone()).await.unwrap(); @@ -425,10 +423,9 @@ mod sp_repartition_fuzz_tests { } fn sort_preserving_merge_exec( - sort_exprs: impl IntoIterator, + sort_exprs: LexOrdering, input: Arc, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 772773c91f287..5b28832c08b2d 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -643,7 +643,9 @@ async fn run_window_test( // Table is ordered according to ORDER BY a, b, c In linear test we use PARTITION BY b, ORDER BY a // For WindowAggExec to produce correct result it need table to be ordered by b,a. Hence add a sort. if is_linear { - exec1 = Arc::new(SortExec::new(sort_keys.into(), exec1)) as _; + if let Some(ordering) = LexOrdering::new(sort_keys) { + exec1 = Arc::new(SortExec::new(ordering, exec1)) as _; + } } let extended_schema = schema_add_window_field(&args, &schema, &window_fn, &fn_name)?; @@ -663,8 +665,8 @@ async fn run_window_test( false, )?) as _; let exec2 = DataSourceExec::from_data_source( - MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None)? - .try_with_sort_information(vec![source_sort_keys.clone()])?, + MemorySourceConfig::try_new(&[input1], schema, None)? + .try_with_sort_information(vec![source_sort_keys])?, ); let running_window_exec = Arc::new(BoundedWindowAggExec::try_new( vec![create_window_expr( diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 71705375a4b68..3369a8c39c7bf 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -78,8 +78,8 @@ fn parquet_exec_sorted( source, ) .with_file(PartitionedFile::new("x".to_string(), 100)); - if !sort_exprs.is_empty() { - builder = builder.with_output_ordering(vec![sort_exprs.into()]); + if let Some(ordering) = LexOrdering::new(sort_exprs) { + builder = builder.with_output_ordering(vec![ordering]); } let config = builder.build(); DataSourceExec::from_data_source(config) @@ -97,8 +97,8 @@ fn csv_exec_sorted( Arc::new(CsvSource::new(false, 0, 0)), ) .with_file(PartitionedFile::new("x".to_string(), 100)); - if !sort_exprs.is_empty() { - builder = builder.with_output_ordering(vec![sort_exprs.into()]); + if let Some(ordering) = LexOrdering::new(sort_exprs) { + builder = builder.with_output_ordering(vec![ordering]); } let config = builder.build(); @@ -1475,7 +1475,7 @@ async fn test_with_lost_ordering_unbounded_bounded( let sort_exprs = [sort_expr("a", &schema)]; // create either bounded or unbounded source let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs.clone()) + stream_exec_ordered(&schema, sort_exprs.clone().into()) } else { csv_exec_sorted(&schema, sort_exprs.clone()) }; @@ -2360,8 +2360,8 @@ async fn test_coalesce_propagate() -> Result<()> { #[tokio::test] async fn test_replace_with_partial_sort2() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("a", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + let input_ordering = [sort_expr("a", &schema), sort_expr("c", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering); let physical_plan = sort_exec( [ sort_expr("a", &schema), @@ -2442,8 +2442,8 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { #[tokio::test] async fn test_replace_with_partial_sort() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("a", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + let input_ordering = [sort_expr("a", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering); let physical_plan = sort_exec( [sort_expr("a", &schema), sort_expr("c", &schema)].into(), unbounded_input, @@ -2464,8 +2464,8 @@ async fn test_replace_with_partial_sort() -> Result<()> { #[tokio::test] async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + let input_ordering = [sort_expr("b", &schema), sort_expr("c", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering); let physical_plan = sort_exec( [ sort_expr("a", &schema), @@ -3749,10 +3749,11 @@ async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { #[test] fn test_removes_unused_orthogonal_sort() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + let input_ordering: LexOrdering = + [sort_expr("b", &schema), sort_expr("c", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering.clone()); let orthogonal_sort = sort_exec([sort_expr("a", &schema)].into(), unbounded_input); - let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort); // same sort as data source + let output_sort = sort_exec(input_ordering, orthogonal_sort); // same sort as data source // Test scenario/input has an orthogonal sort: let expected_input = [ @@ -3774,11 +3775,12 @@ fn test_removes_unused_orthogonal_sort() -> Result<()> { #[test] fn test_keeps_used_orthogonal_sort() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + let input_ordering: LexOrdering = + [sort_expr("b", &schema), sort_expr("c", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering.clone()); let orthogonal_sort = sort_exec_with_fetch([sort_expr("a", &schema)].into(), Some(3), unbounded_input); // has fetch, so this orthogonal sort changes the output - let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort); + let output_sort = sort_exec(input_ordering, orthogonal_sort); // Test scenario/input has an orthogonal sort: let expected_input = [ @@ -3798,8 +3800,9 @@ fn test_keeps_used_orthogonal_sort() -> Result<()> { #[test] fn test_handles_multiple_orthogonal_sorts() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs.clone()); + let input_ordering: LexOrdering = + [sort_expr("b", &schema), sort_expr("c", &schema)].into(); + let unbounded_input = stream_exec_ordered(&schema, input_ordering.clone()); let ordering0: LexOrdering = [sort_expr("c", &schema)].into(); let orthogonal_sort_0 = sort_exec(ordering0.clone(), unbounded_input); // has no fetch, so can be removed let ordering1: LexOrdering = [sort_expr("a", &schema)].into(); @@ -3807,7 +3810,7 @@ fn test_handles_multiple_orthogonal_sorts() -> Result<()> { sort_exec_with_fetch(ordering1.clone(), Some(3), orthogonal_sort_0); // has fetch, so this orthogonal sort changes the output let orthogonal_sort_2 = sort_exec(ordering0, orthogonal_sort_1); // has no fetch, so can be removed let orthogonal_sort_3 = sort_exec(ordering1, orthogonal_sort_2); // has no fetch, so can be removed - let output_sort = sort_exec(input_sort_exprs.into(), orthogonal_sort_3); // final sort + let output_sort = sort_exec(input_ordering, orthogonal_sort_3); // final sort // Test scenario/input has an orthogonal sort: let expected_input = [ diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 38fc94655b53c..56d48901f284d 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -19,14 +19,13 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ coalesce_batches_exec, coalesce_partitions_exec, global_limit_exec, local_limit_exec, - sort_exec, sort_preserving_merge_exec, + sort_exec, sort_preserving_merge_exec, stream_exec, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, lit, BinaryExpr}; use datafusion_physical_expr::Partitioning; @@ -37,7 +36,6 @@ use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; fn create_schema() -> SchemaRef { @@ -48,17 +46,6 @@ fn create_schema() -> SchemaRef { ])) } -fn streaming_table_exec(schema: SchemaRef) -> Result> { - Ok(Arc::new(StreamingTableExec::try_new( - Arc::clone(&schema), - vec![Arc::new(DummyStreamPartition { schema }) as _], - None, - None, - true, - None, - )?)) -} - fn projection_exec( schema: SchemaRef, input: Arc, @@ -100,24 +87,11 @@ fn empty_exec(schema: SchemaRef) -> Arc { Arc::new(EmptyExec::new(schema)) } -#[derive(Debug)] -struct DummyStreamPartition { - schema: SchemaRef, -} -impl PartitionStream for DummyStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } -} - #[test] fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero() -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema)?; + let streaming_table = stream_exec(&schema); let global_limit = global_limit_exec(streaming_table, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -142,7 +116,7 @@ fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero() -> fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_limit_when_skip_is_nonzero( ) -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema)?; + let streaming_table = stream_exec(&schema); let global_limit = global_limit_exec(streaming_table, 2, Some(5)); let initial = get_plan_string(&global_limit); @@ -168,7 +142,7 @@ fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_li fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limit( ) -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let streaming_table = stream_exec(&schema); let repartition = repartition_exec(streaming_table)?; let filter = filter_exec(schema, repartition)?; let coalesce_batches = coalesce_batches_exec(filter, 8192); @@ -206,7 +180,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi #[test] fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let streaming_table = stream_exec(&schema); let filter = filter_exec(Arc::clone(&schema), streaming_table)?; let projection = projection_exec(schema, filter)?; let global_limit = global_limit_exec(projection, 0, Some(5)); @@ -238,7 +212,7 @@ fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batches_exec_into_fetching_version( ) -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let streaming_table = stream_exec(&schema); let coalesce_batches = coalesce_batches_exec(streaming_table, 8192); let projection = projection_exec(schema, coalesce_batches)?; let global_limit = global_limit_exec(projection, 0, Some(5)); @@ -269,7 +243,7 @@ fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batc #[test] fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let streaming_table = stream_exec(&schema); let coalesce_batches = coalesce_batches_exec(streaming_table, 8192); let projection = projection_exec(Arc::clone(&schema), coalesce_batches)?; let repartition = repartition_exec(projection)?; @@ -315,7 +289,7 @@ fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions() -> Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let streaming_table = stream_exec(&schema); let repartition = repartition_exec(streaming_table)?; let filter = filter_exec(schema, repartition)?; let coalesce_partitions = coalesce_partitions_exec(filter); diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 7fc0f3166ed2d..631cb7386bd13 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -28,24 +28,25 @@ use arrow::array::{ArrayRef, Int32Array}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion_common::tree_node::{TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_common::config::ConfigOptions; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{self, col, Column}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{ + replace_with_order_preserving_variants, OrderPreservationContext +}; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::collect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::{ - displayable, get_plan_string, ExecutionPlan, Partitioning, + collect, displayable, get_plan_string, ExecutionPlan, Partitioning, }; -use datafusion::datasource::source::DataSourceExec; -use datafusion_common::tree_node::{TransformedResult, TreeNode}; -use datafusion_common::Result; -use datafusion_common::config::ConfigOptions; -use datafusion_expr::{JoinType, Operator}; -use datafusion_physical_expr::expressions::{self, col, Column}; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use object_store::memory::InMemory; use object_store::ObjectStore; @@ -188,16 +189,15 @@ async fn test_replace_multiple_input_repartition_1( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let sort_exprs: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) } else { memory_exec_sorted(&schema, sort_exprs.clone()) }; let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); - let sort = - sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), repartition); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let sort = sort_exec_with_preserve_partitioning(sort_exprs.clone(), repartition); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -257,22 +257,22 @@ async fn test_with_inter_children_change_only( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr_default("a", &schema)]; + let ordering: LexOrdering = [sort_expr_default("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let sort = sort_exec(sort_exprs.clone().into(), coalesce_partitions); + let sort = sort_exec(ordering.clone(), coalesce_partitions); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); let filter = filter_exec(repartition_hash2); - let sort2 = sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), filter); + let sort2 = sort_exec_with_preserve_partitioning(ordering.clone(), filter); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort2); + let physical_plan = sort_preserving_merge_exec(ordering, sort2); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -355,18 +355,17 @@ async fn test_replace_multiple_input_repartition_2( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); let repartition_hash = repartition_exec_hash(filter); - let sort = - sort_exec_with_preserve_partitioning(sort_exprs.clone().into(), repartition_hash); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let sort = sort_exec_with_preserve_partitioning(ordering.clone(), repartition_hash); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -431,21 +430,19 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); let coalesce_batches_exec = coalesce_batches_exec(filter, 8192); - let sort = sort_exec_with_preserve_partitioning( - sort_exprs.clone().into(), - coalesce_batches_exec, - ); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let sort = + sort_exec_with_preserve_partitioning(ordering.clone(), coalesce_batches_exec); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -515,22 +512,20 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr, 8192); let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); let filter = filter_exec(repartition_hash); let coalesce_batches_exec_2 = coalesce_batches_exec(filter, 8192); - let sort = sort_exec_with_preserve_partitioning( - sort_exprs.clone().into(), - coalesce_batches_exec_2, - ); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let sort = + sort_exec_with_preserve_partitioning(ordering.clone(), coalesce_batches_exec_2); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -605,11 +600,11 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, ordering) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, ordering) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -676,22 +671,19 @@ async fn test_with_multiple_replacable_repartitions( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let filter = filter_exec(repartition_hash); let coalesce_batches = coalesce_batches_exec(filter, 8192); let repartition_hash_2 = repartition_exec_hash(coalesce_batches); - let sort = sort_exec_with_preserve_partitioning( - sort_exprs.clone().into(), - repartition_hash_2, - ); - let physical_plan = sort_preserving_merge_exec(sort_exprs.into(), sort); + let sort = sort_exec_with_preserve_partitioning(ordering.clone(), repartition_hash_2); + let physical_plan = sort_preserving_merge_exec(ordering, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -768,18 +760,18 @@ async fn test_not_replace_with_different_orderings( use datafusion_physical_expr::LexOrdering; let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering_a = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, ordering_a) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, ordering_a) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); - let ordering: LexOrdering = + let ordering_c: LexOrdering = [sort_expr_default("c", &repartition_hash.schema())].into(); - let sort = sort_exec_with_preserve_partitioning(ordering.clone(), repartition_hash); - let physical_plan = sort_preserving_merge_exec(ordering, sort); + let sort = sort_exec_with_preserve_partitioning(ordering_c.clone(), repartition_hash); + let physical_plan = sort_preserving_merge_exec(ordering_c, sort); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -836,16 +828,16 @@ async fn test_with_lost_ordering( #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering: LexOrdering = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs.clone()) + stream_exec_ordered_with_projection(&schema, ordering.clone()) } else { - memory_exec_sorted(&schema, sort_exprs.clone()) + memory_exec_sorted(&schema, ordering.clone()) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = sort_exec(sort_exprs.into(), coalesce_partitions); + let physical_plan = sort_exec(ordering, coalesce_partitions); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -907,23 +899,23 @@ async fn test_with_lost_and_kept_ordering( use datafusion_physical_expr::LexOrdering; let schema = create_test_schema()?; - let sort_exprs = [sort_expr("a", &schema)]; + let ordering_a = [sort_expr("a", &schema)].into(); let source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, ordering_a) } else { - memory_exec_sorted(&schema, sort_exprs) + memory_exec_sorted(&schema, ordering_a) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let ordering: LexOrdering = + let ordering_c: LexOrdering = [sort_expr_default("c", &coalesce_partitions.schema())].into(); - let sort = sort_exec(ordering.clone(), coalesce_partitions); + let sort = sort_exec(ordering_c.clone(), coalesce_partitions); let repartition_rr2 = repartition_exec_round_robin(sort); let repartition_hash2 = repartition_exec_hash(repartition_rr2); let filter = filter_exec(repartition_hash2); - let sort2 = sort_exec_with_preserve_partitioning(ordering.clone(), filter); - let physical_plan = sort_preserving_merge_exec(ordering, sort2); + let sort2 = sort_exec_with_preserve_partitioning(ordering_c.clone(), filter); + let physical_plan = sort_preserving_merge_exec(ordering_c, sort2); // Expected inputs unbounded and bounded let expected_input_unbounded = [ @@ -1007,26 +999,24 @@ async fn test_with_multiple_child_trees( #[values(false, true)] source_unbounded: bool, #[values(false, true)] prefer_existing_sort: bool, ) -> Result<()> { - use datafusion_physical_expr::LexOrdering; - let schema = create_test_schema()?; - let left_sort_exprs = [sort_expr("a", &schema)]; + let left_ordering = [sort_expr("a", &schema)].into(); let left_source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, left_sort_exprs) + stream_exec_ordered_with_projection(&schema, left_ordering) } else { - memory_exec_sorted(&schema, left_sort_exprs) + memory_exec_sorted(&schema, left_ordering) }; let left_repartition_rr = repartition_exec_round_robin(left_source); let left_repartition_hash = repartition_exec_hash(left_repartition_rr); let left_coalesce_partitions = Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); - let right_sort_exprs = [sort_expr("a", &schema)]; + let right_ordering = [sort_expr("a", &schema)].into(); let right_source = if source_unbounded { - stream_exec_ordered_with_projection(&schema, right_sort_exprs) + stream_exec_ordered_with_projection(&schema, right_ordering) } else { - memory_exec_sorted(&schema, right_sort_exprs) + memory_exec_sorted(&schema, right_ordering) }; let right_repartition_rr = repartition_exec_round_robin(right_source); let right_repartition_hash = repartition_exec_hash(right_repartition_rr); @@ -1200,7 +1190,7 @@ fn create_test_schema() -> Result { // projection parameter is given static due to testing needs fn memory_exec_sorted( schema: &SchemaRef, - sort_exprs: impl IntoIterator, + ordering: LexOrdering, ) -> Arc { pub fn make_partition(schema: &SchemaRef, sz: i32) -> RecordBatch { let values = (0..sz).collect::>(); @@ -1216,7 +1206,6 @@ fn memory_exec_sorted( let rows = 5; let partitions = 1; - let sort_exprs = sort_exprs.into_iter().collect(); Arc::new({ let data: Vec> = (0..partitions) .map(|_| vec![make_partition(schema, rows)]) @@ -1225,7 +1214,7 @@ fn memory_exec_sorted( DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(&data, schema.clone(), Some(projection)) .unwrap() - .try_with_sort_information(vec![sort_exprs]) + .try_with_sort_information(vec![ordering]) .unwrap(), )) }) diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index fe53c00e8603d..4d8f069a65502 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -487,13 +487,28 @@ impl PartitionStream for TestStreamPartition { } } -/// Create an unbounded stream exec +/// Create an unbounded stream table without data ordering. +pub fn stream_exec(schema: &SchemaRef) -> Arc { + Arc::new( + StreamingTableExec::try_new( + Arc::clone(schema), + vec![Arc::new(TestStreamPartition { + schema: Arc::clone(schema), + }) as _], + None, + vec![], + true, + None, + ) + .unwrap(), + ) +} + +/// Create an unbounded stream table with data ordering. pub fn stream_exec_ordered( schema: &SchemaRef, - sort_exprs: impl IntoIterator, + ordering: LexOrdering, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( StreamingTableExec::try_new( Arc::clone(schema), @@ -501,7 +516,7 @@ pub fn stream_exec_ordered( schema: Arc::clone(schema), }) as _], None, - vec![sort_exprs], + vec![ordering], true, None, ) @@ -509,12 +524,11 @@ pub fn stream_exec_ordered( ) } -// Creates a stream exec source for the test purposes +/// Create an unbounded stream table with data ordering and built-in projection. pub fn stream_exec_ordered_with_projection( schema: &SchemaRef, - sort_exprs: impl IntoIterator, + ordering: LexOrdering, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); let projection: Vec = vec![0, 2, 3]; Arc::new( @@ -524,7 +538,7 @@ pub fn stream_exec_ordered_with_projection( schema: Arc::clone(schema), }) as _], Some(&projection), - vec![sort_exprs], + vec![ordering], true, None, ) diff --git a/datafusion/datasource/benches/split_groups_by_statistics.rs b/datafusion/datasource/benches/split_groups_by_statistics.rs index f7c5e1b44ae00..0dfd79f254475 100644 --- a/datafusion/datasource/benches/split_groups_by_statistics.rs +++ b/datafusion/datasource/benches/split_groups_by_statistics.rs @@ -15,14 +15,16 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; +use std::time::Duration; + use arrow::datatypes::{DataType, Field, Schema}; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::{generate_test_files, verify_sort_integrity}; -use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use std::sync::Arc; -use std::time::Duration; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + +use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; pub fn compare_split_groups_by_statistics_algorithms(c: &mut Criterion) { let file_schema = Arc::new(Schema::new(vec![Field::new( @@ -31,13 +33,8 @@ pub fn compare_split_groups_by_statistics_algorithms(c: &mut Criterion) { false, )])); - let sort_expr = PhysicalSortExpr { - expr: Arc::new(datafusion_physical_expr::expressions::Column::new( - "value", 0, - )), - options: arrow::compute::SortOptions::default(), - }; - let sort_ordering = LexOrdering::from(vec![sort_expr]); + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("value", 0))); + let sort_ordering = LexOrdering::from([sort_expr]); // Small, medium, large number of files let file_counts = [10, 100, 1000]; diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 444bdd15b7e34..d7aba3335ae44 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -1408,13 +1408,10 @@ fn get_projected_output_ordering( break; } - // do not push empty entries - // otherwise we may have `Some(vec![])` at the output ordering. - if new_ordering.is_empty() { + let Some(new_ordering) = LexOrdering::new(new_ordering) else { continue; - } + }; - let new_ordering = LexOrdering::from(new_ordering); // Check if any file groups are not sorted if base_config.file_groups.iter().any(|group| { if group.len() <= 1 { @@ -1481,8 +1478,8 @@ mod tests { }; use arrow::array::{Int32Array, RecordBatch}; - use datafusion_common::assert_batches_eq; use datafusion_common::stats::Precision; + use datafusion_common::{assert_batches_eq, internal_err}; use datafusion_expr::SortExpr; use datafusion_physical_expr::create_physical_sort_expr; @@ -2001,7 +1998,7 @@ mod tests { )))) .collect::>(), )); - let sort_order = LexOrdering::from( + let Some(sort_order) = LexOrdering::new( case.sort .into_iter() .map(|expr| { @@ -2012,7 +2009,9 @@ mod tests { ) }) .collect::>>()?, - ); + ) else { + return internal_err!("This test should always use an ordering"); + }; let partitioned_files = FileGroup::new( case.files.into_iter().map(From::from).collect::>(), @@ -2327,14 +2326,12 @@ mod tests { // Setup sort expression let exec_props = ExecutionProps::new(); let df_schema = DFSchema::try_from_qualified_schema("test", schema.as_ref())?; - let sort_expr = vec![col("value").sort(true, false)]; - - let physical_sort_exprs: Vec<_> = sort_expr - .iter() - .map(|expr| create_physical_sort_expr(expr, &df_schema, &exec_props).unwrap()) - .collect(); - - let sort_ordering = LexOrdering::from(physical_sort_exprs); + let sort_expr = [col("value").sort(true, false)]; + let sort_ordering = sort_expr + .map(|expr| { + create_physical_sort_expr(&expr, &df_schema, &exec_props).unwrap() + }) + .into(); // Test case parameters struct TestCase { diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 36d9ad8027b92..56887da3c6d28 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -847,8 +847,7 @@ mod memory_source_tests { options: SortOptions::default(), }] .into(); - let mut expected_output_order = vec![]; - expected_output_order.extend(sort1.clone()); + let mut expected_output_order = sort1.clone(); expected_output_order.extend(sort2.clone()); let sort_information = vec![sort1.clone(), sort2.clone()]; @@ -859,7 +858,7 @@ mod memory_source_tests { assert_eq!( mem_exec.properties().output_ordering().unwrap(), - &expected_output_order.into() + &expected_output_order ); let eq_properties = mem_exec.properties().equivalence_properties(); assert!(eq_properties.oeq_class().contains(&sort1)); diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 549704cf62023..2d9fb0ba9b6f7 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -34,7 +34,7 @@ use arrow::{ }; use datafusion_common::stats::Precision; use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; -use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; +use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::{ColumnStatistics, Statistics}; @@ -110,25 +110,20 @@ impl MinMaxStatistics { }; let sort_columns = sort_columns_from_physical_sort_exprs(projected_sort_order) - .ok_or(DataFusionError::Plan( - "sort expression must be on column".to_string(), - ))?; + .ok_or(plan_datafusion_err!("sort expression must be on column"))?; // Project the schema & sort order down to just the relevant columns let min_max_schema = Arc::new( projected_schema .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, ); - let min_max_sort_order = LexOrdering::from_iter( - sort_columns - .iter() - .zip(projected_sort_order.iter()) - .enumerate() - .map(|(i, (col, sort))| PhysicalSortExpr { - expr: Arc::new(Column::new(col.name(), i)), - options: sort.options, - }), - ); + + let mut min_max_sort_order = projected_sort_order.clone(); + for (idx, (sort_expr, col)) in + min_max_sort_order.iter_mut().zip(&sort_columns).enumerate() + { + sort_expr.expr = Arc::new(Column::new(col.name(), idx)); + } let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns .iter() @@ -186,9 +181,8 @@ impl MinMaxStatistics { .map_err(|e| e.context("create sort fields"))?; let converter = RowConverter::new(sort_fields)?; - let sort_columns = sort_columns_from_physical_sort_exprs(sort_order).ok_or( - DataFusionError::Plan("sort expression must be on column".to_string()), - )?; + let sort_columns = sort_columns_from_physical_sort_exprs(sort_order) + .ok_or(plan_datafusion_err!("sort expression must be on column"))?; // swap min/max if they're reversed in the ordering let (new_min_cols, new_max_cols): (Vec<_>, Vec<_>) = sort_order diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 3bf09134e739a..99cd978f288a5 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -134,22 +134,21 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if acc_args.order_bys.is_empty() { + let Some(ordering) = LexOrdering::new(acc_args.order_bys.to_vec()) else { return TrivialFirstValueAccumulator::try_new( acc_args.return_type, acc_args.ignore_nulls, ) .map(|acc| Box::new(acc) as _); - } - let ordering_dtypes = acc_args - .order_bys + }; + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; FirstValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.order_bys.iter().cloned().collect(), + ordering, acc_args.ignore_nulls, ) .map(|acc| { @@ -200,18 +199,17 @@ impl AggregateUDFImpl for FirstValue { fn create_accumulator( args: AccumulatorArgs, ) -> Result> { - if args.order_bys.is_empty() { + let Some(ordering) = LexOrdering::new(args.order_bys.to_vec()) else { return internal_err!("Groups accumulator must have an ordering."); - } + }; - let ordering_dtypes = args - .order_bys + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(args.schema)) .collect::>>()?; FirstPrimitiveGroupsAccumulator::::try_new( - args.order_bys.iter().cloned().collect(), + ordering, args.ignore_nulls, args.return_type, &ordering_dtypes, @@ -1057,22 +1055,21 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - if acc_args.order_bys.is_empty() { + let Some(ordering) = LexOrdering::new(acc_args.order_bys.to_vec()) else { return TrivialLastValueAccumulator::try_new( acc_args.return_type, acc_args.ignore_nulls, ) .map(|acc| Box::new(acc) as _); - } - let ordering_dtypes = acc_args - .order_bys + }; + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; LastValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.order_bys.iter().cloned().collect(), + ordering, acc_args.ignore_nulls, ) .map(|acc| { @@ -1147,14 +1144,17 @@ impl AggregateUDFImpl for LastValue { where T: ArrowPrimitiveType + Send, { - let ordering_dtypes = args - .order_bys + let Some(ordering) = LexOrdering::new(args.order_bys.to_vec()) else { + return internal_err!("Groups accumulator must have an ordering."); + }; + + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(args.schema)) .collect::>>()?; Ok(Box::new(FirstPrimitiveGroupsAccumulator::::try_new( - args.order_bys.iter().cloned().collect(), + ordering, args.ignore_nulls, args.return_type, &ordering_dtypes, diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 6b8a1e3575a92..fdd2eac2d99a8 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -148,24 +148,18 @@ impl AggregateUDFImpl for NthValueAgg { } }; - if acc_args.order_bys.is_empty() { + let Some(ordering) = LexOrdering::new(acc_args.order_bys.to_vec()) else { return TrivialNthValueAccumulator::try_new(n, acc_args.return_type) .map(|acc| Box::new(acc) as _); }; - let ordering_dtypes = acc_args - .order_bys + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; let data_type = acc_args.exprs[0].data_type(acc_args.schema)?; - NthValueAccumulator::try_new( - n, - &data_type, - &ordering_dtypes, - acc_args.order_bys.iter().cloned().collect(), - ) - .map(|acc| Box::new(acc) as _) + NthValueAccumulator::try_new(n, &data_type, &ordering_dtypes, ordering) + .map(|acc| Box::new(acc) as _) } fn state_fields(&self, args: StateFieldsArgs) -> Result> { diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 070ecd91c61ac..bcbbac7c2f451 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -394,12 +394,6 @@ impl LexOrdering { self.inner.truncate(len); true } - - /// Transforms each `PhysicalSortExpr` in the `LexOrdering` - /// in place using the provided closure `f`. - pub fn transform(&mut self, f: F) { - self.inner.iter_mut().for_each(f); - } } impl PartialOrd for LexOrdering { @@ -414,12 +408,6 @@ impl PartialOrd for LexOrdering { } } -impl From> for LexOrdering { - fn from(value: Vec) -> Self { - Self::new(value).unwrap() - } -} - impl From<[PhysicalSortExpr; N]> for LexOrdering { fn from(value: [PhysicalSortExpr; N]) -> Self { // TODO: Replace this with a condition on the generic parameter when @@ -460,12 +448,6 @@ impl Display for LexOrdering { } } -impl FromIterator for LexOrdering { - fn from_iter>(iter: T) -> Self { - Self::new(iter).unwrap() - } -} - impl IntoIterator for LexOrdering { type Item = PhysicalSortExpr; type IntoIter = IntoIter; diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 8c28fd96d2e38..2a65cbddb43ab 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::borrow::Borrow; use std::sync::Arc; use crate::expressions::Column; use crate::PhysicalExpr; +use arrow::compute::SortOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; mod class; mod ordering; @@ -53,10 +56,29 @@ pub fn add_offset_to_expr( // an `Ok` value. } +// Convert each tuple to a `PhysicalSortExpr` and construct a vector. +pub fn convert_to_sort_exprs>>( + args: &[(T, SortOptions)], +) -> Vec { + args.iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: Arc::clone(expr.borrow()), + options: *options, + }) + .collect() +} + +// Convert each vector of tuples to a `LexOrdering`. +pub fn convert_to_orderings>>( + args: &[Vec<(T, SortOptions)>], +) -> Vec { + args.iter() + .filter_map(|sort_exprs| LexOrdering::new(convert_to_sort_exprs(sort_exprs))) + .collect() +} + #[cfg(test)] mod tests { - use std::borrow::Borrow; - use super::*; use crate::expressions::col; use crate::{LexRequirement, PhysicalSortExpr}; @@ -64,9 +86,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{plan_datafusion_err, Result}; - use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, PhysicalSortRequirement, - }; + use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; /// Converts a string to a physical sort expression /// @@ -200,27 +220,6 @@ mod tests { .collect() } - // Convert each tuple to a `PhysicalSortExpr` and construct a vector. - pub fn convert_to_sort_exprs>>( - args: &[(T, SortOptions)], - ) -> Vec { - args.iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr.borrow()), - options: *options, - }) - .collect() - } - - // Convert each vector of tuples to a `LexOrdering`. - pub fn convert_to_orderings>>( - args: &[Vec<(T, SortOptions)>], - ) -> Vec { - args.iter() - .map(|sort_exprs| convert_to_sort_exprs(sort_exprs).into()) - .collect() - } - #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index a073345f6e147..30a593878a458 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -214,9 +214,9 @@ impl OrderingEquivalenceClass { /// ordering equivalence class. pub fn add_offset(&mut self, offset: usize) { for ordering in self.orderings.iter_mut() { - ordering.transform(|sort_expr| { + for sort_expr in ordering.iter_mut() { sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); - }) + } } } @@ -338,12 +338,10 @@ impl Display for OrderingEquivalenceClass { mod tests { use std::sync::Arc; - use crate::equivalence::tests::{ - convert_to_orderings, convert_to_sort_exprs, create_test_schema, - }; + use crate::equivalence::tests::create_test_schema; use crate::equivalence::{ - EquivalenceClass, EquivalenceGroup, EquivalenceProperties, - OrderingEquivalenceClass, + convert_to_orderings, convert_to_sort_exprs, EquivalenceClass, EquivalenceGroup, + EquivalenceProperties, OrderingEquivalenceClass, }; use crate::expressions::{col, BinaryExpr, Column}; use crate::utils::tests::TestScalarUDF; diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index bc4e12e94be5b..2215fcbfcb730 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -138,8 +138,8 @@ fn project_index_to_exprs( #[cfg(test)] mod tests { use super::*; - use crate::equivalence::tests::{convert_to_orderings, output_schema}; - use crate::equivalence::EquivalenceProperties; + use crate::equivalence::tests::output_schema; + use crate::equivalence::{convert_to_orderings, EquivalenceProperties}; use crate::expressions::{col, BinaryExpr}; use crate::utils::tests::TestScalarUDF; use crate::{PhysicalExprRef, ScalarFunctionExpr}; diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index d3f1a89a140a0..585053abe69ee 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -378,42 +378,37 @@ pub fn construct_prefix_orderings( /// # Parameters /// /// * `dependencies` - Set of relevant expressions. -/// * `dependency_map` - Map of dependencies for expressions that may appear in `dependencies` +/// * `dependency_map` - Map of dependencies for expressions that may appear in +/// `dependencies`. /// /// # Returns /// -/// A vector of lexical orderings (`Vec`) representing all valid orderings -/// based on the given dependencies. +/// A vector of lexical orderings (`Vec`) representing all valid +/// orderings based on the given dependencies. pub fn generate_dependency_orderings( dependencies: &Dependencies, dependency_map: &DependencyMap, ) -> Vec { // Construct all the valid prefix orderings for each expression appearing - // in the projection: - let relevant_prefixes = dependencies + // in the projection. Note that if relevant prefixes are empty, there is no + // dependency, meaning that dependent is a leading ordering. + dependencies .iter() .flat_map(|dep| { let prefixes = construct_prefix_orderings(dep, dependency_map); (!prefixes.is_empty()).then_some(prefixes) }) - .collect::>(); - // Note that if relevant prefixes are empty, there is no dependency, - // meaning that dependent is a leading ordering. - - // Generate all possible valid orderings: - relevant_prefixes - .into_iter() + // Generate all possible valid orderings: .multi_cartesian_product() .flat_map(|prefix_orderings| { + let length = prefix_orderings.len(); prefix_orderings - .iter() - .permutations(prefix_orderings.len()) + .into_iter() + .permutations(length) .filter_map(|prefixes| { - (!prefixes.is_empty()).then(|| { - prefixes - .into_iter() - .flat_map(|ordering| ordering.clone()) - .collect() + prefixes.into_iter().reduce(|mut acc, ordering| { + acc.extend(ordering); + acc }) }) .collect::>() @@ -428,10 +423,10 @@ mod tests { use super::*; use crate::equivalence::tests::{ - convert_to_sort_exprs, convert_to_sort_reqs, create_test_params, - create_test_schema, output_schema, parse_sort_expr, + convert_to_sort_reqs, create_test_params, create_test_schema, output_schema, + parse_sort_expr, }; - use crate::equivalence::ProjectionMapping; + use crate::equivalence::{convert_to_sort_exprs, ProjectionMapping}; use crate::expressions::{col, BinaryExpr, CastExpr, Column}; use crate::{ConstExpr, EquivalenceProperties, ScalarFunctionExpr}; @@ -440,7 +435,6 @@ mod tests { use datafusion_common::{Constraint, Constraints, Result}; use datafusion_expr::sort_properties::SortProperties; use datafusion_expr::Operator; - use datafusion_functions::string::concat; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ @@ -1124,13 +1118,8 @@ mod tests { let sort = case .sort_columns .iter() - .map(|&name| { - col(name, &schema).map(|col| PhysicalSortExpr { - expr: col, - options: SortOptions::default(), - }) - }) - .collect::>()?; + .map(|&name| col(name, &schema).map(PhysicalSortExpr::new_default)) + .collect::>>()?; assert_eq!( properties.ordering_satisfy(sort), @@ -1178,13 +1167,12 @@ mod tests { let orderings = eq_properties.oeq_class(); let expected_ordering1 = - LexOrdering::from(vec![ - PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc() - ]); - let expected_ordering2 = LexOrdering::from(vec![ + [PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc()].into(); + let expected_ordering2 = [ PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), - ]); + ] + .into(); // The ordering should be [c ASC] and [a ASC, b ASC] assert_eq!(orderings.len(), 2); @@ -1206,20 +1194,21 @@ mod tests { let col_b = col("b", &schema)?; let col_c = col("c", &schema)?; - let a_times_b: Arc = Arc::new(BinaryExpr::new( + let a_times_b = Arc::new(BinaryExpr::new( Arc::clone(&col_a), Operator::Multiply, Arc::clone(&col_b), - )); + )) as _; // Assume existing ordering is [c ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - let initial_ordering = LexOrdering::from(vec![ + let initial_ordering: LexOrdering = [ PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc(), - PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), - PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), - ]); + PhysicalSortExpr::new_default(col_a).asc(), + PhysicalSortExpr::new_default(col_b).asc(), + ] + .into(); eq_properties.add_new_ordering(initial_ordering.clone()); @@ -1247,12 +1236,12 @@ mod tests { let col_b = col("b", &schema)?; let col_c = col("c", &schema)?; - let a_concat_b: Arc = Arc::new(ScalarFunctionExpr::new( + let a_concat_b = Arc::new(ScalarFunctionExpr::new( "concat", concat(), vec![Arc::clone(&col_a), Arc::clone(&col_b)], DataType::Utf8, - )); + )) as _; // Assume existing ordering is [concat(a, b) ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); @@ -1268,14 +1257,12 @@ mod tests { let orderings = eq_properties.oeq_class(); - let expected_ordering1 = LexOrdering::from(vec![PhysicalSortExpr::new_default( - Arc::clone(&a_concat_b), - ) - .asc()]); - let expected_ordering2 = LexOrdering::from(vec![ - PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), - PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), - ]); + let expected_ordering1 = [PhysicalSortExpr::new_default(a_concat_b).asc()].into(); + let expected_ordering2 = [ + PhysicalSortExpr::new_default(col_a).asc(), + PhysicalSortExpr::new_default(col_b).asc(), + ] + .into(); // The ordering should be [concat(a, b) ASC] and [a ASC, b ASC] assert_eq!(orderings.len(), 2); diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 7944e89d0305a..86f55a2302988 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -136,17 +136,16 @@ pub fn updated_right_ordering_equivalence_class( #[cfg(test)] mod tests { - use std::sync::Arc; use super::*; - use crate::equivalence::add_offset_to_expr; - use crate::equivalence::tests::{convert_to_orderings, create_test_schema}; + use crate::equivalence::tests::create_test_schema; + use crate::equivalence::{add_offset_to_expr, convert_to_orderings}; use crate::expressions::col; - use datafusion_common::Result; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Fields, Schema}; + use datafusion_common::Result; #[test] fn test_join_equivalence_properties() -> Result<()> { diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 04b480579553e..54aae774ad392 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -45,7 +45,9 @@ use crate::{ use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{plan_err, Constraint, Constraints, HashMap, Result}; +use datafusion_common::{ + internal_datafusion_err, plan_err, Constraint, Constraints, HashMap, Result, +}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_physical_expr_common::utils::ExprPropertiesNode; @@ -820,12 +822,17 @@ impl EquivalenceProperties { }) .collect::>>()?; // Generate all valid orderings, given substituted expressions. - let result = new_orderings + new_orderings .into_iter() .multi_cartesian_product() - .map(Into::into) - .collect(); - Ok(result) + .map(|o| { + LexOrdering::new(o).ok_or_else(|| { + internal_datafusion_err!( + "Expected a non-empty list of sort expressions" + ) + }) + }) + .collect() } /// In projection, supposed we have a input function 'A DESC B DESC' and the output shares the same expression diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 3ec0a604671b0..373f251203951 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -833,7 +833,7 @@ mod tests { ordering .iter() .map(|name| parse_sort_expr(name, schema)) - .collect::() + .collect::>() }) .collect::>(); diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index b4d0758fd2e81..4d01389f00f61 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -33,8 +33,6 @@ use datafusion_common::tree_node::{ use datafusion_common::{HashMap, HashSet, Result}; use datafusion_expr::Operator; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -266,15 +264,6 @@ pub fn reassign_predicate_columns( .data() } -/// Merge left and right sort expressions, checking for duplicates. -pub fn merge_vectors(left: &LexOrdering, right: &LexOrdering) -> LexOrdering { - left.iter() - .cloned() - .chain(right.iter().cloned()) - .unique() - .collect() -} - #[cfg(test)] pub(crate) mod tests { use std::any::Any; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 12f299a9fc6b5..819d21a871d85 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1055,8 +1055,7 @@ fn get_aggregate_expr_req( if !aggr_expr.order_sensitivity().hard_requires() || !agg_mode.is_first_stage() { return None; } - let mut req = aggr_expr.order_bys().to_vec(); - + let mut sort_exprs = aggr_expr.order_bys().to_vec(); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we // consider the ordering of each partial result. Hence, we do not need to @@ -1067,11 +1066,11 @@ fn get_aggregate_expr_req( // will definitely be satisfied -- Each group by expression will have // distinct values per group, hence all requirements are satisfied. let physical_exprs = group_by.input_exprs(); - req.retain(|sort_expr| { + sort_exprs.retain(|sort_expr| { !physical_exprs_contains(&physical_exprs, &sort_expr.expr) }); } - (!req.is_empty()).then(|| req.into()) + LexOrdering::new(sort_exprs) } /// Concatenates the given slices. diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 927ea2f78a466..ebaba26066736 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -21,6 +21,8 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; +use super::order::GroupOrdering; +use super::AggregateExec; use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ @@ -36,7 +38,6 @@ use crate::{aggregates, metrics, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; -use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; @@ -44,13 +45,11 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; - -use super::order::GroupOrdering; -use super::AggregateExec; -use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; + use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; @@ -519,15 +518,20 @@ impl GroupedHashAggregateStream { let partial_agg_schema = Arc::new(partial_agg_schema); - let spill_expr = group_schema - .fields - .into_iter() - .enumerate() - .map(|(idx, field)| PhysicalSortExpr { - expr: Arc::new(Column::new(field.name().as_str(), idx)) as _, - options: SortOptions::default(), - }) - .collect(); + let spill_expr = + group_schema + .fields + .into_iter() + .enumerate() + .map(|(idx, field)| { + PhysicalSortExpr::new_default(Arc::new(Column::new( + field.name().as_str(), + idx, + )) as _) + }); + let Some(spill_expr) = LexOrdering::new(spill_expr) else { + return internal_err!("Spill expression is empty"); + }; let name = format!("GroupedHashAggregateStream[{partition}]"); let reservation = MemoryConsumer::new(name) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 16cb993d156c5..fd907eef6748f 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1053,7 +1053,7 @@ pub(crate) mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; - use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use insta::assert_snapshot; use rstest::rstest; @@ -1081,24 +1081,21 @@ pub(crate) mod tests { vec![batch] }; - let mut source = - TestMemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); - if !sorted_column_names.is_empty() { - let mut sort_info = vec![]; - for name in sorted_column_names { - let index = schema.index_of(name).unwrap(); - let sort_expr = PhysicalSortExpr { - expr: Arc::new(Column::new(name, index)), - options: SortOptions { - descending: false, - nulls_first: false, - }, - }; - sort_info.push(sort_expr); - } - source = source - .try_with_sort_information(vec![sort_info.into()]) - .unwrap(); + let mut sort_info = vec![]; + for name in sorted_column_names { + let index = schema.index_of(name).unwrap(); + let sort_expr = PhysicalSortExpr { + expr: Arc::new(Column::new(name, index)), + options: SortOptions { + descending: false, + nulls_first: false, + }, + }; + sort_info.push(sort_expr); + } + let mut source = TestMemoryExec::try_new(&[batches], schema, None).unwrap(); + if let Some(ordering) = LexOrdering::new(sort_info) { + source = source.try_with_sort_information(vec![ordering]).unwrap(); } Arc::new(TestMemoryExec::update_cache(Arc::new(source))) diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index eeda2c19435eb..8372334fb50e0 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -52,10 +52,8 @@ use datafusion_common::{ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::utils::{collect_columns, merge_vectors}; -use datafusion_physical_expr::{ - LexOrdering, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, -}; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{LexOrdering, PhysicalExpr, PhysicalExprRef}; use crate::joins::SharedBitmapBuilder; use crate::projection::ProjectionExec; @@ -134,7 +132,7 @@ fn replace_on_columns_of_right_ordering( right_ordering: &mut LexOrdering, ) { for (left_col, right_col) in on_columns { - right_ordering.transform(|item| { + for item in right_ordering.iter_mut() { item.expr = Arc::clone(&item.expr) .transform(|e| { if e.eq(right_col) { @@ -145,7 +143,7 @@ fn replace_on_columns_of_right_ordering( }) .data() .expect("closure is infallible"); - }); + } } } @@ -157,13 +155,13 @@ fn offset_ordering( match join_type { // In the case below, right ordering should be offsetted with the left // side length, since we append the right table to the left table. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => ordering - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: add_offset_to_expr(Arc::clone(&sort_expr.expr), offset), - options: sort_expr.options, - }) - .collect(), + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + let mut ordering = ordering.clone(); + for sort_expr in ordering.iter_mut() { + sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); + } + ordering + } _ => ordering.clone(), } } @@ -190,7 +188,9 @@ pub fn calculate_join_output_ordering( let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); return if let Some(left_ordering) = left_ordering { - Some(merge_vectors(left_ordering, &right_offset)) + let mut result = left_ordering.clone(); + result.extend(right_offset); + Some(result) } else { Some(right_offset) }; @@ -206,13 +206,12 @@ pub fn calculate_join_output_ordering( on_columns, &mut right_ordering.clone(), ); - let right_offset = + let mut right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); if let Some(left_ordering) = left_ordering { - Some(merge_vectors(&right_offset, left_ordering)) - } else { - Some(right_offset) + right_offset.extend(left_ordering.clone()); } + Some(right_offset) } else { left_ordering.cloned() }; @@ -1503,15 +1502,16 @@ pub(super) fn swap_join_projection( #[cfg(test)] mod tests { - use super::*; use std::pin::Pin; + use super::*; + use arrow::array::Int32Array; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Fields}; use arrow::error::{ArrowError, Result as ArrowResult}; use datafusion_common::stats::Precision::{Absent, Exact, Inexact}; use datafusion_common::{arrow_datafusion_err, arrow_err, ScalarValue}; + use datafusion_physical_expr::PhysicalSortExpr; use rstest::rstest; @@ -2322,30 +2322,14 @@ mod tests { #[test] fn test_calculate_join_output_ordering() -> Result<()> { - let options = SortOptions::default(); let left_ordering = LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))), + PhysicalSortExpr::new_default(Arc::new(Column::new("c", 2))), + PhysicalSortExpr::new_default(Arc::new(Column::new("d", 3))), ]); let right_ordering = LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("z", 2))), + PhysicalSortExpr::new_default(Arc::new(Column::new("y", 1))), ]); let join_type = JoinType::Inner; let on_columns = [( @@ -2358,48 +2342,18 @@ mod tests { let expected = [ LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 7)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 6)), - options, - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))), + PhysicalSortExpr::new_default(Arc::new(Column::new("c", 2))), + PhysicalSortExpr::new_default(Arc::new(Column::new("d", 3))), + PhysicalSortExpr::new_default(Arc::new(Column::new("z", 7))), + PhysicalSortExpr::new_default(Arc::new(Column::new("y", 6))), ]), LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 7)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 6)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("z", 7))), + PhysicalSortExpr::new_default(Arc::new(Column::new("y", 6))), + PhysicalSortExpr::new_default(Arc::new(Column::new("a", 0))), + PhysicalSortExpr::new_default(Arc::new(Column::new("c", 2))), + PhysicalSortExpr::new_default(Arc::new(Column::new("d", 3))), ]), ]; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 72934c74446eb..09543ddd91871 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -635,7 +635,7 @@ pub fn update_expr( let mut state = RewriteState::Unchanged; let new_expr = Arc::clone(expr) - .transform_up(|expr: Arc| { + .transform_up(|expr| { if state == RewriteState::RewrittenInvalid { return Ok(Transformed::no(expr)); } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 3feaa1ed0fd51..65862827b0654 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -371,12 +371,10 @@ impl ExternalSorter { streams.push(stream); } - let expressions: LexOrdering = self.expr.iter().cloned().collect(); - StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(&expressions) + .with_expressions(&self.expr.clone()) .with_metrics(self.metrics.baseline.clone()) .with_batch_size(self.batch_size) .with_fetch(None) @@ -694,12 +692,10 @@ impl ExternalSorter { }) .collect::>()?; - let expressions: LexOrdering = self.expr.iter().cloned().collect(); - StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(&expressions) + .with_expressions(&self.expr.clone()) .with_metrics(metrics) .with_batch_size(self.batch_size) .with_fetch(None) @@ -723,7 +719,7 @@ impl ExternalSorter { ); let schema = batch.schema(); - let expressions: LexOrdering = self.expr.iter().cloned().collect(); + let expressions = self.expr.clone(); let row_converter = Arc::clone(&self.sort_keys_row_converter); let stream = futures::stream::once(async move { let _timer = metrics.elapsed_compute().timer(); @@ -1289,25 +1285,19 @@ impl ExecutionPlan for SortExec { return Ok(None); } - let mut updated_exprs = vec![]; - for sort in self.expr() { - let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? + let mut updated_exprs = self.expr().clone(); + for sort in updated_exprs.iter_mut() { + let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? else { return Ok(None); }; - updated_exprs.push(PhysicalSortExpr { - expr: new_expr, - options: sort.options, - }); + sort.expr = updated_expr; } Ok(Some(Arc::new( - SortExec::new( - updated_exprs.into(), - make_with_child(projection, self.input())?, - ) - .with_fetch(self.fetch()) - .with_preserve_partitioning(self.preserve_partitioning()), + SortExec::new(updated_exprs, make_with_child(projection, self.input())?) + .with_fetch(self.fetch()) + .with_preserve_partitioning(self.preserve_partitioning()), ))) } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index bcfc880f054a8..6c5dd7cfaea20 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,9 +33,7 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, OrderingRequirements, PhysicalSortExpr, -}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, OrderingRequirements}; use log::{debug, trace}; @@ -363,21 +361,18 @@ impl ExecutionPlan for SortPreservingMergeExec { return Ok(None); } - let mut updated_exprs = vec![]; - for sort in self.expr() { + let mut updated_exprs = self.expr().clone(); + for sort in updated_exprs.iter_mut() { let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? else { return Ok(None); }; - updated_exprs.push(PhysicalSortExpr { - expr: updated_expr, - options: sort.options, - }); + sort.expr = updated_expr; } Ok(Some(Arc::new( SortPreservingMergeExec::new( - updated_exprs.into(), + updated_exprs, make_with_child(projection, self.input())?, ) .with_fetch(self.fetch()), diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 2cba30a98275c..c7ce18149411f 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -35,7 +35,7 @@ use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use async_trait::async_trait; use futures::stream::StreamExt; @@ -306,20 +306,16 @@ impl ExecutionPlan for StreamingTableExec { ); let mut lex_orderings = vec![]; - for lex_ordering in self.projected_output_ordering().into_iter() { - let mut orderings = vec![]; - for order in lex_ordering { - let Some(new_ordering) = - update_expr(&order.expr, projection.expr(), false)? + for mut ordering in self.projected_output_ordering().into_iter() { + for sort_expr in ordering.iter_mut() { + let Some(new_sort_expr) = + update_expr(&sort_expr.expr, projection.expr(), false)? else { return Ok(None); }; - orderings.push(PhysicalSortExpr { - expr: new_ordering, - options: order.options, - }); + sort_expr.expr = new_sort_expr; } - lex_orderings.push(orderings.into()); + lex_orderings.push(ordering); } StreamingTableExec::try_new( diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 0012b06f4e012..d8b6f0e400b83 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -818,7 +818,7 @@ mod tests { // Input ordering uses only column "a" (a prefix of the full sort). let prefix = vec![sort_expr_a.clone()]; - let full_expr = LexOrdering::from(vec![sort_expr_a, sort_expr_b]); + let full_expr = LexOrdering::from([sort_expr_a, sort_expr_b]); // Create a dummy runtime environment and metrics. let runtime = Arc::new(RuntimeEnv::default()); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 2b666093f29e0..b0975312abd5d 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -642,15 +642,13 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; - use crate::test; - use crate::test::TestMemoryExec; + use crate::test::{self, TestMemoryExec}; use arrow::compute::SortOptions; use arrow::datatypes::DataType; use datafusion_common::ScalarValue; + use datafusion_physical_expr::equivalence::convert_to_orderings; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; - use datafusion_physical_expr_common::sort_expr::LexOrdering; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { @@ -666,19 +664,6 @@ mod tests { Ok(schema) } - // Convert each tuple to PhysicalSortExpr - fn convert_to_sort_exprs( - in_data: &[(&Arc, SortOptions)], - ) -> LexOrdering { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(*expr), - options: *options, - }) - .collect::() - } - #[tokio::test] async fn test_union_partitions() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); @@ -854,18 +839,9 @@ mod tests { (first_child_orderings, second_child_orderings, union_orderings), ) in test_cases.iter().enumerate() { - let first_orderings = first_child_orderings - .iter() - .map(|ordering| convert_to_sort_exprs(ordering)) - .collect::>(); - let second_orderings = second_child_orderings - .iter() - .map(|ordering| convert_to_sort_exprs(ordering)) - .collect::>(); - let union_expected_orderings = union_orderings - .iter() - .map(|ordering| convert_to_sort_exprs(ordering)) - .collect::>(); + let first_orderings = convert_to_orderings(first_child_orderings); + let second_orderings = convert_to_orderings(second_child_orderings); + let union_expected_orderings = convert_to_orderings(union_orderings); let child1 = Arc::new(TestMemoryExec::update_cache(Arc::new( TestMemoryExec::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d8ea48b3a8c3a..2b1dea1715bc5 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -696,16 +696,14 @@ mod tests { /// Created a sorted Streaming Table exec pub fn streaming_table_exec( schema: &SchemaRef, - sort_exprs: impl IntoIterator, + ordering: LexOrdering, infinite_source: bool, ) -> Result> { - let sort_exprs = sort_exprs.into_iter().collect(); - Ok(Arc::new(StreamingTableExec::try_new( Arc::clone(schema), vec![], None, - Some(sort_exprs), + Some(ordering), infinite_source, None, )?)) @@ -912,13 +910,14 @@ mod tests { // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST // Column e is not ordered. - let sort_exprs = vec![ + let ordering = [ sort_expr("a", &test_schema), sort_expr("b", &test_schema), sort_expr("c", &test_schema), sort_expr("d", &test_schema), - ]; - let exec_unbounded = streaming_table_exec(&test_schema, sort_exprs, true)?; + ] + .into(); + let exec_unbounded = streaming_table_exec(&test_schema, ordering, true)?; // test cases consists of vector of tuples. Where each tuple represents a single test case. // First field in the tuple is Vec where each element in the vector represents PARTITION BY columns @@ -1032,13 +1031,14 @@ mod tests { // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST // Column e is not ordered. - let sort_exprs = vec![ + let ordering = [ sort_expr("a", &test_schema), sort_expr("b", &test_schema), sort_expr("c", &test_schema), sort_expr("d", &test_schema), - ]; - let exec_unbounded = streaming_table_exec(&test_schema, sort_exprs, true)?; + ] + .into(); + let exec_unbounded = streaming_table_exec(&test_schema, ordering, true)?; // test cases consists of vector of tuples. Where each tuple represents a single test case. // First field in the tuple is Vec where each element in the vector represents PARTITION BY columns diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 4ee504ac6ca93..d8a724f1a8d44 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -37,7 +37,7 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; -use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, Literal, NegativeExpr, NotExpr, TryCastExpr, UnKnownColumn, @@ -525,13 +525,13 @@ pub fn parse_protobuf_file_scan_config( let mut output_ordering = vec![]; for node_collection in &proto.output_ordering { - let sort_expr = parse_physical_sort_exprs( + let sort_exprs = parse_physical_sort_exprs( &node_collection.physical_sort_expr_nodes, registry, &schema, codec, )?; - output_ordering.push(sort_expr.into()); + output_ordering.extend(LexOrdering::new(sort_exprs)); } let config = FileScanConfigBuilder::new(object_store_url, file_schema, file_source) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 9e5769b62a482..034981482ac0f 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1284,11 +1284,7 @@ impl protobuf::PhysicalPlanNode { &left_schema, extension_codec, )?; - let left_sort_exprs = if left_sort_exprs.is_empty() { - None - } else { - Some(left_sort_exprs.into()) - }; + let left_sort_exprs = LexOrdering::new(left_sort_exprs); let right_sort_exprs = parse_physical_sort_exprs( &sym_join.right_sort_exprs, @@ -1296,11 +1292,7 @@ impl protobuf::PhysicalPlanNode { &right_schema, extension_codec, )?; - let right_sort_exprs = if right_sort_exprs.is_empty() { - None - } else { - Some(right_sort_exprs.into()) - }; + let right_sort_exprs = LexOrdering::new(right_sort_exprs); let partition_mode = protobuf::StreamPartitionMode::try_from( sym_join.partition_mode, @@ -1412,47 +1404,45 @@ impl protobuf::PhysicalPlanNode { runtime: &RuntimeEnv, extension_codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let input: Arc = - into_physical_plan(&sort.input, registry, runtime, extension_codec)?; + let input = into_physical_plan(&sort.input, registry, runtime, extension_codec)?; let exprs = sort - .expr - .iter() - .map(|expr| { - let expr = expr.expr_type.as_ref().ok_or_else(|| { + .expr + .iter() + .map(|expr| { + let expr = expr.expr_type.as_ref().ok_or_else(|| { + proto_error(format!( + "physical_plan::from_proto() Unexpected expr {self:?}" + )) + })?; + if let ExprType::Sort(sort_expr) = expr { + let expr = sort_expr + .expr + .as_ref() + .ok_or_else(|| { proto_error(format!( - "physical_plan::from_proto() Unexpected expr {self:?}" + "physical_plan::from_proto() Unexpected sort expr {self:?}" )) - })?; - if let ExprType::Sort(sort_expr) = expr { - let expr = sort_expr - .expr - .as_ref() - .ok_or_else(|| { - proto_error(format!( - "physical_plan::from_proto() Unexpected sort expr {self:?}" - )) - })? - .as_ref(); - Ok(PhysicalSortExpr { - expr: parse_physical_expr(expr, registry, input.schema().as_ref(), extension_codec)?, - options: SortOptions { - descending: !sort_expr.asc, - nulls_first: sort_expr.nulls_first, - }, - }) - } else { - internal_err!( - "physical_plan::from_proto() {self:?}" - ) - } + })? + .as_ref(); + Ok(PhysicalSortExpr { + expr: parse_physical_expr(expr, registry, input.schema().as_ref(), extension_codec)?, + options: SortOptions { + descending: !sort_expr.asc, + nulls_first: sort_expr.nulls_first, + }, }) - .collect::>()?; - let fetch = if sort.fetch < 0 { - None - } else { - Some(sort.fetch as usize) + } else { + internal_err!( + "physical_plan::from_proto() {self:?}" + ) + } + }) + .collect::>>()?; + let Some(ordering) = LexOrdering::new(exprs) else { + return internal_err!("SortExec requires an ordering"); }; - let new_sort = SortExec::new(exprs, input) + let fetch = (sort.fetch >= 0).then_some(sort.fetch as usize); + let new_sort = SortExec::new(ordering, input) .with_fetch(fetch) .with_preserve_partitioning(sort.preserve_partitioning); @@ -1466,8 +1456,7 @@ impl protobuf::PhysicalPlanNode { runtime: &RuntimeEnv, extension_codec: &dyn PhysicalExtensionCodec, ) -> Result> { - let input: Arc = - into_physical_plan(&sort.input, registry, runtime, extension_codec)?; + let input = into_physical_plan(&sort.input, registry, runtime, extension_codec)?; let exprs = sort .expr .iter() @@ -1503,14 +1492,13 @@ impl protobuf::PhysicalPlanNode { internal_err!("physical_plan::from_proto() {self:?}") } }) - .collect::>()?; - let fetch = if sort.fetch < 0 { - None - } else { - Some(sort.fetch as usize) + .collect::>>()?; + let Some(ordering) = LexOrdering::new(exprs) else { + return internal_err!("SortExec requires an ordering"); }; + let fetch = (sort.fetch >= 0).then_some(sort.fetch as usize); Ok(Arc::new( - SortPreservingMergeExec::new(exprs, input).with_fetch(fetch), + SortPreservingMergeExec::new(ordering, input).with_fetch(fetch), )) } From 95f71829ecccc9f42c91e4d60234b1e833ef8ec6 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 00:35:41 +0300 Subject: [PATCH 090/167] Transition to fallible LexOrdering constructor - 7 --- .../combine_partial_final_agg.rs | 20 +-- .../enforce_distribution.rs | 34 +++-- .../physical_optimizer/enforce_sorting.rs | 141 ++++++++---------- .../limited_distinct_aggregation.rs | 8 +- .../tests/physical_optimizer/test_utils.rs | 7 +- datafusion/proto/src/physical_plan/mod.rs | 4 +- 6 files changed, 97 insertions(+), 117 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 568be0d18f245..de8f7b36a3a70 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -136,7 +136,7 @@ fn aggregations_not_combined() -> datafusion_common::Result<()> { let plan = final_aggregate_exec( repartition_exec(partial_aggregate_exec( - parquet_exec(&schema), + parquet_exec(schema.clone()), PhysicalGroupBy::default(), aggr_expr.clone(), )), @@ -157,7 +157,7 @@ fn aggregations_not_combined() -> datafusion_common::Result<()> { let plan = final_aggregate_exec( partial_aggregate_exec( - parquet_exec(&schema), + parquet_exec(schema), PhysicalGroupBy::default(), aggr_expr1, ), @@ -183,7 +183,7 @@ fn aggregations_combined() -> datafusion_common::Result<()> { let plan = final_aggregate_exec( partial_aggregate_exec( - parquet_exec(&schema), + parquet_exec(schema), PhysicalGroupBy::default(), aggr_expr.clone(), ), @@ -215,11 +215,8 @@ fn aggregations_with_group_combined() -> datafusion_common::Result<()> { vec![(col("c", &schema)?, "c".to_string())]; let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); + let partial_agg = + partial_aggregate_exec(parquet_exec(schema), partial_group_by, aggr_expr.clone()); let groups: Vec<(Arc, String)> = vec![(col("c", &partial_agg.schema())?, "c".to_string())]; @@ -245,11 +242,8 @@ fn aggregations_with_limit_combined() -> datafusion_common::Result<()> { vec![(col("c", &schema)?, "c".to_string())]; let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); + let partial_agg = + partial_aggregate_exec(parquet_exec(schema), partial_group_by, aggr_expr.clone()); let groups: Vec<(Arc, String)> = vec![(col("c", &partial_agg.schema())?, "c".to_string())]; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index db66c230dc998..33b012f6ce5b8 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use crate::physical_optimizer::test_utils::{ check_integrity, coalesce_partitions_exec, parquet_exec_with_sort, repartition_exec, schema, sort_exec, sort_exec_with_preserve_partitioning, sort_merge_join_exec, - sort_preserving_merge_exec, + sort_preserving_merge_exec, union_exec, }; use arrow::compute::SortOptions; @@ -167,7 +167,7 @@ impl ExecutionPlan for SortRequiredExec { } fn parquet_exec() -> Arc { - parquet_exec_with_sort(vec![]) + parquet_exec_with_sort(schema(), vec![]) } fn parquet_exec_multiple() -> Arc { @@ -321,10 +321,6 @@ fn limit_exec(input: Arc) -> Arc { )) } -fn union_exec(input: Vec>) -> Arc { - Arc::new(UnionExec::new(input)) -} - fn sort_required_exec_with_req( input: Arc, sort_exprs: LexOrdering, @@ -2094,7 +2090,10 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { options: SortOptions::default(), }] .into(); - let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + let input = union_exec(vec![ + parquet_exec_with_sort(schema, vec![sort_key.clone()]); + 2 + ]); let plan = sort_preserving_merge_exec(sort_key, input); // Test: run EnforceDistribution, then EnforceSort. @@ -2134,7 +2133,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { }] .into(); let plan = sort_required_exec_with_req( - filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), + filter_exec(parquet_exec_with_sort(schema, vec![sort_key.clone()])), sort_key, ); @@ -2173,7 +2172,7 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { }] .into(); let input1 = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), + parquet_exec_with_sort(schema, vec![sort_key.clone()]), sort_key, ); let input2 = filter_exec(parquet_exec()); @@ -2836,7 +2835,7 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let plan_parquet = sort_preserving_merge_exec( sort_key.clone(), - parquet_exec_with_sort(vec![sort_key.clone()]), + parquet_exec_with_sort(schema, vec![sort_key.clone()]), ); let plan_csv = sort_preserving_merge_exec(sort_key.clone(), csv_exec_with_sort(vec![sort_key])); @@ -2877,7 +2876,10 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { .into(); // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let input_parquet = - union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + union_exec(vec![ + parquet_exec_with_sort(schema, vec![sort_key.clone()]); + 2 + ]); let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); @@ -2952,7 +2954,7 @@ fn parallelization_does_not_benefit() -> Result<()> { // SortRequired // Parquet(sorted) let plan_parquet = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), + parquet_exec_with_sort(schema, vec![sort_key.clone()]), sort_key.clone(), ); let plan_csv = @@ -3001,8 +3003,10 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> ("a".to_string(), "a2".to_string()), ("c".to_string(), "c2".to_string()), ]; - let proj_parquet = - projection_exec_with_alias(parquet_exec_with_sort(vec![sort_key]), alias_pairs); + let proj_parquet = projection_exec_with_alias( + parquet_exec_with_sort(schema, vec![sort_key]), + alias_pairs, + ); let sort_key_after_projection = [PhysicalSortExpr { expr: col("c2", &proj_parquet.schema())?, options: SortOptions::default(), @@ -3384,7 +3388,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { }] .into(); let alias = vec![("a".to_string(), "a".to_string())]; - let input = parquet_exec_with_sort(vec![sort_key]); + let input = parquet_exec_with_sort(schema, vec![sort_key]); let physical_plan = aggregate_exec_with_alias(input, alias); // TestConfig: diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 3369a8c39c7bf..6d40b1b952399 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -21,9 +21,9 @@ use crate::physical_optimizer::test_utils::{ aggregate_exec, bounded_window_exec, bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, - hash_join_exec, local_limit_exec, memory_exec, parquet_exec, projection_exec, - repartition_exec, sort_exec, sort_exec_with_fetch, sort_expr, sort_expr_options, - sort_merge_join_exec, sort_preserving_merge_exec, + hash_join_exec, local_limit_exec, memory_exec, parquet_exec, parquet_exec_with_sort, + projection_exec, repartition_exec, sort_exec, sort_exec_with_fetch, sort_expr, + sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, sort_preserving_merge_exec_with_fetch, spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, }; @@ -54,7 +54,7 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec, WindowAggExec}; use datafusion_physical_plan::{displayable, get_plan_string, ExecutionPlan, InputOrderMode}; -use datafusion::datasource::physical_plan::{CsvSource, ParquetSource}; +use datafusion::datasource::physical_plan::CsvSource; use datafusion::datasource::listing::PartitionedFile; use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting, PlanWithCorrespondingCoalescePartitions, PlanWithCorrespondingSort, parallelize_sorts, ensure_sorting}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; @@ -65,26 +65,6 @@ use datafusion_physical_optimizer::PhysicalOptimizerRule; use rstest::rstest; -/// Created a sorted parquet exec -fn parquet_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect::>(); - let source = Arc::new(ParquetSource::default()); - let mut builder = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - source, - ) - .with_file(PartitionedFile::new("x".to_string(), 100)); - if let Some(ordering) = LexOrdering::new(sort_exprs) { - builder = builder.with_output_ordering(vec![ordering]); - } - let config = builder.build(); - DataSourceExec::from_data_source(config) -} - /// Create a sorted Csv exec fn csv_exec_sorted( schema: &SchemaRef, @@ -197,8 +177,9 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let left_schema = create_test_schema2()?; let right_schema = create_test_schema3()?; let left_input = memory_exec(&left_schema); - let parquet_sort_exprs = [sort_expr("a", &right_schema)]; - let right_input = parquet_exec_sorted(&right_schema, parquet_sort_exprs); + let parquet_ordering = [sort_expr("a", &right_schema)].into(); + let right_input = + parquet_exec_with_sort(right_schema.clone(), vec![parquet_ordering]); let on = vec![( Arc::new(Column::new_with_schema("col_a", &left_schema)?) as _, Arc::new(Column::new_with_schema("c", &right_schema)?) as _, @@ -225,7 +206,7 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { #[tokio::test] async fn test_do_not_remove_sort_with_limit() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -233,8 +214,8 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { .into(); let sort = sort_exec(ordering.clone(), source1); let limit = local_limit_exec(sort, 100); - let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + let parquet_ordering = [sort_expr("nullable_col", &schema)].into(); + let source2 = parquet_exec_with_sort(schema, vec![parquet_ordering]); let union = union_exec(vec![source2, limit]); let repartition = repartition_exec(union); let physical_plan = sort_preserving_merge_exec(ordering, repartition); @@ -267,10 +248,10 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { #[tokio::test] async fn test_union_inputs_sorted() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); let sort = sort_exec(ordering.clone(), source1); - let source2 = parquet_exec_sorted(&schema, ordering.clone()); + let source2 = parquet_exec_with_sort(schema, vec![ordering.clone()]); let union = union_exec(vec![source2, sort]); let physical_plan = sort_preserving_merge_exec(ordering, union); @@ -291,14 +272,15 @@ async fn test_union_inputs_sorted() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); let sort = sort_exec(ordering.clone(), source1); - let parquet_sort_exprs = [ + let parquet_ordering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + ] + .into(); + let source2 = parquet_exec_with_sort(schema, vec![parquet_ordering]); let union = union_exec(vec![source2, sort]); let physical_plan = sort_preserving_merge_exec(ordering, union); @@ -319,15 +301,15 @@ async fn test_union_inputs_different_sorted() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted2() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let sort_exprs: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ] .into(); let sort = sort_exec(sort_exprs.clone(), source1); - let parquet_sort_exprs = [sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + let parquet_ordering = [sort_expr("nullable_col", &schema)].into(); + let source2 = parquet_exec_with_sort(schema, vec![parquet_ordering]); let union = union_exec(vec![source2, sort]); let physical_plan = sort_preserving_merge_exec(sort_exprs, union); @@ -357,7 +339,7 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted3() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -367,7 +349,7 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { let ordering2 = [sort_expr("nullable_col", &schema)].into(); let sort2 = sort_exec(ordering2, source1); let parquet_ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); - let source2 = parquet_exec_sorted(&schema, parquet_ordering.clone()); + let source2 = parquet_exec_with_sort(schema, vec![parquet_ordering.clone()]); let union = union_exec(vec![sort1, source2, sort2]); let physical_plan = sort_preserving_merge_exec(parquet_ordering, union); @@ -401,7 +383,7 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted4() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -410,7 +392,7 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { let ordering2: LexOrdering = [sort_expr("nullable_col", &schema)].into(); let sort1 = sort_exec(ordering2.clone(), source1.clone()); let sort2 = sort_exec(ordering2.clone(), source1); - let source2 = parquet_exec_sorted(&schema, ordering2); + let source2 = parquet_exec_with_sort(schema, vec![ordering2]); let union = union_exec(vec![sort1, source2, sort2]); let physical_plan = sort_preserving_merge_exec(ordering1, union); @@ -445,7 +427,7 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted5() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -496,7 +478,7 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted6() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [sort_expr("nullable_col", &schema)].into(); let sort1 = sort_exec(ordering1, source1.clone()); let ordering2 = [ @@ -507,7 +489,7 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { let repartition = repartition_exec(source1); let spm = sort_preserving_merge_exec(ordering2, repartition); let parquet_ordering: LexOrdering = [sort_expr("nullable_col", &schema)].into(); - let source2 = parquet_exec_sorted(&schema, parquet_ordering.clone()); + let source2 = parquet_exec_with_sort(schema, vec![parquet_ordering.clone()]); let union = union_exec(vec![sort1, source2, spm]); let physical_plan = sort_preserving_merge_exec(parquet_ordering, union); @@ -547,8 +529,7 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted7() -> Result<()> { let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1: LexOrdering = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -556,7 +537,6 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { .into(); let sort1 = sort_exec(ordering1.clone(), source1.clone()); let sort2 = sort_exec(ordering1, source1); - let union = union_exec(vec![sort1, sort2]); let ordering2 = [sort_expr("nullable_col", &schema)].into(); let physical_plan = sort_preserving_merge_exec(ordering2, union); @@ -587,8 +567,7 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted8() -> Result<()> { let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -641,7 +620,7 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { #[tokio::test] async fn test_soft_hard_requirements_remove_soft_requirement() -> Result<()> { let schema = create_test_schema()?; - let source = parquet_exec_sorted(&schema, vec![]); + let source = parquet_exec(schema.clone()); let sort_exprs = [sort_expr_options( "nullable_col", &schema, @@ -679,7 +658,7 @@ async fn test_soft_hard_requirements_remove_soft_requirement() -> Result<()> { async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( ) -> Result<()> { let schema = create_test_schema()?; - let source = parquet_exec_sorted(&schema, vec![]); + let source = parquet_exec(schema.clone()); let ordering = [sort_expr_options( "nullable_col", &schema, @@ -776,7 +755,7 @@ async fn test_soft_hard_requirements_remove_soft_requirement_without_pushdowns( #[tokio::test] async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> { let schema = create_test_schema()?; - let source = parquet_exec_sorted(&schema, vec![]); + let source = parquet_exec(schema.clone()); let ordering = [sort_expr_options( "nullable_col", &schema, @@ -905,7 +884,7 @@ async fn test_soft_hard_requirements_multiple_soft_requirements() -> Result<()> #[tokio::test] async fn test_soft_hard_requirements_multiple_sorts() -> Result<()> { let schema = create_test_schema()?; - let source = parquet_exec_sorted(&schema, vec![]); + let source = parquet_exec(schema.clone()); let ordering = [sort_expr_options( "nullable_col", &schema, @@ -974,7 +953,7 @@ async fn test_soft_hard_requirements_multiple_sorts() -> Result<()> { async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_requirement( ) -> Result<()> { let schema = create_test_schema()?; - let source = parquet_exec_sorted(&schema, vec![]); + let source = parquet_exec(schema.clone()); let ordering = [sort_expr_options( "nullable_col", &schema, @@ -1039,13 +1018,14 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ #[tokio::test] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; - let sort_exprs1 = [ + let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = [sort_expr("nullable_col", &schema)]; + ] + .into(); + let ordering2 = [sort_expr("nullable_col", &schema)].into(); // Reverse of the above - let ordering: LexOrdering = [sort_expr_options( + let ordering3: LexOrdering = [sort_expr_options( "nullable_col", &schema, SortOptions { @@ -1054,13 +1034,13 @@ async fn test_window_multi_path_sort() -> Result<()> { }, )] .into(); - let source1 = parquet_exec_sorted(&schema, sort_exprs1); - let source2 = parquet_exec_sorted(&schema, sort_exprs2); - let sort1 = sort_exec(ordering.clone(), source1); - let sort2 = sort_exec(ordering.clone(), source2); + let source1 = parquet_exec_with_sort(schema.clone(), vec![ordering1]); + let source2 = parquet_exec_with_sort(schema, vec![ordering2]); + let sort1 = sort_exec(ordering3.clone(), source1); + let sort2 = sort_exec(ordering3.clone(), source2); let union = union_exec(vec![sort1, sort2]); - let spm = sort_preserving_merge_exec(ordering.clone(), union); - let physical_plan = bounded_window_exec("nullable_col", ordering, spm); + let spm = sort_preserving_merge_exec(ordering3.clone(), union); + let physical_plan = bounded_window_exec("nullable_col", ordering3, spm); // The `WindowAggExec` gets its sorting from multiple children jointly. // During the removal of `SortExec`s, it should be able to remove the @@ -1096,8 +1076,8 @@ async fn test_window_multi_path_sort2() -> Result<()> { ] .into(); let ordering2: LexOrdering = [sort_expr("nullable_col", &schema)].into(); - let source1 = parquet_exec_sorted(&schema, ordering2.clone()); - let source2 = parquet_exec_sorted(&schema, ordering2.clone()); + let source1 = parquet_exec_with_sort(schema.clone(), vec![ordering2.clone()]); + let source2 = parquet_exec_with_sort(schema, vec![ordering2.clone()]); let sort1 = sort_exec(ordering1.clone(), source1); let sort2 = sort_exec(ordering1.clone(), source2); let union = union_exec(vec![sort1, sort2]); @@ -1130,7 +1110,7 @@ async fn test_window_multi_path_sort2() -> Result<()> { #[tokio::test] async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let ordering1 = [ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), @@ -1187,8 +1167,8 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { let left_schema = create_test_schema()?; let right_schema = create_test_schema2()?; - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); + let left = parquet_exec(left_schema); + let right = parquet_exec(right_schema); // Join on (nullable_col == col_a) let join_on = vec![( @@ -1260,8 +1240,8 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { let left_schema = create_test_schema()?; let right_schema = create_test_schema2()?; - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); + let left = parquet_exec(left_schema); + let right = parquet_exec(right_schema); // Join on (nullable_col == col_a) let join_on = vec![( @@ -1333,8 +1313,8 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let left_schema = create_test_schema()?; let right_schema = create_test_schema2()?; - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); + let left = parquet_exec(left_schema); + let right = parquet_exec(right_schema); // Join on (nullable_col == col_a) let join_on = vec![( @@ -1401,7 +1381,7 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { #[tokio::test] async fn test_multilayer_coalesce_partitions() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); + let source1 = parquet_exec(schema.clone()); let repartition = repartition_exec(source1); let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before @@ -1649,8 +1629,8 @@ async fn test_window_multi_layer_requirement() -> Result<()> { #[tokio::test] async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { let schema = create_test_schema3()?; - let input_sort_exprs = [sort_expr("b", &schema), sort_expr("c", &schema)]; - let parquet_input = parquet_exec_sorted(&schema, input_sort_exprs); + let parquet_ordering = [sort_expr("b", &schema), sort_expr("c", &schema)].into(); + let parquet_input = parquet_exec_with_sort(schema.clone(), vec![parquet_ordering]); let physical_plan = sort_exec( [ sort_expr("a", &schema), @@ -2486,15 +2466,16 @@ async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> #[tokio::test] async fn test_window_partial_constant_and_set_monotonicity() -> Result<()> { let input_schema = create_test_schema()?; - let sort_exprs = vec![sort_expr_options( + let ordering = [sort_expr_options( "nullable_col", &input_schema, SortOptions { descending: false, nulls_first: false, }, - )]; - let source = parquet_exec_sorted(&input_schema, sort_exprs); + )] + .into(); + let source = parquet_exec_with_sort(input_schema.clone(), vec![ordering]) as _; // Function definition - Alias of the resulting column - Arguments of the function #[derive(Clone)] diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index c04a75e00b232..409d392f7819e 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -235,13 +235,13 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { #[test] fn test_has_order_by() -> Result<()> { + let schema = schema(); let sort_key = [PhysicalSortExpr { - expr: col("a", &schema())?, + expr: col("a", &schema)?, options: SortOptions::default(), }] .into(); - let source = parquet_exec_with_sort(vec![sort_key]); - let schema = source.schema(); + let source = parquet_exec_with_sort(schema.clone(), vec![sort_key]); // `SELECT a FROM DataSourceExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec @@ -263,7 +263,7 @@ fn test_has_order_by() -> Result<()> { "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; - let plan: Arc = Arc::new(limit_exec); + let plan = Arc::new(limit_exec) as _; assert_plan_matches_expected(&plan, &expected)?; Ok(()) } diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 4d8f069a65502..1e64167241a8b 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -69,10 +69,10 @@ use datafusion_physical_plan::{ }; /// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { +pub fn parquet_exec(schema: SchemaRef) -> Arc { let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), + schema, Arc::new(ParquetSource::default()), ) .with_file(PartitionedFile::new("x".to_string(), 100)) @@ -83,11 +83,12 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { /// Create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( + schema: SchemaRef, output_ordering: Vec, ) -> Arc { let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), - schema(), + schema, Arc::new(ParquetSource::default()), ) .with_file(PartitionedFile::new("x".to_string(), 100)) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 034981482ac0f..ef0ff068649ca 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1441,7 +1441,7 @@ impl protobuf::PhysicalPlanNode { let Some(ordering) = LexOrdering::new(exprs) else { return internal_err!("SortExec requires an ordering"); }; - let fetch = (sort.fetch >= 0).then_some(sort.fetch as usize); + let fetch = (sort.fetch >= 0).then_some(sort.fetch as _); let new_sort = SortExec::new(ordering, input) .with_fetch(fetch) .with_preserve_partitioning(sort.preserve_partitioning); @@ -1496,7 +1496,7 @@ impl protobuf::PhysicalPlanNode { let Some(ordering) = LexOrdering::new(exprs) else { return internal_err!("SortExec requires an ordering"); }; - let fetch = (sort.fetch >= 0).then_some(sort.fetch as usize); + let fetch = (sort.fetch >= 0).then_some(sort.fetch as _); Ok(Arc::new( SortPreservingMergeExec::new(ordering, input).with_fetch(fetch), )) From a9b3b70ca0951b5794911918e189e7d6b821c16a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 11:38:47 +0300 Subject: [PATCH 091/167] Transition to fallible LexOrdering constructor - 8 --- datafusion/core/benches/physical_plan.rs | 6 +- datafusion/core/benches/sort.rs | 3 +- .../tests/fuzz_cases/equivalence/ordering.rs | 12 +- .../fuzz_cases/equivalence/projection.rs | 10 +- .../fuzz_cases/equivalence/properties.rs | 5 +- .../physical_optimizer/enforce_sorting.rs | 1 - .../physical_optimizer/projection_pushdown.rs | 145 +++++++++--------- datafusion/datasource/src/file_scan_config.rs | 8 +- .../functions-aggregate/src/array_agg.rs | 7 +- .../physical-expr-common/src/sort_expr.rs | 120 +++++++-------- .../physical-expr/src/equivalence/mod.rs | 14 +- .../physical-expr/src/equivalence/ordering.rs | 8 +- .../src/equivalence/properties/dependency.rs | 15 +- .../src/equivalence/properties/mod.rs | 5 +- .../src/enforce_sorting/sort_pushdown.rs | 40 +++-- .../src/output_requirements.rs | 46 +++--- datafusion/physical-optimizer/src/utils.rs | 10 +- .../physical-plan/src/aggregates/mod.rs | 8 +- .../src/joins/nested_loop_join.rs | 11 +- datafusion/physical-plan/src/windows/mod.rs | 29 ++-- datafusion/proto/src/physical_plan/mod.rs | 21 ++- .../tests/cases/roundtrip_physical_plan.rs | 21 +-- 22 files changed, 246 insertions(+), 299 deletions(-) diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index aa65c0b5e98b2..e4838572f60fb 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -50,11 +50,7 @@ fn sort_preserving_merge_operator( let sort = sort .iter() - .map(|name| PhysicalSortExpr { - expr: col(name, &schema).unwrap(), - options: Default::default(), - }) - .collect::>(); + .map(|name| PhysicalSortExpr::new_default(col(name, &schema).unwrap())); let sort = LexOrdering::new(sort).unwrap(); let exec = MemorySourceConfig::try_new_exec( diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index aba1269ed268d..80b988d64ae9b 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -274,8 +274,7 @@ fn make_sort_exprs(schema: &Schema) -> LexOrdering { let sort_exprs = schema .fields() .iter() - .map(|f| PhysicalSortExpr::new_default(col(f.name(), schema).unwrap())) - .collect::>(); + .map(|f| PhysicalSortExpr::new_default(col(f.name(), schema).unwrap())); LexOrdering::new(sort_exprs).unwrap() } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 72a8ad7210122..0e650bf921990 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -62,11 +62,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { for exprs in col_exprs.iter().combinations(n_req) { let sort_exprs = exprs .into_iter() - .map(|expr| PhysicalSortExpr { - expr: Arc::clone(expr), - options: SORT_OPTIONS, - }) - .collect::>(); + .map(|expr| PhysicalSortExpr::new(Arc::clone(expr), SORT_OPTIONS)); let Some(ordering) = LexOrdering::new(sort_exprs) else { unreachable!("Test should always produce non-degenerate orderings"); }; @@ -137,11 +133,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { for exprs in exprs.iter().combinations(n_req) { let sort_exprs = exprs .into_iter() - .map(|expr| PhysicalSortExpr { - expr: Arc::clone(expr), - options: SORT_OPTIONS, - }) - .collect::>(); + .map(|expr| PhysicalSortExpr::new(Arc::clone(expr), SORT_OPTIONS)); let Some(ordering) = LexOrdering::new(sort_exprs) else { unreachable!("Test should always produce non-degenerate orderings"); }; diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index 04823b986325b..775e904cd998e 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -163,13 +163,9 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { for n_req in 1..=projected_exprs.len() { for exprs in projected_exprs.iter().combinations(n_req) { - let sort_exprs = exprs - .into_iter() - .map(|expr| PhysicalSortExpr { - expr: Arc::clone(expr), - options: SORT_OPTIONS, - }) - .collect::>(); + let sort_exprs = exprs.into_iter().map(|expr| { + PhysicalSortExpr::new(Arc::clone(expr), SORT_OPTIONS) + }); let Some(ordering) = LexOrdering::new(sort_exprs) else { unreachable!( "Test should always produce non-degenerate orderings" diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index d3b0613de8d81..0602689611d19 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -75,9 +75,8 @@ fn test_find_longest_permutation_random() -> Result<()> { let ordering2 = indices .iter() .zip(ordering.iter()) - .map(|(&idx, sort_expr)| PhysicalSortExpr { - expr: Arc::clone(&exprs[idx]), - options: sort_expr.options, + .map(|(&idx, sort_expr)| { + PhysicalSortExpr::new(Arc::clone(&exprs[idx]), sort_expr.options) }) .collect::>(); assert_eq!( diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 6d40b1b952399..ad9b03d9ce6b9 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -70,7 +70,6 @@ fn csv_exec_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect::>(); let mut builder = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), schema.clone(), diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index a967b6df5713f..54d546d299e3c 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -39,7 +39,7 @@ use datafusion_physical_expr::expressions::{ use datafusion_physical_expr::{Distribution, Partitioning, ScalarFunctionExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ - LexRequirement, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, + OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; @@ -655,20 +655,23 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv, - Some(OrderingRequirements::new_single(LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ]))), + Some(OrderingRequirements::new_single( + [ + PhysicalSortRequirement::new( + Arc::new(Column::new("b", 1)), + Some(SortOptions::default()), + ), + PhysicalSortRequirement::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + Some(SortOptions::default()), + ), + ] + .into(), + )), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -701,20 +704,23 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = OrderingRequirements::new_single(LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ])); + let expected_reqs = OrderingRequirements::new_single( + [ + PhysicalSortRequirement::new( + Arc::new(Column::new("b", 2)), + Some(SortOptions::default()), + ), + PhysicalSortRequirement::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_a", 1)), + )), + Some(SortOptions::default()), + ), + ] + .into(), + ); assert_eq!( after_optimize .as_any() @@ -797,15 +803,15 @@ fn test_filter_after_projection() -> Result<()> { Arc::new(Column::new("a", 0)), )), )); - let filter: Arc = Arc::new(FilterExec::try_new(predicate, csv)?); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let filter = Arc::new(FilterExec::try_new(predicate, csv)?); + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("a", 0)), "a_new".to_string()), (Arc::new(Column::new("b", 1)), "b".to_string()), (Arc::new(Column::new("d", 3)), "d".to_string()), ], filter.clone(), - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ @@ -882,7 +888,7 @@ fn test_join_after_projection() -> Result<()> { None, StreamJoinPartitionMode::SinglePartition, )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), @@ -891,7 +897,7 @@ fn test_join_after_projection() -> Result<()> { (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), ], join, - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", @@ -947,7 +953,7 @@ fn test_join_after_required_projection() -> Result<()> { let left_csv = create_simple_csv_exec(); let right_csv = create_simple_csv_exec(); - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + let join = Arc::new(SymmetricHashJoinExec::try_new( left_csv, right_csv, vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], @@ -996,7 +1002,7 @@ fn test_join_after_required_projection() -> Result<()> { None, StreamJoinPartitionMode::SinglePartition, )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("a", 5)), "a".to_string()), (Arc::new(Column::new("b", 6)), "b".to_string()), @@ -1010,7 +1016,7 @@ fn test_join_after_required_projection() -> Result<()> { (Arc::new(Column::new("e", 4)), "e".to_string()), ], join, - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", @@ -1063,7 +1069,7 @@ fn test_nested_loop_join_after_projection() -> Result<()> { Field::new("c", DataType::Int32, true), ]); - let join: Arc = Arc::new(NestedLoopJoinExec::try_new( + let join = Arc::new(NestedLoopJoinExec::try_new( left_csv, right_csv, Some(JoinFilter::new( @@ -1073,12 +1079,12 @@ fn test_nested_loop_join_after_projection() -> Result<()> { )), &JoinType::Inner, None, - )?); + )?) as _; - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![(col_left_c, "c".to_string())], Arc::clone(&join), - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c]", @@ -1106,7 +1112,7 @@ fn test_hash_join_after_projection() -> Result<()> { let left_csv = create_simple_csv_exec(); let right_csv = create_simple_csv_exec(); - let join: Arc = Arc::new(HashJoinExec::try_new( + let join = Arc::new(HashJoinExec::try_new( left_csv, right_csv, vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], @@ -1154,7 +1160,7 @@ fn test_hash_join_after_projection() -> Result<()> { PartitionMode::Auto, true, )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), @@ -1162,7 +1168,7 @@ fn test_hash_join_after_projection() -> Result<()> { (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), ], join.clone(), - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" @@ -1176,7 +1182,7 @@ fn test_hash_join_after_projection() -> Result<()> { let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"]; assert_eq!(get_plan_string(&after_optimize), expected); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("a", 0)), "a".to_string()), (Arc::new(Column::new("b", 1)), "b".to_string()), @@ -1199,7 +1205,7 @@ fn test_hash_join_after_projection() -> Result<()> { #[test] fn test_repartition_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); - let repartition: Arc = Arc::new(RepartitionExec::try_new( + let repartition = Arc::new(RepartitionExec::try_new( csv, Partitioning::Hash( vec![ @@ -1210,14 +1216,14 @@ fn test_repartition_after_projection() -> Result<()> { 6, ), )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("b", 1)), "b_new".to_string()), (Arc::new(Column::new("a", 0)), "a".to_string()), (Arc::new(Column::new("d", 3)), "d_new".to_string()), ], repartition, - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", @@ -1261,18 +1267,12 @@ fn test_sort_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_exec = SortExec::new( [ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("b", 1))), + PhysicalSortExpr::new_default(Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + ))), ] .into(), csv, @@ -1312,18 +1312,12 @@ fn test_sort_preserving_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_exec = SortPreservingMergeExec::new( [ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, + PhysicalSortExpr::new_default(Arc::new(Column::new("b", 1))), + PhysicalSortExpr::new_default(Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + ))), ] .into(), csv, @@ -1361,16 +1355,15 @@ fn test_sort_preserving_after_projection() -> Result<()> { #[test] fn test_union_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); - let union: Arc = - Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); - let projection: Arc = Arc::new(ProjectionExec::try_new( + let union = Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); + let projection = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c".to_string()), (Arc::new(Column::new("a", 0)), "new_a".to_string()), (Arc::new(Column::new("b", 1)), "b".to_string()), ], union.clone(), - )?); + )?) as _; let initial = get_plan_string(&projection); let expected_initial = [ diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index d7aba3335ae44..530eb354142ab 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -1396,10 +1396,10 @@ fn get_projected_output_ordering( let name = col.name(); if let Some((idx, _)) = projected_schema.column_with_name(name) { // Compute the new sort expression (with correct index) after projection: - new_ordering.push(PhysicalSortExpr { - expr: Arc::new(Column::new(name, idx)), - options: *options, - }); + new_ordering.push(PhysicalSortExpr::new( + Arc::new(Column::new(name, idx)), + *options, + )); continue; } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index a2fd1a5351f5a..50ffb580d33c7 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -168,12 +168,11 @@ impl AggregateUDFImpl for ArrayAgg { )?)); } - let order_bys = acc_args.order_bys; - if order_bys.is_empty() { + let Some(ordering) = LexOrdering::new(acc_args.order_bys.to_vec()) else { return Ok(Box::new(ArrayAggAccumulator::try_new(&data_type)?)); }; - let ordering_dtypes = order_bys + let ordering_dtypes = ordering .iter() .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; @@ -181,7 +180,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - LexOrdering::new(order_bys.to_vec()).unwrap(), + ordering, acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index bcbbac7c2f451..134ee7ef61ef9 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -136,29 +136,9 @@ impl PhysicalSortExpr { to_str(&self.options) ) } -} - -impl PartialEq for PhysicalSortExpr { - fn eq(&self, other: &Self) -> bool { - self.options == other.options && self.expr.eq(&other.expr) - } -} -impl Hash for PhysicalSortExpr { - fn hash(&self, state: &mut H) { - self.expr.hash(state); - self.options.hash(state); - } -} - -impl Display for PhysicalSortExpr { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - write!(f, "{} {}", self.expr, to_str(&self.options)) - } -} - -impl PhysicalSortExpr { - /// evaluate the sort expression into SortColumn that can be passed into arrow sort kernel + /// Evaluates the sort expression into a `SortColumn` that can be passed + /// into the arrow sort kernel. pub fn evaluate_to_sort_column(&self, batch: &RecordBatch) -> Result { let array_to_sort = match self.expr.evaluate(batch)? { ColumnarValue::Array(array) => array, @@ -200,6 +180,25 @@ impl PhysicalSortExpr { } } +impl PartialEq for PhysicalSortExpr { + fn eq(&self, other: &Self) -> bool { + self.options == other.options && self.expr.eq(&other.expr) + } +} + +impl Hash for PhysicalSortExpr { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + self.options.hash(state); + } +} + +impl Display for PhysicalSortExpr { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "{} {}", self.expr, to_str(&self.options)) + } +} + /// Represents sort requirement associated with a plan /// /// If the requirement includes [`SortOptions`] then both the @@ -279,12 +278,6 @@ impl PhysicalSortRequirement { Self { expr, options } } - /// Replace the required expression for this requirement with the new one - pub fn with_expr(mut self, expr: Arc) -> Self { - self.expr = expr; - self - } - /// Returns whether this requirement is equal or more specific than `other`. pub fn compatible(&self, other: &Self) -> bool { self.expr.eq(&other.expr) @@ -372,14 +365,15 @@ impl LexOrdering { /// /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a /// Some(ASC)]`. - pub fn collapse(self) -> Self { + pub fn collapse(mut self) -> Self { let mut orderings = Vec::::new(); - for element in self { + for element in self.inner { if !orderings.iter().any(|item| item.expr.eq(&element.expr)) { orderings.push(element); } } - Self { inner: orderings } + self.inner = orderings; + self } /// Truncates the `LexOrdering`, keeping only the first `len` elements. @@ -474,11 +468,11 @@ pub struct LexRequirement { } impl LexRequirement { - /// Creates a new [`LexOrdering`] from the given vector of sort requirements. - /// The vector must not be empty. - pub fn new(inner: Vec) -> Self { - debug_assert!(!inner.is_empty()); - Self { inner } + /// Creates a new [`LexRequirement`] from the given vector of sort expressions. + /// If the vector is empty, returns `None`. + pub fn new(inner: impl IntoIterator) -> Option { + let inner = inner.into_iter().collect::>(); + (!inner.is_empty()).then(|| Self { inner }) } /// Appends an element to the back of the `LexRequirement`. @@ -508,20 +502,15 @@ impl LexRequirement { /// /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a /// Some(ASC)]`. - pub fn collapse(self) -> Self { + pub fn collapse(mut self) -> Self { let mut reqs = Vec::::new(); - for element in self { + for element in self.inner { if !reqs.iter().any(|item| item.expr.eq(&element.expr)) { reqs.push(element); } } - Self::new(reqs) - } -} - -impl From> for LexRequirement { - fn from(value: Vec) -> Self { - Self::new(value) + self.inner = reqs; + self } } @@ -550,12 +539,6 @@ impl DerefMut for LexRequirement { } } -impl FromIterator for LexRequirement { - fn from_iter>(iter: T) -> Self { - Self::new(iter.into_iter().collect()) - } -} - impl IntoIterator for LexRequirement { type Item = PhysicalSortRequirement; type IntoIter = IntoIter; @@ -584,12 +567,6 @@ impl From for LexRequirement { } } -impl From> for LexRequirement { - fn from(value: Vec) -> Self { - Self::new(value.into_iter().map(Into::into).collect()) - } -} - impl From for LexOrdering { fn from(value: LexRequirement) -> Self { // Can construct directly as `value` is non-degenerate: @@ -599,12 +576,6 @@ impl From for LexOrdering { } } -impl From> for LexOrdering { - fn from(value: Vec) -> Self { - Self::new(value.into_iter().map(Into::into)).unwrap() - } -} - /// Represents a plan's input ordering requirements. Vector elements represent /// alternative ordering requirements in the order of preference. #[derive(Debug, Clone, PartialEq)] @@ -634,6 +605,11 @@ impl OrderingRequirements { Self::Hard(vec![requirement]) } + /// Creates a new instance with a single soft requirement. + pub fn new_single_soft(requirement: LexRequirement) -> Self { + Self::Soft(vec![requirement]) + } + /// Adds an alternative requirement to the list of alternatives. pub fn add_alternative(&mut self, requirement: LexRequirement) { match self { @@ -669,3 +645,21 @@ impl From for OrderingRequirements { Self::new_single(ordering.into()) } } + +impl Deref for OrderingRequirements { + type Target = [LexRequirement]; + + fn deref(&self) -> &Self::Target { + match &self { + Self::Hard(alts) | Self::Soft(alts) => alts.as_slice(), + } + } +} + +impl DerefMut for OrderingRequirements { + fn deref_mut(&mut self) -> &mut Self::Target { + match self { + Self::Hard(alts) | Self::Soft(alts) => alts.as_mut_slice(), + } + } +} diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 2a65cbddb43ab..75f0d77db7c5b 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -61,10 +61,7 @@ pub fn convert_to_sort_exprs>>( args: &[(T, SortOptions)], ) -> Vec { args.iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr.borrow()), - options: *options, - }) + .map(|(expr, options)| PhysicalSortExpr::new(Arc::clone(expr.borrow()), *options)) .collect() } @@ -213,11 +210,10 @@ mod tests { pub fn convert_to_sort_reqs( args: &[(&Arc, Option)], ) -> LexRequirement { - args.iter() - .map(|(expr, options)| { - PhysicalSortRequirement::new(Arc::clone(*expr), *options) - }) - .collect() + let exprs = args.iter().map(|(expr, options)| { + PhysicalSortRequirement::new(Arc::clone(*expr), *options) + }); + LexRequirement::new(exprs).unwrap() } #[test] diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 30a593878a458..bf1df465a64bc 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -103,12 +103,10 @@ impl OrderingEquivalenceClass { /// Adds new orderings into this ordering equivalence class pub fn add_new_orderings( &mut self, - orderings: impl IntoIterator>, + sort_exprs: impl IntoIterator>, ) { - self.orderings.extend(orderings.into_iter().filter_map(|o| { - let sort_exprs = o.into_iter().collect::>(); - LexOrdering::new(sort_exprs) - })); + self.orderings + .extend(sort_exprs.into_iter().filter_map(LexOrdering::new)); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); } diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 585053abe69ee..6375616dbe34c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1284,11 +1284,12 @@ mod tests { let col_c = col("c", &schema)?; let eq_properties = EquivalenceProperties::new(schema); - let lex_a = LexRequirement::new(vec![PhysicalSortRequirement { + let lex_a: LexRequirement = [PhysicalSortRequirement { expr: Arc::clone(&col_a), options: None, - }]); - let lex_a_b = LexRequirement::new(vec![ + }] + .into(); + let lex_a_b: LexRequirement = [ PhysicalSortRequirement { expr: col_a, options: None, @@ -1297,11 +1298,13 @@ mod tests { expr: col_b, options: None, }, - ]); - let lex_c = LexRequirement::new(vec![PhysicalSortRequirement { + ] + .into(); + let lex_c = [PhysicalSortRequirement { expr: col_c, options: None, - }]); + }] + .into(); assert!(eq_properties.requirements_compatible(lex_a.clone(), lex_a.clone())); assert!(!eq_properties.requirements_compatible(lex_a.clone(), lex_a_b.clone())); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 54aae774ad392..2e9e8fa5ea808 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -505,9 +505,8 @@ impl EquivalenceProperties { let reqs = sort_reqs .into_iter() .map(|req| self.eq_group.normalize_sort_requirement(req.clone())) - .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)) - .collect::>(); - (!reqs.is_empty()).then(|| LexRequirement::new(reqs).collapse()) + .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)); + LexRequirement::new(reqs).map(|r| r.collapse()) } /// Checks whether the given ordering is satisfied by any of the existing diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 07e872a4ce34e..e8d8db7240782 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -292,7 +292,7 @@ fn pushdown_requirement_to_children( let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = - shift_right_required(parent_required.first(), right_offset)?; + shift_right_required(parent_required.first().clone(), right_offset)?; let new_right_required_expr = LexOrdering::from(new_right_required); try_pushdown_requirements_to_join( smj, @@ -524,26 +524,19 @@ fn expr_source_side( } fn shift_right_required( - parent_required: &LexRequirement, + mut parent_required: LexRequirement, left_columns_len: usize, ) -> Result { - let new_right_required = parent_required - .iter() - .filter_map(|r| { - let col = r.expr.as_any().downcast_ref::()?; - col.index().checked_sub(left_columns_len).map(|offset| { - r.clone() - .with_expr(Arc::new(Column::new(col.name(), offset))) - }) - }) - .collect::>(); - if new_right_required.len() == parent_required.len() { - Ok(new_right_required.into()) - } else { - plan_err!( - "Expect to shift all the parent required column indexes for SortMergeJoin" - ) + for req in parent_required.iter_mut() { + let Some(col) = req.expr.as_any().downcast_ref::() else { + return plan_err!( + "Expect to shift all the parent required column indexes for SortMergeJoin" + ); + }; + let offset = col.index() - left_columns_len; + req.expr = Arc::new(Column::new(col.name(), offset)); } + Ok(parent_required) } /// Handles the custom pushdown of parent-required sorting requirements down to @@ -642,9 +635,12 @@ fn handle_custom_pushdown( let result = maintains_input_order .iter() .map(|&maintains_order| { - maintains_order.then(|| { - OrderingRequirements::new_single(updated_parent_req.clone().into()) - }) + if maintains_order { + LexRequirement::new(updated_parent_req.clone()) + .map(OrderingRequirements::new_single) + } else { + None + } }) .collect(); @@ -719,7 +715,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - Some(OrderingRequirements::new_single(updated_parent_req.into())), + LexRequirement::new(updated_parent_req).map(OrderingRequirements::new_single), ])) } else { Ok(None) diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 4f6a58fb25cbf..d2a6179171ead 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -31,18 +31,16 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; use datafusion_physical_expr::Distribution; -use datafusion_physical_expr_common::sort_expr::{ - OrderingRequirements, PhysicalSortRequirement, -}; +use datafusion_physical_expr_common::sort_expr::OrderingRequirements; use datafusion_physical_plan::projection::{ make_with_child, update_expr, ProjectionExec, }; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, }; -use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; /// This rule either adds or removes [`OutputRequirements`]s to/from the physical /// plan according to its `mode` attribute, which is set by the constructors @@ -211,22 +209,21 @@ impl ExecutionPlan for OutputRequirementExec { projection: &ProjectionExec, ) -> Result>> { // If the projection does not narrow the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { + let proj_exprs = projection.expr(); + if proj_exprs.len() >= projection.input().schema().fields().len() { return Ok(None); } - let mut updated_sort_reqs = vec![]; - // None or empty_vec can be treated in the same way. - if let Some(reqs) = self.required_input_ordering().swap_remove(0) { - for req in reqs.into_single().iter() { - let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? - else { - return Ok(None); - }; - updated_sort_reqs.push(PhysicalSortRequirement { - expr: new_expr, - options: req.options, - }); + let mut requirements = self.required_input_ordering().swap_remove(0); + if let Some(reqs) = requirements.as_mut() { + for lex in reqs.iter_mut() { + for item in lex.iter_mut() { + let Some(new_expr) = update_expr(&item.expr, proj_exprs, false)? + else { + return Ok(None); + }; + item.expr = new_expr; + } } } @@ -245,15 +242,10 @@ impl ExecutionPlan for OutputRequirementExec { dist => dist.clone(), }; - make_with_child(projection, &self.input()) - .map(|input| { - OutputRequirementExec::new( - input, - Some(OrderingRequirements::new_single(updated_sort_reqs.into())), - dist_req, - ) - }) - .map(|e| Some(Arc::new(e) as _)) + make_with_child(projection, &self.input()).map(|input| { + let e = OutputRequirementExec::new(input, requirements, dist_req); + Some(Arc::new(e) as _) + }) } } diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 6c480e4c67165..e90872a1a0a92 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -46,11 +46,11 @@ pub fn add_sort_above( .equivalence_properties() .is_expr_constant(&sort_expr.expr) }); - if sort_reqs.is_empty() { + let sort_exprs = sort_reqs.into_iter().map(Into::into).collect::>(); + let Some(ordering) = LexOrdering::new(sort_exprs) else { return node; - } - let mut new_sort = - SortExec::new(sort_reqs.into(), Arc::clone(&node.plan)).with_fetch(fetch); + }; + let mut new_sort = SortExec::new(ordering, Arc::clone(&node.plan)).with_fetch(fetch); if node.plan.output_partitioning().partition_count() > 1 { new_sort = new_sort.with_preserve_partitioning(true); } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 819d21a871d85..540c3680fabc7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -497,12 +497,8 @@ impl AggregateExec { )?; new_requirements.extend(req); - let required_input_ordering = if new_requirements.is_empty() { - None - } else { - let reqs = LexRequirement::from(new_requirements).collapse(); - OrderingRequirements::new(vec![reqs], true) - }; + let required_input_ordering = LexRequirement::new(new_requirements) + .map(|r| OrderingRequirements::new_single_soft(r.collapse())); // If our aggregation has grouping sets then our base grouping exprs will // be expanded based on the flags in `group_by.groups` where for each diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index fd907eef6748f..e4d7a27a68a54 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1084,13 +1084,10 @@ pub(crate) mod tests { let mut sort_info = vec![]; for name in sorted_column_names { let index = schema.index_of(name).unwrap(); - let sort_expr = PhysicalSortExpr { - expr: Arc::new(Column::new(name, index)), - options: SortOptions { - descending: false, - nulls_first: false, - }, - }; + let sort_expr = PhysicalSortExpr::new( + Arc::new(Column::new(name, index)), + SortOptions::new(false, false), + ); sort_info.push(sort_expr); } let mut source = TestMemoryExec::try_new(&[batches], schema, None).unwrap(); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 2b1dea1715bc5..783729a905944 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -47,7 +47,7 @@ use datafusion_physical_expr::window::{ }; use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, OrderingRequirements, PhysicalSortRequirement, + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortRequirement, }; use itertools::Itertools; @@ -303,12 +303,8 @@ pub(crate) fn calc_requirements< } let mut alternatives = vec![]; - if !sort_reqs_with_partition.is_empty() { - alternatives.push(sort_reqs_with_partition.into()); - } - if !sort_reqs.is_empty() { - alternatives.push(sort_reqs.into()); - } + alternatives.extend(LexRequirement::new(sort_reqs_with_partition)); + alternatives.extend(LexRequirement::new(sort_reqs)); OrderingRequirements::new(alternatives, false) } @@ -760,30 +756,25 @@ mod tests { let mut orderbys = vec![]; for (col_name, descending, nulls_first) in ob_params { let expr = col(col_name, &schema)?; - let options = SortOptions { - descending, - nulls_first, - }; - orderbys.push(PhysicalSortExpr { expr, options }); + let options = SortOptions::new(descending, nulls_first); + orderbys.push(PhysicalSortExpr::new(expr, options)); } let mut expected: Option = None; for expected_param in expected_params.clone() { let mut requirements = vec![]; for (col_name, reqs) in expected_param { - let options = reqs.map(|(descending, nulls_first)| SortOptions { - descending, - nulls_first, + let options = reqs.map(|(descending, nulls_first)| { + SortOptions::new(descending, nulls_first) }); let expr = col(col_name, &schema)?; requirements.push(PhysicalSortRequirement::new(expr, options)); } - if !requirements.is_empty() { + if let Some(requirements) = LexRequirement::new(requirements) { if let Some(alts) = expected.as_mut() { - alts.add_alternative(requirements.into()); + alts.add_alternative(requirements); } else { - expected = - Some(OrderingRequirements::new_single(requirements.into())); + expected = Some(OrderingRequirements::new_single(requirements)); } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index ef0ff068649ca..6596bfc2030c7 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1637,9 +1637,12 @@ impl protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) + .map(|sort_exprs| { + LexRequirement::new(sort_exprs.into_iter().map(Into::into)) + }) }) - .transpose()?; + .transpose()? + .flatten(); Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), @@ -1672,9 +1675,12 @@ impl protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) + .map(|sort_exprs| { + LexRequirement::new(sort_exprs.into_iter().map(Into::into)) + }) }) - .transpose()?; + .transpose()? + .flatten(); Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), @@ -1710,9 +1716,12 @@ impl protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(LexRequirement::from) + .map(|sort_exprs| { + LexRequirement::new(sort_exprs.into_iter().map(Into::into)) + }) }) - .transpose()?; + .transpose()? + .flatten(); Ok(Arc::new(DataSinkExec::new( input, Arc::new(data_sink), diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 2c13b945bd1f2..8e835ae73af72 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -60,7 +60,7 @@ use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::{SlidingAggregateWindowExpr, StandardWindowExpr}; use datafusion::physical_expr::{ - LexOrdering, LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, + LexOrdering, PhysicalSortRequirement, ScalarFunctionExpr, }; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -1324,13 +1324,14 @@ fn roundtrip_json_sink() -> Result<()> { file_sink_config, JsonWriterOptions::new(CompressionTypeVariant::UNCOMPRESSED), )); - let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( + let sort_order = [PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]); + )] + .into(); roundtrip_test(Arc::new(DataSinkExec::new( input, @@ -1361,13 +1362,14 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( + let sort_order = [PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]); + )] + .into(); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1417,13 +1419,14 @@ fn roundtrip_parquet_sink() -> Result<()> { file_sink_config, TableParquetOptions::default(), )); - let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( + let sort_order = [PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]); + )] + .into(); roundtrip_test(Arc::new(DataSinkExec::new( input, @@ -1465,7 +1468,7 @@ fn roundtrip_sym_hash_join() -> Result<()> { options: Default::default(), }]), ] { - for right_order in &[ + for right_order in [ None, LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_right.index_of("col")?)), @@ -1480,7 +1483,7 @@ fn roundtrip_sym_hash_join() -> Result<()> { join_type, false, left_order.clone(), - right_order.clone(), + right_order, *partition_mode, )?))?; } From e291ecdfc916e66d906830922ddc2e11b409c224 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 11:43:26 +0300 Subject: [PATCH 092/167] Transition to fallible LexOrdering constructor - 9 --- .../physical_optimizer/enforce_sorting.rs | 11 ++++----- .../src/equivalence/properties/dependency.rs | 23 ++++--------------- .../physical-plan/src/aggregates/mod.rs | 15 ++++-------- 3 files changed, 13 insertions(+), 36 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index ad9b03d9ce6b9..3c2da331cf439 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -973,13 +973,10 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ partition_bys2, bounded_window, ); - let requirement = [PhysicalSortRequirement { - expr: col("non_nullable_col", &schema)?, - options: Some(SortOptions { - descending: false, - nulls_first: true, - }), - }] + let requirement = [PhysicalSortRequirement::new( + col("non_nullable_col", &schema)?, + Some(SortOptions(false, true)), + )] .into(); let physical_plan = Arc::new(OutputRequirementExec::new( bounded_window2, diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 6375616dbe34c..311a446179ccf 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1284,27 +1284,14 @@ mod tests { let col_c = col("c", &schema)?; let eq_properties = EquivalenceProperties::new(schema); - let lex_a: LexRequirement = [PhysicalSortRequirement { - expr: Arc::clone(&col_a), - options: None, - }] - .into(); + let lex_a: LexRequirement = + [PhysicalSortRequirement::new(Arc::clone(&col_a), None)].into(); let lex_a_b: LexRequirement = [ - PhysicalSortRequirement { - expr: col_a, - options: None, - }, - PhysicalSortRequirement { - expr: col_b, - options: None, - }, + PhysicalSortRequirement::new(col_a, None), + PhysicalSortRequirement::new(col_b, None), ] .into(); - let lex_c = [PhysicalSortRequirement { - expr: col_c, - options: None, - }] - .into(); + let lex_c = [PhysicalSortRequirement::new(col_c, None)].into(); assert!(eq_properties.requirements_compatible(lex_a.clone(), lex_a.clone())); assert!(!eq_properties.requirements_compatible(lex_a.clone(), lex_a_b.clone())); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 540c3680fabc7..a03d4a4577cc7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -483,9 +483,8 @@ impl AggregateExec { let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); let mut new_requirements = indices .iter() - .map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&groupby_exprs[idx]), - options: None, + .map(|&idx| { + PhysicalSortRequirement::new(Arc::clone(&groupby_exprs[idx]), None) }) .collect::>(); @@ -2388,14 +2387,8 @@ mod tests { ]; let common_requirement = vec![ - PhysicalSortRequirement { - expr: Arc::clone(col_a), - options: Some(options), - }, - PhysicalSortRequirement { - expr: Arc::clone(col_c), - options: Some(options), - }, + PhysicalSortRequirement::new(Arc::clone(col_a), Some(options)), + PhysicalSortRequirement::new(Arc::clone(col_c), Some(options)), ]; let mut aggr_exprs = order_by_exprs .into_iter() From 39e6a9e551fc5345cf3947ba94c74c9914f4ca8e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 11:46:19 +0300 Subject: [PATCH 093/167] Transition to fallible LexOrdering constructor - 10 --- datafusion/core/tests/physical_optimizer/enforce_sorting.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 3c2da331cf439..b6fcdf08c869a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -975,7 +975,7 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ ); let requirement = [PhysicalSortRequirement::new( col("non_nullable_col", &schema)?, - Some(SortOptions(false, true)), + Some(SortOptions::new(false, true)), )] .into(); let physical_plan = Arc::new(OutputRequirementExec::new( From 08100079ca6888439a0ee8385ed188df62aa9510 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 11:49:23 +0300 Subject: [PATCH 094/167] Transition to fallible LexOrdering constructor - 11 --- .../tests/physical_optimizer/enforce_sorting.rs | 2 +- .../tests/physical_optimizer/projection_pushdown.rs | 4 ++-- datafusion/physical-expr-common/src/sort_expr.rs | 13 ++++++++----- .../src/enforce_sorting/sort_pushdown.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 4 ++-- 6 files changed, 16 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index b6fcdf08c869a..344ae780cea40 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -980,7 +980,7 @@ async fn test_soft_hard_requirements_with_multiple_soft_requirements_and_output_ .into(); let physical_plan = Arc::new(OutputRequirementExec::new( bounded_window2, - Some(OrderingRequirements::new_single(requirement)), + Some(OrderingRequirements::new(requirement)), Distribution::SinglePartition, )); diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 54d546d299e3c..bf621165ff418 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -655,7 +655,7 @@ fn test_output_req_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv, - Some(OrderingRequirements::new_single( + Some(OrderingRequirements::new( [ PhysicalSortRequirement::new( Arc::new(Column::new("b", 1)), @@ -704,7 +704,7 @@ fn test_output_req_after_projection() -> Result<()> { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = OrderingRequirements::new_single( + let expected_reqs = OrderingRequirements::new( [ PhysicalSortRequirement::new( Arc::new(Column::new("b", 2)), diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 134ee7ef61ef9..e5617502bbcfe 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -590,7 +590,10 @@ pub enum OrderingRequirements { impl OrderingRequirements { /// Creates a new instance from the given alternatives. If an empty list of /// alternatives are given, returns `None`. - pub fn new(alternatives: Vec, soft: bool) -> Option { + pub fn new_alternatives( + alternatives: Vec, + soft: bool, + ) -> Option { (!alternatives.is_empty()).then(|| { if soft { Self::Soft(alternatives) @@ -601,12 +604,12 @@ impl OrderingRequirements { } /// Creates a new instance with a single hard requirement. - pub fn new_single(requirement: LexRequirement) -> Self { + pub fn new(requirement: LexRequirement) -> Self { Self::Hard(vec![requirement]) } /// Creates a new instance with a single soft requirement. - pub fn new_single_soft(requirement: LexRequirement) -> Self { + pub fn new_soft(requirement: LexRequirement) -> Self { Self::Soft(vec![requirement]) } @@ -636,13 +639,13 @@ impl OrderingRequirements { impl From for OrderingRequirements { fn from(requirement: LexRequirement) -> Self { - Self::new_single(requirement) + Self::new(requirement) } } impl From for OrderingRequirements { fn from(ordering: LexOrdering) -> Self { - Self::new_single(ordering.into()) + Self::new(ordering.into()) } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index e8d8db7240782..af7836e0de4bf 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -637,7 +637,7 @@ fn handle_custom_pushdown( .map(|&maintains_order| { if maintains_order { LexRequirement::new(updated_parent_req.clone()) - .map(OrderingRequirements::new_single) + .map(OrderingRequirements::new) } else { None } @@ -715,7 +715,7 @@ fn handle_hash_join( // Populating with the updated requirements for children that maintain order Ok(Some(vec![ None, - LexRequirement::new(updated_parent_req).map(OrderingRequirements::new_single), + LexRequirement::new(updated_parent_req).map(OrderingRequirements::new), ])) } else { Ok(None) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a03d4a4577cc7..16e999b197e83 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -497,7 +497,7 @@ impl AggregateExec { new_requirements.extend(req); let required_input_ordering = LexRequirement::new(new_requirements) - .map(|r| OrderingRequirements::new_single_soft(r.collapse())); + .map(|r| OrderingRequirements::new_soft(r.collapse())); // If our aggregation has grouping sets then our base grouping exprs will // be expanded based on the flags in `group_by.groups` where for each diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 783729a905944..347a82ab87a9e 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -306,7 +306,7 @@ pub(crate) fn calc_requirements< alternatives.extend(LexRequirement::new(sort_reqs_with_partition)); alternatives.extend(LexRequirement::new(sort_reqs)); - OrderingRequirements::new(alternatives, false) + OrderingRequirements::new_alternatives(alternatives, false) } /// This function calculates the indices such that when partition by expressions reordered with the indices @@ -774,7 +774,7 @@ mod tests { if let Some(alts) = expected.as_mut() { alts.add_alternative(requirements); } else { - expected = Some(OrderingRequirements::new_single(requirements)); + expected = Some(OrderingRequirements::new(requirements)); } } } From 1ba1830076b62aa8c3ddeb92e952298ea41bff2c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 22:15:43 +0300 Subject: [PATCH 095/167] Simplify constant expressions --- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 59 ++++++------------- .../physical-expr/src/equivalence/ordering.rs | 3 +- .../src/equivalence/properties/mod.rs | 40 ++++++------- .../src/equivalence/properties/union.rs | 32 +++++----- .../physical-plan/src/aggregates/mod.rs | 3 +- datafusion/physical-plan/src/filter.rs | 15 ++--- datafusion/physical-plan/src/windows/mod.rs | 8 +-- 9 files changed, 70 insertions(+), 94 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 0d63bfc3a2f60..d0298f9545cc5 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -376,7 +376,7 @@ pub fn generate_table_for_eq_properties( // Fill constant columns for constant in eq_properties.constants() { - let col = constant.expr().as_any().downcast_ref::().unwrap(); + let col = constant.expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) as ArrayRef; diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 23848433b069d..b4667c12891c5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -150,7 +150,7 @@ mod sp_repartition_fuzz_tests { // Fill constant columns for constant in eq_properties.constants() { - let col = constant.expr().as_any().downcast_ref::().unwrap(); + let col = constant.expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 2d5610184e9d0..3ad8ce95d005b 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -29,16 +29,16 @@ use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; -/// A structure representing a expression known to be constant in a physical execution plan. +/// A structure representing a expression known to be constant in a physical +/// execution plan. /// -/// The `ConstExpr` struct encapsulates an expression that is constant during the execution -/// of a query. For example if a predicate like `A = 5` applied earlier in the plan `A` would -/// be known constant +/// The `ConstExpr` struct encapsulates an expression that is constant during +/// the execution of a query. For example if a filter like `A = 5` appears +/// earlier in the plan, `A` would become a constant in subsequent operations. /// /// # Fields /// /// - `expr`: Constant expression for a node in the physical plan. -/// /// - `across_partitions`: A boolean flag indicating whether the constant /// expression is the same across partitions. If set to `true`, the constant /// expression has same value for all partitions. If set to `false`, the @@ -58,10 +58,10 @@ use indexmap::{IndexMap, IndexSet}; #[derive(Debug, Clone)] pub struct ConstExpr { /// The expression that is known to be constant (e.g. a `Column`) - pub(crate) expr: Arc, + pub expr: Arc, /// Does the constant have the same value across all partitions? See /// struct docs for more details - across_partitions: AcrossPartitions, + pub across_partitions: AcrossPartitions, } /// Represents whether a constant expression's value is uniform or varies across @@ -70,7 +70,7 @@ pub struct ConstExpr { /// different partitions. /// - `Uniform(Option)`: The constant expression has the same value /// across all partitions, or is `None` if the value is unknown. -#[derive(PartialEq, Clone, Debug)] +#[derive(Clone, Debug, Eq, PartialEq)] pub enum AcrossPartitions { Heterogeneous, Uniform(Option), @@ -91,41 +91,16 @@ impl PartialEq for ConstExpr { impl ConstExpr { /// Create a new constant expression from a physical expression. /// - /// Note you can also use `ConstExpr::from` to create a constant expression - /// from a reference as well. - pub fn new(expr: Arc) -> Self { + /// Note that you can also use `ConstExpr::from` to create a constant + /// expression from just a physical expression, with the *safe* assumption + /// of heterogenous values across partitions. + pub fn new(expr: Arc, across_partitions: AcrossPartitions) -> Self { Self { expr, - // By default, assume constant expressions are not same across partitions. - across_partitions: Default::default(), + across_partitions, } } - /// Sets the `across_partitions` flag. - pub fn with_across_partitions(mut self, across_partitions: AcrossPartitions) -> Self { - self.across_partitions = across_partitions; - self - } - - /// Indicates whther the expression the same across all partitions. - pub fn across_partitions(&self) -> AcrossPartitions { - self.across_partitions.clone() - } - - pub fn expr(&self) -> &Arc { - &self.expr - } - - pub fn map(&self, f: F) -> Option - where - F: Fn(&Arc) -> Option>, - { - f(&self.expr).map(|expr| Self { - expr, - across_partitions: self.across_partitions.clone(), - }) - } - /// Returns a [`Display`]able list of `ConstExpr`. pub fn format_list(input: &[ConstExpr]) -> impl Display + '_ { struct DisplayableList<'a>(&'a [ConstExpr]); @@ -168,13 +143,17 @@ impl Display for ConstExpr { impl From> for ConstExpr { fn from(expr: Arc) -> Self { - Self::new(expr) + Self { + expr, + // By default, assume constant expressions are not same across partitions. + across_partitions: Default::default(), + } } } impl From<&Arc> for ConstExpr { fn from(expr: &Arc) -> Self { - Self::new(Arc::clone(expr)) + Self::from(Arc::clone(expr)) } } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index bf1df465a64bc..d47d9c76cf611 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -663,8 +663,7 @@ mod tests { eq_properties.add_equivalence_group(eq_group); let constants = constants.into_iter().map(|expr| { - ConstExpr::from(expr) - .with_across_partitions(AcrossPartitions::Uniform(None)) + ConstExpr::new(Arc::clone(expr), AcrossPartitions::Uniform(None)) }); eq_properties = eq_properties.with_constants(constants); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 2e9e8fa5ea808..8a155f9194c66 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -252,9 +252,8 @@ impl EquivalenceProperties { /// Removes constant expressions that may change across partitions. /// This method should be used when data from different partitions are merged. pub fn clear_per_partition_constants(&mut self) { - self.constants.retain(|item| { - matches!(item.across_partitions(), AcrossPartitions::Uniform(_)) - }) + self.constants + .retain(|item| matches!(item.across_partitions, AcrossPartitions::Uniform(_))) } /// Extends this `EquivalenceProperties` by adding the orderings inside the @@ -297,15 +296,15 @@ impl EquivalenceProperties { if self.is_expr_constant(left) { // Left expression is constant, add right as constant if !const_exprs_contains(&self.constants, right) { - let const_expr = ConstExpr::from(right) - .with_across_partitions(self.get_expr_constant_value(left)); + let across_parts = self.get_expr_constant_value(left); + let const_expr = ConstExpr::new(Arc::clone(right), across_parts); self.constants.push(const_expr); } } else if self.is_expr_constant(right) { // Right expression is constant, add left as constant if !const_exprs_contains(&self.constants, left) { - let const_expr = ConstExpr::from(left) - .with_across_partitions(self.get_expr_constant_value(right)); + let across_parts = self.get_expr_constant_value(right); + let const_expr = ConstExpr::new(Arc::clone(left), across_parts); self.constants.push(const_expr); } } @@ -326,12 +325,9 @@ impl EquivalenceProperties { let normalized_constants = constants .into_iter() .filter_map(|c| { - let across_partitions = c.across_partitions(); let normalized_expr = self.eq_group.normalize_expr(c.expr); - (!const_exprs_contains(&self.constants, &normalized_expr)).then(|| { - let const_expr = ConstExpr::from(normalized_expr); - const_expr.with_across_partitions(across_partitions) - }) + (!const_exprs_contains(&self.constants, &normalized_expr)) + .then(|| ConstExpr::new(normalized_expr, c.across_partitions)) }) .collect::>(); @@ -1076,12 +1072,14 @@ impl EquivalenceProperties { .constants .iter() .flat_map(|const_expr| { - const_expr - .map(|expr| self.eq_group.project_expr(mapping, expr)) - .map(|projected_expr| { - projected_expr - .with_across_partitions(const_expr.across_partitions()) - }) + self.eq_group.project_expr(mapping, &const_expr.expr).map( + |projected_expr| { + ConstExpr::new( + projected_expr, + const_expr.across_partitions.clone(), + ) + }, + ) }) .collect::>(); @@ -1095,7 +1093,7 @@ impl EquivalenceProperties { } else { AcrossPartitions::Heterogeneous }; - let const_expr = ConstExpr::from(target).with_across_partitions(uniform); + let const_expr = ConstExpr::new(Arc::clone(target), uniform); projected_constants.push(const_expr); } } @@ -1256,7 +1254,7 @@ impl EquivalenceProperties { .filter(|const_expr| { !across_partitions || matches!( - const_expr.across_partitions(), + const_expr.across_partitions, AcrossPartitions::Uniform { .. } ) }) @@ -1311,7 +1309,7 @@ impl EquivalenceProperties { for const_expr in self.constants.iter() { if normalized_expr.eq(&const_expr.expr) { - return const_expr.across_partitions(); + return const_expr.across_partitions.clone(); } } diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 373f251203951..028f4803fccd8 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -57,13 +57,13 @@ fn calculate_union_binary( .iter() .find(|rhs_const| rhs_const.expr.eq(&lhs_const.expr)) .map(|rhs_const| { - let const_expr = lhs_const.clone(); - // If both sides have matching constant values, preserve the value: - if lhs_const.across_partitions() == rhs_const.across_partitions() { - const_expr - } else { - const_expr.with_across_partitions(AcrossPartitions::Heterogeneous) + let mut const_expr = lhs_const.clone(); + // If both sides have matching constant values, preserve it. + // Otherwise, set fall back to heterogeneous values. + if lhs_const.across_partitions != rhs_const.across_partitions { + const_expr.across_partitions = AcrossPartitions::Heterogeneous; } + const_expr }) }) .collect::>(); @@ -791,7 +791,7 @@ mod tests { let rhs_constants = rhs.constants(); for rhs_constant in rhs_constants { assert!( - const_exprs_contains(lhs_constants, rhs_constant.expr()), + const_exprs_contains(lhs_constants, &rhs_constant.expr), "{err_msg}\nlhs: {lhs}\nrhs: {rhs}" ); } @@ -839,7 +839,7 @@ mod tests { let constants = constants .iter() - .map(|col_name| ConstExpr::new(col(col_name, schema).unwrap())) + .map(|col_name| ConstExpr::from(col(col_name, schema).unwrap())) .collect::>(); EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings) @@ -858,14 +858,18 @@ mod tests { let literal_10 = ScalarValue::Int32(Some(10)); // Create first input with a=10 - let const_expr1 = ConstExpr::new(Arc::clone(&col_a)) - .with_across_partitions(AcrossPartitions::Uniform(Some(literal_10.clone()))); + let const_expr1 = ConstExpr::new( + Arc::clone(&col_a), + AcrossPartitions::Uniform(Some(literal_10.clone())), + ); let input1 = EquivalenceProperties::new(Arc::clone(&schema)) .with_constants(vec![const_expr1]); // Create second input with a=10 - let const_expr2 = ConstExpr::new(Arc::clone(&col_a)) - .with_across_partitions(AcrossPartitions::Uniform(Some(literal_10.clone()))); + let const_expr2 = ConstExpr::new( + Arc::clone(&col_a), + AcrossPartitions::Uniform(Some(literal_10.clone())), + ); let input2 = EquivalenceProperties::new(Arc::clone(&schema)) .with_constants(vec![const_expr2]); @@ -874,9 +878,9 @@ mod tests { // Verify column 'a' remains constant with value 10 let const_a = &union_props.constants()[0]; - assert!(const_a.expr().eq(&col_a)); + assert!(const_a.expr.eq(&col_a)); assert_eq!( - const_a.across_partitions(), + const_a.across_partitions, AcrossPartitions::Uniform(Some(literal_10)) ); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 16e999b197e83..c0db97a972bd5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -679,7 +679,8 @@ impl AggregateExec { if group_expr_mapping.map.is_empty() { let mut constants = eq_properties.constants().to_vec(); let new_constants = aggr_exprs.iter().enumerate().map(|(idx, func)| { - ConstExpr::new(Arc::new(Column::new(func.name(), idx))) + let column = Arc::new(Column::new(func.name(), idx)); + ConstExpr::from(column as Arc) }); constants.extend(new_constants); eq_properties = eq_properties.with_constants(constants); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index a8a9973ea0434..a8b5db87b06f0 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -228,19 +228,15 @@ impl FilterExec { binary.right(), input_eqs.get_expr_constant_value(binary.right()), ); - res_constants.push( - ConstExpr::new(Arc::clone(expr)) - .with_across_partitions(across_parts), - ); + res_constants + .push(ConstExpr::new(Arc::clone(expr), across_parts)); } else if input_eqs.is_expr_constant(binary.right()) { let (expr, across_parts) = ( binary.left(), input_eqs.get_expr_constant_value(binary.left()), ); - res_constants.push( - ConstExpr::new(Arc::clone(expr)) - .with_across_partitions(across_parts), - ); + res_constants + .push(ConstExpr::new(Arc::clone(expr), across_parts)); } } } @@ -272,8 +268,7 @@ impl FilterExec { .min_value .get_value(); let expr = Arc::new(column) as _; - ConstExpr::new(expr) - .with_across_partitions(AcrossPartitions::Uniform(value.cloned())) + ConstExpr::new(expr, AcrossPartitions::Uniform(value.cloned())) }); // This is for statistics eq_properties = eq_properties.with_constants(constants); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 347a82ab87a9e..1bcbeb722f343 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -22,7 +22,6 @@ mod utils; mod window_agg_exec; use std::borrow::Borrow; -use std::iter; use std::sync::Arc; use crate::{ @@ -387,17 +386,18 @@ pub(crate) fn window_equivalence_properties( // unbounded starting point. // First, check if the frame covers the whole table: if plain_expr.get_window_frame().end_bound.is_unbounded() { - let window_col = Column::new(expr.name(), i + input_schema_len); + let window_col = + Arc::new(Column::new(expr.name(), i + input_schema_len)) as _; if no_partitioning { // Window function has a constant result across the table: window_eq_properties = window_eq_properties - .with_constants(iter::once(ConstExpr::new(Arc::new(window_col)))) + .with_constants(std::iter::once(ConstExpr::from(window_col))) } else { // Window function results in a partial constant value in // some ordering. Adjust the ordering equivalences accordingly: let new_lexs = all_satisfied_lexs.into_iter().flat_map(|lex| { let new_partial_consts = - sort_options_resolving_constant(Arc::new(window_col.clone())); + sort_options_resolving_constant(Arc::clone(&window_col)); new_partial_consts.into_iter().map(move |partial| { let mut existing = lex.clone(); From f2f48a10fb5055934f1c546dbca265b7cce83b78 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 22:48:32 +0300 Subject: [PATCH 096/167] Simplify constant expressions - 2 --- datafusion/core/tests/fuzz_cases/equivalence/utils.rs | 2 +- .../fuzz_cases/sort_preserving_repartition_fuzz.rs | 3 ++- datafusion/physical-expr/src/equivalence/class.rs | 6 ------ .../src/equivalence/properties/dependency.rs | 11 +++++++---- .../physical-expr/src/equivalence/properties/mod.rs | 4 ++-- datafusion/physical-plan/src/windows/mod.rs | 2 +- 6 files changed, 13 insertions(+), 15 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index d0298f9545cc5..2d7e546f856b9 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -102,7 +102,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.with_constants([ConstExpr::from(col_e)]); + eq_properties = eq_properties.with_constants([ConstExpr::from(Arc::clone(col_e))]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index b4667c12891c5..513020b8e4719 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -81,7 +81,8 @@ mod sp_repartition_fuzz_tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.with_constants([ConstExpr::from(col_e)]); + eq_properties = + eq_properties.with_constants([ConstExpr::from(Arc::clone(col_e))]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 3ad8ce95d005b..7409ec1bcaace 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -151,12 +151,6 @@ impl From> for ConstExpr { } } -impl From<&Arc> for ConstExpr { - fn from(expr: &Arc) -> Self { - Self::from(Arc::clone(expr)) - } -} - /// Checks whether `expr` is among in the `const_exprs`. pub fn const_exprs_contains( const_exprs: &[ConstExpr], diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 311a446179ccf..5999dcaa54c7e 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -891,7 +891,8 @@ mod tests { let col_h = &col("h", &test_schema)?; // Add column h as constant - eq_properties = eq_properties.with_constants(vec![ConstExpr::from(col_h)]); + eq_properties = + eq_properties.with_constants(vec![ConstExpr::from(Arc::clone(col_h))]); let test_cases = vec![ // TEST CASE 1 @@ -1309,11 +1310,12 @@ mod tests { // Setup constant columns let col_a = col("a", &schema)?; let col_b = col("b", &schema)?; - eq_properties = eq_properties.with_constants([ConstExpr::from(&col_a)]); + eq_properties = + eq_properties.with_constants([ConstExpr::from(Arc::clone(&col_a))]); let sort_exprs = vec![ PhysicalSortExpr { - expr: Arc::clone(&col_a), + expr: col_a, options: SortOptions::default(), }, PhysicalSortExpr { @@ -1490,7 +1492,8 @@ mod tests { let asc = SortOptions::default(); // Constants: c is constant - eq_properties = eq_properties.with_constants([ConstExpr::from(&col_c)]); + eq_properties = + eq_properties.with_constants([ConstExpr::from(Arc::clone(&col_c))]); // Equality: b = d eq_properties.add_equal_conditions(&col_b, &col_d)?; diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 8a155f9194c66..857db06e0d5d2 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1212,8 +1212,8 @@ impl EquivalenceProperties { // Note that these expressions are not properly "constants". This is just // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties = - eq_properties.with_constants(std::iter::once(ConstExpr::from(expr))); + eq_properties = eq_properties + .with_constants(std::iter::once(ConstExpr::from(Arc::clone(expr)))); search_indices.shift_remove(idx); } // Add new ordered section to the state. diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1bcbeb722f343..92b966c74da04 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -596,7 +596,7 @@ pub fn get_window_mode( }) .collect::>(); // Treat partition by exprs as constant. During analysis of requirements are satisfied. - let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); + let const_exprs = partitionby_exprs.iter().cloned().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); let reverse_orderby_keys = orderby_keys.iter().map(|e| e.reverse()).collect::>(); From b905976d651057171eec18f7bc54c8a8e01b7648 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 23:02:36 +0300 Subject: [PATCH 097/167] Simplify constant expressions - 3 --- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 3 +-- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../src/equivalence/properties/dependency.rs | 17 ++++++++--------- .../src/equivalence/properties/joins.rs | 4 ++-- .../src/equivalence/properties/mod.rs | 18 +++++++----------- .../src/equivalence/properties/union.rs | 19 ++++++++++--------- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/filter.rs | 5 ++--- datafusion/physical-plan/src/windows/mod.rs | 10 +++++----- 10 files changed, 38 insertions(+), 44 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 2d7e546f856b9..27f7c6c82c8f6 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -102,7 +102,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.with_constants([ConstExpr::from(Arc::clone(col_e))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 513020b8e4719..b47e6cd5aae56 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -81,8 +81,7 @@ mod sp_repartition_fuzz_tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = - eq_properties.with_constants([ConstExpr::from(Arc::clone(col_e))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d47d9c76cf611..0a07a7c240b8e 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -665,7 +665,7 @@ mod tests { let constants = constants.into_iter().map(|expr| { ConstExpr::new(Arc::clone(expr), AcrossPartitions::Uniform(None)) }); - eq_properties = eq_properties.with_constants(constants); + eq_properties.add_constants(constants); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 5999dcaa54c7e..b557f14136b77 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -891,8 +891,7 @@ mod tests { let col_h = &col("h", &test_schema)?; // Add column h as constant - eq_properties = - eq_properties.with_constants(vec![ConstExpr::from(Arc::clone(col_h))]); + eq_properties.add_constants(vec![ConstExpr::from(Arc::clone(col_h))]); let test_cases = vec![ // TEST CASE 1 @@ -1101,13 +1100,15 @@ mod tests { for [left, right] in &case.equal_conditions { properties.add_equal_conditions(left, right)? } - properties.with_constants( + properties.add_constants( case.constants.iter().cloned().map(ConstExpr::from), - ) + ); + properties }, // Constants before equal conditions { - let mut properties = base_properties.clone().with_constants( + let mut properties = base_properties.clone(); + properties.add_constants( case.constants.iter().cloned().map(ConstExpr::from), ); for [left, right] in &case.equal_conditions { @@ -1310,8 +1311,7 @@ mod tests { // Setup constant columns let col_a = col("a", &schema)?; let col_b = col("b", &schema)?; - eq_properties = - eq_properties.with_constants([ConstExpr::from(Arc::clone(&col_a))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_a))]); let sort_exprs = vec![ PhysicalSortExpr { @@ -1492,8 +1492,7 @@ mod tests { let asc = SortOptions::default(); // Constants: c is constant - eq_properties = - eq_properties.with_constants([ConstExpr::from(Arc::clone(&col_c))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_c))]); // Equality: b = d eq_properties.add_equal_conditions(&col_b, &col_d)?; diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 86f55a2302988..5d72fd5ff9be2 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -104,10 +104,10 @@ pub fn join_equivalence_properties( } match join_type { JoinType::LeftAnti | JoinType::LeftSemi => { - result = result.with_constants(left_constants); + result.add_constants(left_constants); } JoinType::RightAnti | JoinType::RightSemi => { - result = result.with_constants(right_constants); + result.add_constants(right_constants); } _ => {} } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 857db06e0d5d2..6a533bd1c52bb 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -240,7 +240,8 @@ impl EquivalenceProperties { pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); - self.with_constants(other.constants) + self.add_constants(other.constants); + self } /// Clears (empties) the ordering equivalence class within this object. @@ -318,10 +319,7 @@ impl EquivalenceProperties { } /// Track/register physical expressions with constant values. - pub fn with_constants( - mut self, - constants: impl IntoIterator, - ) -> Self { + pub fn add_constants(&mut self, constants: impl IntoIterator) { let normalized_constants = constants .into_iter() .filter_map(|c| { @@ -338,8 +336,6 @@ impl EquivalenceProperties { for ordering in self.normalized_oeq_class().iter() { self.discover_new_orderings(&ordering[0].expr).unwrap(); } - - self } // Discover new valid orderings in light of a new equality. @@ -570,7 +566,7 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties = eq_properties.with_constants(std::iter::once(const_expr)); + eq_properties.add_constants(std::iter::once(const_expr)); } true } @@ -622,7 +618,7 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties = eq_properties.with_constants(std::iter::once(const_expr)); + eq_properties.add_constants(std::iter::once(const_expr)); } // All sort expressions are satisfied, return full length: @@ -1212,8 +1208,8 @@ impl EquivalenceProperties { // Note that these expressions are not properly "constants". This is just // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties = eq_properties - .with_constants(std::iter::once(ConstExpr::from(Arc::clone(expr)))); + eq_properties + .add_constants(std::iter::once(ConstExpr::from(Arc::clone(expr)))); search_indices.shift_remove(idx); } // Add new ordered section to the state. diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 028f4803fccd8..ed708d690d9ed 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -75,9 +75,8 @@ fn calculate_union_binary( orderings.add_satisfied_orderings(&rhs, &lhs); let orderings = orderings.build(); - let mut eq_properties = - EquivalenceProperties::new(lhs.schema).with_constants(constants); - + let mut eq_properties = EquivalenceProperties::new(lhs.schema); + eq_properties.add_constants(constants); eq_properties.add_new_orderings(orderings); Ok(eq_properties) } @@ -842,8 +841,10 @@ mod tests { .map(|col_name| ConstExpr::from(col(col_name, schema).unwrap())) .collect::>(); - EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings) - .with_constants(constants) + let mut props = + EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings); + props.add_constants(constants); + props } } @@ -862,16 +863,16 @@ mod tests { Arc::clone(&col_a), AcrossPartitions::Uniform(Some(literal_10.clone())), ); - let input1 = EquivalenceProperties::new(Arc::clone(&schema)) - .with_constants(vec![const_expr1]); + let mut input1 = EquivalenceProperties::new(Arc::clone(&schema)); + input1.add_constants(vec![const_expr1]); // Create second input with a=10 let const_expr2 = ConstExpr::new( Arc::clone(&col_a), AcrossPartitions::Uniform(Some(literal_10.clone())), ); - let input2 = EquivalenceProperties::new(Arc::clone(&schema)) - .with_constants(vec![const_expr2]); + let mut input2 = EquivalenceProperties::new(Arc::clone(&schema)); + input2.add_constants(vec![const_expr2]); // Calculate union properties let union_props = calculate_union(vec![input1, input2], schema)?; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c0db97a972bd5..9b92e5d22bb13 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -683,7 +683,7 @@ impl AggregateExec { ConstExpr::from(column as Arc) }); constants.extend(new_constants); - eq_properties = eq_properties.with_constants(constants); + eq_properties.add_constants(constants); } // Group by expression will be a distinct value after the aggregation. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index a8b5db87b06f0..79667729750f4 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -271,11 +271,10 @@ impl FilterExec { ConstExpr::new(expr, AcrossPartitions::Uniform(value.cloned())) }); // This is for statistics - eq_properties = eq_properties.with_constants(constants); + eq_properties.add_constants(constants); // This is for logical constant (for example: a = '1', then a could be marked as a constant) // to do: how to deal with multiple situation to represent = (for example c1 between 0 and 0) - eq_properties = - eq_properties.with_constants(Self::extend_constants(input, predicate)); + eq_properties.add_constants(Self::extend_constants(input, predicate)); let mut output_partitioning = input.output_partitioning().clone(); // If contains projection, update the PlanProperties. diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 92b966c74da04..2cb2aa5091b3f 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -390,8 +390,8 @@ pub(crate) fn window_equivalence_properties( Arc::new(Column::new(expr.name(), i + input_schema_len)) as _; if no_partitioning { // Window function has a constant result across the table: - window_eq_properties = window_eq_properties - .with_constants(std::iter::once(ConstExpr::from(window_col))) + window_eq_properties + .add_constants(std::iter::once(ConstExpr::from(window_col))) } else { // Window function results in a partial constant value in // some ordering. Adjust the ordering equivalences accordingly: @@ -586,7 +586,7 @@ pub fn get_window_mode( orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Option<(bool, InputOrderMode)> { - let input_eqs = input.equivalence_properties().clone(); + let mut input_eqs = input.equivalence_properties().clone(); let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); let partition_by_reqs = indices .iter() @@ -597,7 +597,7 @@ pub fn get_window_mode( .collect::>(); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().cloned().map(ConstExpr::from); - let partition_by_eqs = input_eqs.with_constants(const_exprs); + input_eqs.add_constants(const_exprs); let reverse_orderby_keys = orderby_keys.iter().map(|e| e.reverse()).collect::>(); for (should_swap, orderbys) in @@ -605,7 +605,7 @@ pub fn get_window_mode( { let mut req = partition_by_reqs.clone(); req.extend(orderbys.iter().cloned().map(Into::into)); - if req.is_empty() || partition_by_eqs.ordering_satisfy_requirement(req) { + if req.is_empty() || input_eqs.ordering_satisfy_requirement(req) { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { InputOrderMode::Sorted From a9c17f64596df1288bdc630f7b2ac12f3cc04644 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 16 Apr 2025 23:29:01 +0300 Subject: [PATCH 098/167] Simplify constant expressions - 4 --- datafusion/physical-expr/src/equivalence/properties/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 6a533bd1c52bb..58fa71aa4b885 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -125,8 +125,8 @@ use itertools::Itertools; /// # let col_c = col("c", &schema).unwrap(); /// // This object represents data that is sorted by a ASC, c DESC /// // with a single constant value of b -/// let mut eq_properties = EquivalenceProperties::new(schema) -/// .with_constants(vec![ConstExpr::from(col_b)]); +/// let mut eq_properties = EquivalenceProperties::new(schema); +/// eq_properties.add_constants(vec![ConstExpr::from(col_b)]); /// eq_properties.add_new_ordering([ /// PhysicalSortExpr::new_default(col_a).asc(), /// PhysicalSortExpr::new_default(col_c).desc(), From 47ca84ce8621befca7984ab9fd175dd3816cfd96 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 17 Apr 2025 00:09:19 +0300 Subject: [PATCH 099/167] Simplify constant expressions - 5 --- .../physical-expr/src/equivalence/class.rs | 11 ++--- .../src/equivalence/properties/mod.rs | 46 +++++++------------ 2 files changed, 20 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7409ec1bcaace..6dfc3e13d694f 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -50,9 +50,7 @@ use indexmap::{IndexMap, IndexSet}; /// # use datafusion_physical_expr::ConstExpr; /// # use datafusion_physical_expr::expressions::lit; /// let col = lit(5); -/// // Create a constant expression from a physical expression ref -/// let const_expr = ConstExpr::from(&col); -/// // create a constant expression from a physical expression +/// // Create a constant expression from a physical expression: /// let const_expr = ConstExpr::from(col); /// ``` #[derive(Debug, Clone)] @@ -183,8 +181,8 @@ impl EquivalenceClass { } } - // Create a new equivalence class from a pre-existing `Vec`. - pub fn new(exprs: Vec>) -> Self { + // Create a new equivalence class from a pre-existing collection. + pub fn new(exprs: impl IntoIterator>) -> Self { Self { exprs: exprs.into_iter().collect(), } @@ -240,8 +238,7 @@ impl EquivalenceClass { .exprs .iter() .cloned() - .map(|e| add_offset_to_expr(e, offset)) - .collect(); + .map(|e| add_offset_to_expr(e, offset)); Self::new(new_exprs) } } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 58fa71aa4b885..545b3daeb6177 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -42,7 +42,6 @@ use crate::{ PhysicalSortExpr, PhysicalSortRequirement, }; -use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ @@ -996,16 +995,13 @@ impl EquivalenceProperties { } }) .flat_map(|(options, relevant_deps)| { - let sort_expr = PhysicalSortExpr { - expr: Arc::clone(target), - options, - }; + let sort_expr = PhysicalSortExpr::new(Arc::clone(target), options); // Generate dependent orderings (i.e. prefixes for `sort_expr`): let mut dependency_orderings = generate_dependency_orderings(&relevant_deps, &dependency_map); // Append `sort_expr` to the dependent orderings: if dependency_orderings.is_empty() { - dependency_orderings.push([sort_expr.clone()].into()); + dependency_orderings.push([sort_expr].into()); } else { for ordering in dependency_orderings.iter_mut() { ordering.push(sort_expr.clone()); @@ -1067,7 +1063,7 @@ impl EquivalenceProperties { let mut projected_constants = self .constants .iter() - .flat_map(|const_expr| { + .filter_map(|const_expr| { self.eq_group.project_expr(mapping, &const_expr.expr).map( |projected_expr| { ConstExpr::new( @@ -1165,32 +1161,23 @@ impl EquivalenceProperties { // This algorithm should reach a fixed point in at most `exprs.len()` // iterations. let mut search_indices = (0..exprs.len()).collect::>(); - for _idx in 0..exprs.len() { + for _ in 0..exprs.len() { // Get ordered expressions with their indices. let ordered_exprs = search_indices .iter() - .flat_map(|&idx| { + .filter_map(|&idx| { let ExprProperties { sort_properties, .. } = eq_properties.get_expr_properties(Arc::clone(&exprs[idx])); match sort_properties { - SortProperties::Ordered(options) => Some(( - PhysicalSortExpr { - expr: Arc::clone(&exprs[idx]), - options, - }, - idx, - )), + SortProperties::Ordered(options) => { + let expr = Arc::clone(&exprs[idx]); + Some((PhysicalSortExpr::new(expr, options), idx)) + } SortProperties::Singleton => { - // Assign default ordering to constant expressions - let options = SortOptions::default(); - Some(( - PhysicalSortExpr { - expr: Arc::clone(&exprs[idx]), - options, - }, - idx, - )) + // Assign default ordering to constant expressions: + let expr = Arc::clone(&exprs[idx]); + Some((PhysicalSortExpr::new_default(expr), idx)) } SortProperties::Unordered => None, } @@ -1208,8 +1195,8 @@ impl EquivalenceProperties { // Note that these expressions are not properly "constants". This is just // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties - .add_constants(std::iter::once(ConstExpr::from(Arc::clone(expr)))); + let const_expr = ConstExpr::from(Arc::clone(expr)); + eq_properties.add_constants(std::iter::once(const_expr)); search_indices.shift_remove(idx); } // Add new ordered section to the state. @@ -1387,7 +1374,7 @@ impl EquivalenceProperties { let new_eq_exprs = eq_class .into_iter() .map(|expr| with_new_schema(expr, &schema)) - .collect::>()?; + .collect::>>()?; eq_classes.push(EquivalenceClass::new(new_eq_exprs)); } @@ -1600,7 +1587,6 @@ impl Hash for ExprWrapper { #[cfg(test)] mod tests { - use super::*; use crate::expressions::{col, BinaryExpr}; @@ -1623,7 +1609,7 @@ mod tests { Arc::clone(&col_b), Operator::Plus, Arc::clone(&col_d), - )) as Arc; + )) as _; let constants = vec![Arc::clone(&col_a), Arc::clone(&col_b)]; let expr = Arc::clone(&b_plus_d); From 2f8043999457aa2d3c6c72b29f6a666f1024fe8a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 17 Apr 2025 00:19:20 +0300 Subject: [PATCH 100/167] Simplify constant expressions - 6 --- .../physical-expr/src/equivalence/class.rs | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 6dfc3e13d694f..063c79cce4de5 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use std::vec::IntoIter; use super::{add_offset_to_expr, ProjectionMapping}; -use crate::expressions::Column; +use crate::expressions::{Column, Literal}; use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -91,7 +91,8 @@ impl ConstExpr { /// /// Note that you can also use `ConstExpr::from` to create a constant /// expression from just a physical expression, with the *safe* assumption - /// of heterogenous values across partitions. + /// of heterogenous values across partitions unless the expression is a + /// literal. pub fn new(expr: Arc, across_partitions: AcrossPartitions) -> Self { Self { expr, @@ -141,10 +142,17 @@ impl Display for ConstExpr { impl From> for ConstExpr { fn from(expr: Arc) -> Self { + // By default, assume constant expressions are not same across partitions. + // However, if we have a literal, it will have a single value that is the + // same across all partitions. + let across = if let Some(lit) = expr.as_any().downcast_ref::() { + AcrossPartitions::Uniform(Some(lit.value().clone())) + } else { + AcrossPartitions::Heterogeneous + }; Self { expr, - // By default, assume constant expressions are not same across partitions. - across_partitions: Default::default(), + across_partitions: across, } } } From 591fc743f2fe3a7ad4b11042febd2cc013d76609 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 17 Apr 2025 10:48:23 +0300 Subject: [PATCH 101/167] Simplify constant expressions - 7 --- .../physical-expr/src/equivalence/class.rs | 57 +++++++++---------- .../src/equivalence/properties/mod.rs | 7 +-- 2 files changed, 31 insertions(+), 33 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 063c79cce4de5..82175067403f5 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -29,6 +29,19 @@ use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; +/// Represents whether a constant expression's value is uniform or varies across +/// partitions. Has two variants: +/// - `Heterogeneous`: The constant expression may have different values for +/// different partitions. +/// - `Uniform(Option)`: The constant expression has the same value +/// across all partitions, or is `None` if the value is unknown. +#[derive(Clone, Debug, Default, Eq, PartialEq)] +pub enum AcrossPartitions { + #[default] + Heterogeneous, + Uniform(Option), +} + /// A structure representing a expression known to be constant in a physical /// execution plan. /// @@ -53,7 +66,7 @@ use indexmap::{IndexMap, IndexSet}; /// // Create a constant expression from a physical expression: /// let const_expr = ConstExpr::from(col); /// ``` -#[derive(Debug, Clone)] +#[derive(Clone, Debug)] pub struct ConstExpr { /// The expression that is known to be constant (e.g. a `Column`) pub expr: Arc, @@ -62,42 +75,22 @@ pub struct ConstExpr { pub across_partitions: AcrossPartitions, } -/// Represents whether a constant expression's value is uniform or varies across -/// partitions. Has two variants: -/// - `Heterogeneous`: The constant expression may have different values for -/// different partitions. -/// - `Uniform(Option)`: The constant expression has the same value -/// across all partitions, or is `None` if the value is unknown. -#[derive(Clone, Debug, Eq, PartialEq)] -pub enum AcrossPartitions { - Heterogeneous, - Uniform(Option), -} - -impl Default for AcrossPartitions { - fn default() -> Self { - Self::Heterogeneous - } -} - -impl PartialEq for ConstExpr { - fn eq(&self, other: &Self) -> bool { - self.across_partitions == other.across_partitions && self.expr.eq(&other.expr) - } -} - impl ConstExpr { - /// Create a new constant expression from a physical expression. + /// Create a new constant expression from a physical expression, specifying + /// whether the constant expression is the same across partitions. /// /// Note that you can also use `ConstExpr::from` to create a constant /// expression from just a physical expression, with the *safe* assumption /// of heterogenous values across partitions unless the expression is a /// literal. pub fn new(expr: Arc, across_partitions: AcrossPartitions) -> Self { - Self { - expr, - across_partitions, + let mut result = ConstExpr::from(expr); + // Override the across partitions specification if the expression is not + // a literal. + if result.across_partitions == AcrossPartitions::Heterogeneous { + result.across_partitions = across_partitions; } + result } /// Returns a [`Display`]able list of `ConstExpr`. @@ -121,6 +114,12 @@ impl ConstExpr { } } +impl PartialEq for ConstExpr { + fn eq(&self, other: &Self) -> bool { + self.across_partitions == other.across_partitions && self.expr.eq(&other.expr) + } +} + impl Display for ConstExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{}", self.expr)?; diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 545b3daeb6177..c441658ac9aa7 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -321,10 +321,9 @@ impl EquivalenceProperties { pub fn add_constants(&mut self, constants: impl IntoIterator) { let normalized_constants = constants .into_iter() - .filter_map(|c| { - let normalized_expr = self.eq_group.normalize_expr(c.expr); - (!const_exprs_contains(&self.constants, &normalized_expr)) - .then(|| ConstExpr::new(normalized_expr, c.across_partitions)) + .filter_map(|mut c| { + c.expr = self.eq_group.normalize_expr(c.expr); + (!const_exprs_contains(&self.constants, &c.expr)).then_some(c) }) .collect::>(); From 79ad9d7215ce296bce4adf8fe6014d5be419082b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 17 Apr 2025 11:17:21 +0300 Subject: [PATCH 102/167] Simplify constant expressions - 8 --- datafusion/catalog/src/listing_schema.rs | 6 ++--- datafusion/catalog/src/memory/table.rs | 2 +- datafusion/catalog/src/stream.rs | 2 +- .../common/src/functional_dependencies.rs | 22 +++++-------------- .../core/src/datasource/listing/table.rs | 2 +- .../src/datasource/listing_table_factory.rs | 4 ++-- datafusion/datasource/src/file_scan_config.rs | 8 ++----- datafusion/expr/src/logical_plan/plan.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 22 +++++-------------- .../physical-expr/src/equivalence/ordering.rs | 7 +----- .../src/equivalence/properties/mod.rs | 14 +++++------- datafusion/sql/src/query.rs | 2 +- 12 files changed, 29 insertions(+), 64 deletions(-) diff --git a/datafusion/catalog/src/listing_schema.rs b/datafusion/catalog/src/listing_schema.rs index cc2c2ee606b3d..2e4eac964b181 100644 --- a/datafusion/catalog/src/listing_schema.rs +++ b/datafusion/catalog/src/listing_schema.rs @@ -25,9 +25,7 @@ use std::sync::{Arc, Mutex}; use crate::{SchemaProvider, TableProvider, TableProviderFactory}; use crate::Session; -use datafusion_common::{ - Constraints, DFSchema, DataFusionError, HashMap, TableReference, -}; +use datafusion_common::{DFSchema, DataFusionError, HashMap, TableReference}; use datafusion_expr::CreateExternalTable; use async_trait::async_trait; @@ -143,7 +141,7 @@ impl ListingSchemaProvider { order_exprs: vec![], unbounded: false, options: Default::default(), - constraints: Constraints::empty(), + constraints: Default::default(), column_defaults: Default::default(), }, ) diff --git a/datafusion/catalog/src/memory/table.rs b/datafusion/catalog/src/memory/table.rs index 86b8d7578fa20..e996e1974d9e8 100644 --- a/datafusion/catalog/src/memory/table.rs +++ b/datafusion/catalog/src/memory/table.rs @@ -86,7 +86,7 @@ impl MemTable { .into_iter() .map(|e| Arc::new(RwLock::new(e))) .collect::>(), - constraints: Constraints::empty(), + constraints: Constraints::default(), column_defaults: HashMap::new(), sort_order: Arc::new(Mutex::new(vec![])), }) diff --git a/datafusion/catalog/src/stream.rs b/datafusion/catalog/src/stream.rs index 94685c73548d1..99c432b738e5b 100644 --- a/datafusion/catalog/src/stream.rs +++ b/datafusion/catalog/src/stream.rs @@ -256,7 +256,7 @@ impl StreamConfig { Self { source, order: vec![], - constraints: Constraints::empty(), + constraints: Constraints::default(), } } diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 32313377086e6..2d2d15dfbd2e6 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -36,22 +36,18 @@ pub enum Constraint { } /// This object encapsulates a list of functional constraints: -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +#[derive(Clone, Debug, Default, Eq, Hash, PartialEq, PartialOrd)] pub struct Constraints { inner: Vec, } impl Constraints { - /// Create empty constraints - pub fn empty() -> Self { - Constraints::new_unverified(vec![]) - } - /// Create a new [`Constraints`] object from the given `constraints`. - /// Users should use the [`Constraints::empty`] or [`SqlToRel::new_constraint_from_table_constraints`] functions - /// for constructing [`Constraints`]. This constructor is for internal - /// purposes only and does not check whether the argument is valid. The user - /// is responsible for supplying a valid vector of [`Constraint`] objects. + /// Users should use the [`Constraints::default`] or [`SqlToRel::new_constraint_from_table_constraints`] + /// functions for constructing [`Constraints`] instances. This constructor + /// is for internal purposes only and does not check whether the argument + /// is valid. The user is responsible for supplying a valid vector of + /// [`Constraint`] objects. /// /// [`SqlToRel::new_constraint_from_table_constraints`]: https://docs.rs/datafusion/latest/datafusion/sql/planner/struct.SqlToRel.html#method.new_constraint_from_table_constraints pub fn new_unverified(constraints: Vec) -> Self { @@ -93,12 +89,6 @@ impl Constraints { } } -impl Default for Constraints { - fn default() -> Self { - Constraints::empty() - } -} - impl IntoIterator for Constraints { type Item = Constraint; type IntoIter = IntoIter; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index fffe5d14e6fc4..4b4cd87987afa 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -766,7 +766,7 @@ impl ListingTable { options, definition: None, collected_statistics: Arc::new(DefaultFileStatisticsCache::default()), - constraints: Constraints::empty(), + constraints: Constraints::default(), column_defaults: HashMap::new(), }; diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 636d1623c5e91..2426af320e09c 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -202,7 +202,7 @@ mod tests { order_exprs: vec![], unbounded: false, options: HashMap::from([("format.has_header".into(), "true".into())]), - constraints: Constraints::empty(), + constraints: Constraints::default(), column_defaults: HashMap::new(), }; let table_provider = factory.create(&state, &cmd).await.unwrap(); @@ -242,7 +242,7 @@ mod tests { order_exprs: vec![], unbounded: false, options, - constraints: Constraints::empty(), + constraints: Constraints::default(), column_defaults: HashMap::new(), }; let table_provider = factory.create(&state, &cmd).await.unwrap(); diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 530eb354142ab..266ded970c6fd 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -615,7 +615,7 @@ impl FileScanConfig { object_store_url, file_schema, file_groups: vec![], - constraints: Constraints::empty(), + constraints: Constraints::default(), projection: None, limit: None, table_partition_cols: vec![], @@ -704,10 +704,7 @@ impl FileScanConfig { pub fn projected_constraints(&self) -> Constraints { let indexes = self.projection_indices(); - - self.constraints - .project(&indexes) - .unwrap_or_else(Constraints::empty) + self.constraints.project(&indexes).unwrap_or_default() } /// Set the projection of the files @@ -2174,7 +2171,6 @@ mod tests { wrap_partition_type_in_dict(DataType::Utf8), false, )]) - .with_constraints(Constraints::empty()) .with_statistics(Statistics::new_unknown(&file_schema)) .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( "test.parquet".to_string(), diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 76b45d5d723ae..bad41f85a9aba 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -991,7 +991,7 @@ impl LogicalPlan { Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { input: Arc::new(input), - constraints: Constraints::empty(), + constraints: Constraints::default(), name: name.clone(), if_not_exists: *if_not_exists, or_replace: *or_replace, diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 82175067403f5..a2535fe9608c3 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -173,7 +173,7 @@ pub fn const_exprs_contains( /// /// Two `EquivalenceClass`es are equal if they contains the same expressions in /// without any ordering. -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Clone, Debug, Default, Eq, PartialEq)] pub struct EquivalenceClass { /// The expressions in this equivalence class. The order doesn't matter for /// equivalence purposes. @@ -181,13 +181,6 @@ pub struct EquivalenceClass { } impl EquivalenceClass { - /// Create a new empty equivalence class. - pub fn new_empty() -> Self { - Self { - exprs: IndexSet::new(), - } - } - // Create a new equivalence class from a pre-existing collection. pub fn new(exprs: impl IntoIterator>) -> Self { Self { @@ -272,17 +265,12 @@ impl From for Vec> { } /// A collection of distinct `EquivalenceClass`es -#[derive(Debug, Clone)] +#[derive(Clone, Debug, Default)] pub struct EquivalenceGroup { classes: Vec, } impl EquivalenceGroup { - /// Creates an empty equivalence group. - pub fn empty() -> Self { - Self { classes: vec![] } - } - /// Creates an equivalence group from the given equivalence classes. pub fn new(classes: Vec) -> Self { let mut result = Self { classes }; @@ -501,7 +489,7 @@ impl EquivalenceGroup { // The key is the source expression, and the value is the equivalence // class that contains the corresponding target expression. - let mut new_classes: IndexMap<_, _> = IndexMap::new(); + let mut new_classes = IndexMap::<_, EquivalenceClass>::new(); for (source, target) in mapping.iter() { // We need to find equivalent projected expressions. For example, // consider a table with columns `[a, b, c]` with `a` == `b`, and @@ -511,7 +499,7 @@ impl EquivalenceGroup { let normalized_expr = self.normalize_expr(Arc::clone(source)); new_classes .entry(normalized_expr) - .or_insert_with(EquivalenceClass::new_empty) + .or_default() .push(Arc::clone(target)); } // Only add equivalence classes with at least two members as singleton @@ -970,7 +958,7 @@ mod tests { Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ])); - let mut group = EquivalenceGroup::empty(); + let mut group = EquivalenceGroup::default(); group.add_equal_conditions(&col("a", &schema)?, &col("b", &schema)?); let projected_schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 0a07a7c240b8e..876adaa6be295 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -40,17 +40,12 @@ use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// /// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table /// ordering. In this case, we say that these orderings are equivalent. -#[derive(Debug, Clone, Eq, PartialEq, Hash, Default)] +#[derive(Clone, Debug, Default, Eq, PartialEq, Hash)] pub struct OrderingEquivalenceClass { orderings: Vec, } impl OrderingEquivalenceClass { - /// Creates new empty ordering equivalence class. - pub fn empty() -> Self { - Default::default() - } - /// Clears (empties) this ordering equivalence class. pub fn clear(&mut self) { self.orderings.clear(); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index c441658ac9aa7..2554f6fd25012 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -155,10 +155,10 @@ impl EquivalenceProperties { /// Creates an empty `EquivalenceProperties` object. pub fn new(schema: SchemaRef) -> Self { Self { - eq_group: EquivalenceGroup::empty(), - oeq_class: OrderingEquivalenceClass::empty(), + eq_group: EquivalenceGroup::default(), + oeq_class: OrderingEquivalenceClass::default(), constants: vec![], - constraints: Constraints::empty(), + constraints: Constraints::default(), schema, } } @@ -176,10 +176,10 @@ impl EquivalenceProperties { ) -> Self { let orderings = orderings.into_iter().filter_map(LexOrdering::new).collect(); Self { - eq_group: EquivalenceGroup::empty(), + eq_group: EquivalenceGroup::default(), oeq_class: OrderingEquivalenceClass::new(orderings), constants: vec![], - constraints: Constraints::empty(), + constraints: Constraints::default(), schema, } } @@ -1122,9 +1122,7 @@ impl EquivalenceProperties { let eq_group = self.eq_group.project(mapping); let oeq_class = OrderingEquivalenceClass::new(self.projected_orderings(mapping)); let constants = self.projected_constants(mapping); - let constraints = self - .projected_constraints(mapping) - .unwrap_or_else(Constraints::empty); + let constraints = self.projected_constraints(mapping).unwrap_or_default(); Self { schema: output_schema, eq_group, diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index ea641320c01b4..541f2f553f187 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -137,7 +137,7 @@ impl SqlToRel<'_, S> { Some(into) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( CreateMemoryTable { name: self.object_name_to_table_reference(into.name)?, - constraints: Constraints::empty(), + constraints: Constraints::default(), input: Arc::new(plan), if_not_exists: false, or_replace: false, From 61a41bc9969699d148b9089a65b7b776a297e9d8 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 17 Apr 2025 12:12:14 +0300 Subject: [PATCH 103/167] Simplify constant expressions - 9 --- datafusion/physical-expr/src/equivalence/class.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index a2535fe9608c3..94e12436d14cb 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -190,8 +190,8 @@ impl EquivalenceClass { /// Return the "canonical" expression for this class (the first element) /// if non-empty. - fn canonical_expr(&self) -> Option> { - self.exprs.iter().next().cloned() + pub fn canonical_expr(&self) -> Option<&Arc> { + self.exprs.iter().next() } /// Insert the expression into this class, meaning it is known to be equal to @@ -398,10 +398,12 @@ impl EquivalenceGroup { pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.transform(|expr| { for cls in self.iter() { - if cls.contains(&expr) { - // The unwrap below is safe because the guard above ensures - // that the class is not empty. - return Ok(Transformed::yes(cls.canonical_expr().unwrap())); + // If the equivalence class is non-empty, and it contains this + // expression, use its canonical version: + if let Some(canonical) = cls.canonical_expr() { + if cls.contains(&expr) { + return Ok(Transformed::yes(Arc::clone(canonical))); + } } } Ok(Transformed::no(expr)) From ea6fe10225474804eca0dbf9ee44e4694d02bdc0 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 21 Apr 2025 12:52:46 +0300 Subject: [PATCH 104/167] Fix imports --- datafusion/functions-aggregate/src/string_agg.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 5577f37fb9823..1195e2a59cff8 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -17,12 +17,15 @@ //! [`StringAgg`] accumulator for the `string_agg` function +use std::any::Any; +use std::mem::size_of_val; + use crate::array_agg::ArrayAgg; + use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; use datafusion_common::cast::as_generic_string_array; -use datafusion_common::Result; -use datafusion_common::{internal_err, not_impl_err, ScalarValue}; +use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::{ Accumulator, AggregateUDFImpl, Documentation, Signature, TypeSignature, Volatility, @@ -30,8 +33,6 @@ use datafusion_expr::{ use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; use datafusion_macros::user_doc; use datafusion_physical_expr::expressions::Literal; -use std::any::Any; -use std::mem::size_of_val; make_udaf_expr_and_func!( StringAgg, From 9b225459a46a171745898b5416dde92857bb63ae Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 24 Apr 2025 01:38:49 +0300 Subject: [PATCH 105/167] Remove explicit constant tracking from equivalences --- .../tests/fuzz_cases/equivalence/utils.rs | 44 +- .../sort_preserving_repartition_fuzz.rs | 2 +- datafusion/datasource/src/memory.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 363 +++++++++++------ .../physical-expr/src/equivalence/mod.rs | 56 +-- .../physical-expr/src/equivalence/ordering.rs | 17 +- .../src/equivalence/projection.rs | 2 +- .../src/equivalence/properties/dependency.rs | 47 +-- .../src/equivalence/properties/joins.rs | 27 +- .../src/equivalence/properties/mod.rs | 383 ++++-------------- .../src/equivalence/properties/union.rs | 51 ++- datafusion/physical-optimizer/src/utils.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/filter.rs | 24 +- datafusion/physical-plan/src/test.rs | 2 +- 15 files changed, 451 insertions(+), 575 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 27f7c6c82c8f6..a987d080fdb0b 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -100,7 +100,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); // Define a and f are aliases - eq_properties.add_equal_conditions(col_a, col_f)?; + eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_f))?; // Column e has constant value. eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); @@ -168,49 +168,49 @@ fn add_equal_conditions_test() -> Result<()> { ])); let mut eq_properties = EquivalenceProperties::new(schema); - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; - let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; - let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + let col_a = Arc::new(Column::new("a", 0)) as _; + let col_b = Arc::new(Column::new("b", 1)) as _; + let col_c = Arc::new(Column::new("c", 2)) as _; + let col_x = Arc::new(Column::new("x", 3)) as _; + let col_y = Arc::new(Column::new("y", 4)) as _; // a and b are aliases - eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_a), Arc::clone(&col_b))?; assert_eq!(eq_properties.eq_group().len(), 1); // This new entry is redundant, size shouldn't increase - eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_a))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 2); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); // b and c are aliases. Existing equivalence class should expand, // however there shouldn't be any new equivalence class - eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_c))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 3); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); + assert!(eq_groups.contains(&col_c)); // This is a new set of equality. Hence equivalent class count should be 2. - eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_x), Arc::clone(&col_y))?; assert_eq!(eq_properties.eq_group().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. - eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_x), Arc::clone(&col_a))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 5); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); - assert!(eq_groups.contains(&col_x_expr)); - assert!(eq_groups.contains(&col_y_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); + assert!(eq_groups.contains(&col_c)); + assert!(eq_groups.contains(&col_x)); + assert!(eq_groups.contains(&col_y)); Ok(()) } @@ -327,7 +327,7 @@ pub fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { let col_f = &col("f", &test_schema)?; let col_g = &col("g", &test_schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); - eq_properties.add_equal_conditions(col_a, col_c)?; + eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_c))?; let option_asc = SortOptions { descending: false, diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index b47e6cd5aae56..30aa1799b7670 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -79,7 +79,7 @@ mod sp_repartition_fuzz_tests { let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); // Define a and f are aliases - eq_properties.add_equal_conditions(col_a, col_f)?; + eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_f))?; // Column e has constant value. eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 56887da3c6d28..f035d266c015e 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -712,7 +712,7 @@ impl MemorySourceConfig { sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() - .into_inner(); + .take(); } self.sort_information = sort_information; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 94e12436d14cb..f54ab74400dbc 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -87,6 +87,7 @@ impl ConstExpr { let mut result = ConstExpr::from(expr); // Override the across partitions specification if the expression is not // a literal. + // TODO: Consider dropping across partitions from the constructor. if result.across_partitions == AcrossPartitions::Heterogeneous { result.across_partitions = across_partitions; } @@ -156,16 +157,6 @@ impl From> for ConstExpr { } } -/// Checks whether `expr` is among in the `const_exprs`. -pub fn const_exprs_contains( - const_exprs: &[ConstExpr], - expr: &Arc, -) -> bool { - const_exprs - .iter() - .any(|const_expr| const_expr.expr.eq(expr)) -} - /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates (e.g. `a = b`), typically equi-join conditions and @@ -177,15 +168,20 @@ pub fn const_exprs_contains( pub struct EquivalenceClass { /// The expressions in this equivalence class. The order doesn't matter for /// equivalence purposes. - exprs: IndexSet>, + pub(crate) exprs: IndexSet>, + /// Indicates whether the expressions in this equivalence class have a + /// constant value. A `Some` value indicates constant-ness. + pub(crate) constant: Option, } impl EquivalenceClass { // Create a new equivalence class from a pre-existing collection. pub fn new(exprs: impl IntoIterator>) -> Self { - Self { - exprs: exprs.into_iter().collect(), + let mut class = Self::default(); + for expr in exprs { + class.push(expr); } + class } /// Return the "canonical" expression for this class (the first element) @@ -197,12 +193,34 @@ impl EquivalenceClass { /// Insert the expression into this class, meaning it is known to be equal to /// all other expressions in this class. pub fn push(&mut self, expr: Arc) { + if let Some(lit) = expr.as_any().downcast_ref::() { + let expr_across = AcrossPartitions::Uniform(Some(lit.value().clone())); + if let Some(across) = self.constant.as_mut() { + // TODO: Return an error if constant values do not agree. + if *across == AcrossPartitions::Heterogeneous { + *across = expr_across; + } + } else { + self.constant = Some(expr_across); + } + } self.exprs.insert(expr); } /// Inserts all the expressions from other into this class. pub fn extend(&mut self, other: Self) { self.exprs.extend(other.exprs); + match (&self.constant, &other.constant) { + (Some(across), Some(_)) => { + // TODO: Return an error if constant values do not agree. + if across == &AcrossPartitions::Heterogeneous { + self.constant = other.constant; + } + } + (Some(_), None) => {} + (None, Some(_)) => self.constant = other.constant, + (None, None) => {} + } } /// Returns true if this equivalence class contains the given expression. @@ -226,6 +244,13 @@ impl EquivalenceClass { self.exprs.is_empty() } + /// Returns whether this equivalence class is trivial, meaning that it is + /// either empty, or contains a single expression that is not a constant. + /// Such classes are not useful, and can be removed from equivalence groups. + pub fn is_trivial(&self) -> bool { + self.exprs.is_empty() || (self.exprs.len() == 1 && self.constant.is_none()) + } + /// Iterate over all elements in this class (in some arbitrary order). pub fn iter(&self) -> impl Iterator> { self.exprs.iter() @@ -254,6 +279,7 @@ impl IntoIterator for EquivalenceClass { impl Display for EquivalenceClass { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + // TODO: Annotate constants. write!(f, "[{}]", format_physical_expr_list(&self.exprs)) } } @@ -272,8 +298,10 @@ pub struct EquivalenceGroup { impl EquivalenceGroup { /// Creates an equivalence group from the given equivalence classes. - pub fn new(classes: Vec) -> Self { - let mut result = Self { classes }; + pub fn new(classes: impl IntoIterator) -> Self { + let mut result = Self { + classes: classes.into_iter().collect(), + }; result.remove_redundant_entries(); result } @@ -293,13 +321,70 @@ impl EquivalenceGroup { self.classes.iter() } + /// Returns an iterator over the equivalence classes in this group. + pub fn iter_mut(&mut self) -> impl Iterator { + self.classes.iter_mut() + } + + /// Adds `expr` as a constant expression to this equivalence group. + pub fn add_constant(&mut self, const_expr: ConstExpr) { + for cls in self.classes.iter_mut() { + if cls.contains(&const_expr.expr) { + // If the expression is already in an equivalence class, we + // should adjust the constant-ness of the class if necessary: + if let Some(across) = cls.constant.as_mut() { + // TODO: Return an error if constant values do not agree. + if *across == AcrossPartitions::Heterogeneous { + *across = const_expr.across_partitions; + } + } else { + cls.constant = Some(const_expr.across_partitions); + } + return; + } else if let Some(across @ AcrossPartitions::Uniform(_)) = &cls.constant { + // If the expression is not in some equivalence class, but has + // the same constant value with it, add it to that class: + if const_expr.across_partitions.eq(across) { + cls.push(const_expr.expr); + return; + } + } + } + // If the expression is not in *any* equivalence class, create a new + // one with the expression as the only member: + let mut new_class = EquivalenceClass::new(std::iter::once(const_expr.expr)); + if new_class.constant.is_none() { + new_class.constant = Some(const_expr.across_partitions); + } + self.classes.push(new_class); + } + + /// Removes constant expressions that may change across partitions. + /// This method should be used when merging data from different partitions. + pub fn clear_per_partition_constants(&mut self) { + let mut idx = 0; + while idx < self.classes.len() { + let cls = &mut self.classes[idx]; + if let Some(AcrossPartitions::Heterogeneous) = cls.constant { + if cls.len() == 1 { + // If this class becomes trivial, remove it entirely: + self.classes.swap_remove(idx); + continue; + } else { + cls.constant = None; + } + } + idx += 1; + } + } + /// Adds the equality `left` = `right` to this equivalence group. /// New equality conditions often arise after steps like `Filter(a = b)`, /// `Alias(a, a as b)` etc. pub fn add_equal_conditions( &mut self, - left: &Arc, - right: &Arc, + left: Arc, + right: Arc, ) { let mut idx = 0; let size = self.classes.len(); @@ -307,10 +392,10 @@ impl EquivalenceGroup { let mut second_class = None; while (idx < size) && (first_class.is_none() || second_class.is_none()) { let cls = &self.classes[idx]; - if first_class.is_none() && cls.contains(left) { + if first_class.is_none() && cls.contains(&left) { first_class = Some(idx); } - if second_class.is_none() && cls.contains(right) { + if second_class.is_none() && cls.contains(&right) { second_class = Some(idx); } idx += 1; @@ -333,19 +418,16 @@ impl EquivalenceGroup { } (Some(group_idx), None) => { // Right side is new, extend left side's class: - self.classes[group_idx].push(Arc::clone(right)); + self.classes[group_idx].push(right); } (None, Some(group_idx)) => { // Left side is new, extend right side's class: - self.classes[group_idx].push(Arc::clone(left)); + self.classes[group_idx].push(left); } (None, None) => { // None of the expressions is among existing classes. // Create a new equivalence class and extend the group. - self.classes.push(EquivalenceClass::new(vec![ - Arc::clone(left), - Arc::clone(right), - ])); + self.classes.push(EquivalenceClass::new([left, right])); } } } @@ -353,11 +435,7 @@ impl EquivalenceGroup { /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { // Remove duplicate entries from each equivalence class: - self.classes.retain_mut(|cls| { - // Keep groups that have at least two entries as singleton class is - // meaningless (i.e. it contains no non-trivial information): - cls.len() > 1 - }); + self.classes.retain(|cls| !cls.is_trivial()); // Unify/bridge groups that have common expressions: self.bridge_classes() } @@ -441,6 +519,9 @@ impl EquivalenceGroup { /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. + /// + /// TODO: Write a multiple `expr` version to avoid searching for equivalence + /// classes for every source expression in `mapping` multiple times. pub fn project_expr( &self, mapping: &ProjectionMapping, @@ -458,10 +539,8 @@ impl EquivalenceGroup { // If we match an equivalent expression to `source`, then we can // project. For example, if we have the mapping `(a as a1, a + c)` // and the equivalence class `(a, b)`, expression `b` projects to `a1`. - if self - .get_equivalence_class(source) - .is_some_and(|group| group.contains(expr)) - { + let eq_class = self.get_equivalence_class(source); + if eq_class.is_some_and(|group| group.contains(expr)) { return Some(Arc::clone(target)); } } @@ -481,16 +560,16 @@ impl EquivalenceGroup { /// Projects this equivalence group according to the given projection mapping. pub fn project(&self, mapping: &ProjectionMapping) -> Self { - let projected_classes = self.iter().filter_map(|cls| { - let new_class = cls + let projected_classes = self.iter().map(|cls| { + let new_exprs = cls .iter() - .filter_map(|expr| self.project_expr(mapping, expr)) - .collect::>(); - (new_class.len() > 1).then_some(EquivalenceClass::new(new_class)) + .filter_map(|expr| self.project_expr(mapping, expr)); + EquivalenceClass::new(new_exprs) }); // The key is the source expression, and the value is the equivalence // class that contains the corresponding target expression. + let mut new_constants = vec![]; let mut new_classes = IndexMap::<_, EquivalenceClass>::new(); for (source, target) in mapping.iter() { // We need to find equivalent projected expressions. For example, @@ -503,20 +582,57 @@ impl EquivalenceGroup { .entry(normalized_expr) .or_default() .push(Arc::clone(target)); + // Save new constants arising from the projection: + if let Some(across) = self.is_expr_constant(source) { + let const_expr = ConstExpr::new(Arc::clone(target), across); + new_constants.push(const_expr); + } } - // Only add equivalence classes with at least two members as singleton - // equivalence classes are meaningless. - let new_classes = new_classes - .into_iter() - .filter_map(|(_, cls)| (cls.len() > 1).then_some(cls)); - let classes = projected_classes.chain(new_classes).collect(); - Self::new(classes) + // Union projected classes with new classes to make up the result: + let classes = projected_classes + .chain(new_classes.into_values()) + .filter(|cls| !cls.is_trivial()); + let mut result = Self::new(classes); + // Add new constants arising from the projection to the equivalence group: + for constant in new_constants { + result.add_constant(constant); + } + result + } + + /// Returns a `Some` value if the expression is constant according to + /// equivalence group, and `None` otherwise. The `Some` variant contains + /// an `AcrossPartitions` value indicating whether the expression is + /// constant across partitions, and its actual value (if available). + pub fn is_expr_constant( + &self, + expr: &Arc, + ) -> Option { + if let Some(lit) = expr.as_any().downcast_ref::() { + return Some(AcrossPartitions::Uniform(Some(lit.value().clone()))); + } + for cls in self.iter() { + if cls.constant.is_some() && cls.contains(expr) { + return cls.constant.clone(); + } + } + // TODO: This function should be able to return values of non-literal + // complex constants as well; e.g. it should return `8` for the + // expression `3 + 5`, not an unknown `heterogenous` value. + let children = expr.children(); + if children.is_empty() { + return None; + } + for child in children { + self.is_expr_constant(child)?; + } + Some(AcrossPartitions::Heterogeneous) } /// Returns the equivalence class containing `expr`. If no equivalence class /// contains `expr`, returns `None`. - fn get_equivalence_class( + pub fn get_equivalence_class( &self, expr: &Arc, ) -> Option<&EquivalenceClass> { @@ -534,14 +650,11 @@ impl EquivalenceGroup { match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { let mut result = Self::new( - self.iter() - .cloned() - .chain( - right_equivalences - .iter() - .map(|cls| cls.with_offset(left_size)), - ) - .collect(), + self.iter().cloned().chain( + right_equivalences + .iter() + .map(|cls| cls.with_offset(left_size)), + ), ); // In we have an inner join, expressions in the "on" condition // are equal in the resulting table. @@ -565,7 +678,7 @@ impl EquivalenceGroup { }) .data() .unwrap(); - result.add_equal_conditions(&new_lhs, &new_rhs); + result.add_equal_conditions(new_lhs, new_rhs); } } result @@ -575,9 +688,10 @@ impl EquivalenceGroup { } } - /// Checks if two expressions are equal either directly or through equivalence classes. - /// For complex expressions (e.g. a + b), checks that the expression trees are structurally - /// identical and their leaf nodes are equivalent either directly or through equivalence classes. + /// Checks if two expressions are equal directly or through equivalence + /// classes. For complex expressions (e.g. `a + b`), checks that the + /// expression trees are structurally identical and their leaf nodes are + /// equivalent either directly or through equivalence classes. pub fn exprs_equal( &self, left: &Arc, @@ -627,11 +741,6 @@ impl EquivalenceGroup { .zip(right_children) .all(|(left_child, right_child)| self.exprs_equal(left_child, right_child)) } - - /// Return the inner classes of this equivalence group. - pub fn into_inner(self) -> Vec { - self.classes - } } impl IntoIterator for EquivalenceGroup { @@ -687,12 +796,22 @@ mod tests { for (entries, expected) in test_cases { let entries = entries .into_iter() - .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(|entry| { + entry.into_iter().map(|idx| { + let c = Column::new(format!("col_{}", idx).as_str(), idx); + Arc::new(c) as _ + }) + }) .map(EquivalenceClass::new) .collect::>(); let expected = expected .into_iter() - .map(|entry| entry.into_iter().map(lit).collect::>()) + .map(|entry| { + entry.into_iter().map(|idx| { + let c = Column::new(format!("col_{}", idx).as_str(), idx); + Arc::new(c) as _ + }) + }) .map(EquivalenceClass::new) .collect::>(); let mut eq_groups = EquivalenceGroup::new(entries.clone()); @@ -712,58 +831,46 @@ mod tests { #[test] fn test_remove_redundant_entries_eq_group() -> Result<()> { + let c = |idx| Arc::new(Column::new(format!("col_{}", idx).as_str(), idx)) as _; let entries = [ - EquivalenceClass::new(vec![lit(1), lit(1), lit(2)]), - // This group is meaningless should be removed - EquivalenceClass::new(vec![lit(3), lit(3)]), - EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), + EquivalenceClass::new([c(1), c(1), lit(20)]), + EquivalenceClass::new([lit(30), lit(30)]), + EquivalenceClass::new([c(2), c(3), c(4)]), ]; // Given equivalences classes are not in succinct form. // Expected form is the most plain representation that is functionally same. let expected = [ - EquivalenceClass::new(vec![lit(1), lit(2)]), - EquivalenceClass::new(vec![lit(4), lit(5), lit(6)]), + EquivalenceClass::new([c(1), lit(20)]), + EquivalenceClass::new([lit(30)]), + EquivalenceClass::new([c(2), c(3), c(4)]), ]; - let mut eq_groups = EquivalenceGroup::new(entries.to_vec()); + let mut eq_groups = EquivalenceGroup::new(entries); eq_groups.remove_redundant_entries(); - - let eq_groups = eq_groups.classes; - assert_eq!(eq_groups.len(), expected.len()); - assert_eq!(eq_groups.len(), 2); - - assert_eq!(eq_groups[0], expected[0]); - assert_eq!(eq_groups[1], expected[1]); + assert_eq!(eq_groups.classes, expected); Ok(()) } #[test] fn test_schema_normalize_expr_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); + let col_a = Arc::new(Column::new("a", 0)) as Arc; + let col_b = Arc::new(Column::new("b", 1)) as _; + let col_c = Arc::new(Column::new("c", 2)) as _; // Assume that column a and c are aliases. - let (_test_schema, eq_properties) = create_test_params()?; - - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; - let col_c_expr = Arc::new(col_c.clone()) as Arc; - // Test cases for equivalence normalization, - // First entry in the tuple is argument, second entry is expected result after normalization. + let (_, eq_properties) = create_test_params()?; + // Test cases for equivalence normalization. First entry in the tuple is + // the argument, second entry is expected result after normalization. let expressions = vec![ // Normalized version of the column a and c should go to a // (by convention all the expressions inside equivalence class are mapped to the first entry // in this case a is the first entry in the equivalence class.) - (&col_a_expr, &col_a_expr), - (&col_c_expr, &col_a_expr), + (Arc::clone(&col_a), Arc::clone(&col_a)), + (col_c, col_a), // Cannot normalize column b - (&col_b_expr, &col_b_expr), + (Arc::clone(&col_b), Arc::clone(&col_b)), ]; let eq_group = eq_properties.eq_group(); for (expr, expected_eq) in expressions { - assert!( - expected_eq.eq(&eq_group.normalize_expr(Arc::clone(expr))), - "error in test: expr: {expr:?}" - ); + assert!(expected_eq.eq(&eq_group.normalize_expr(expr))); } Ok(()) @@ -771,21 +878,15 @@ mod tests { #[test] fn test_contains_any() { - let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) - as Arc; - let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) - as Arc; - let lit2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; - let lit1 = - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - - let cls1 = - EquivalenceClass::new(vec![Arc::clone(&lit_true), Arc::clone(&lit_false)]); - let cls2 = - EquivalenceClass::new(vec![Arc::clone(&lit_true), Arc::clone(&col_b_expr)]); - let cls3 = EquivalenceClass::new(vec![Arc::clone(&lit2), Arc::clone(&lit1)]); + let lit_true = Arc::new(Literal::new(ScalarValue::from(true))) as _; + let lit_false = Arc::new(Literal::new(ScalarValue::from(false))) as _; + let col_a_expr = Arc::new(Column::new("a", 0)) as _; + let col_b_expr = Arc::new(Column::new("b", 1)) as _; + let col_c_expr = Arc::new(Column::new("c", 2)) as _; + + let cls1 = EquivalenceClass::new([Arc::clone(&lit_true), col_a_expr]); + let cls2 = EquivalenceClass::new([lit_true, col_b_expr]); + let cls3 = EquivalenceClass::new([col_c_expr, lit_false]); // lit_true is common assert!(cls1.contains_any(&cls2)); @@ -804,21 +905,19 @@ mod tests { } // Create test columns - let col_a = Arc::new(Column::new("a", 0)) as Arc; - let col_b = Arc::new(Column::new("b", 1)) as Arc; - let col_x = Arc::new(Column::new("x", 2)) as Arc; - let col_y = Arc::new(Column::new("y", 3)) as Arc; + let col_a = Arc::new(Column::new("a", 0)) as _; + let col_b = Arc::new(Column::new("b", 1)) as _; + let col_x = Arc::new(Column::new("x", 2)) as _; + let col_y = Arc::new(Column::new("y", 3)) as _; // Create test literals - let lit_1 = - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let lit_2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit_1 = Arc::new(Literal::new(ScalarValue::from(1))) as _; + let lit_2 = Arc::new(Literal::new(ScalarValue::from(2))) as _; // Create equivalence group with classes (a = x) and (b = y) - let eq_group = EquivalenceGroup::new(vec![ - EquivalenceClass::new(vec![Arc::clone(&col_a), Arc::clone(&col_x)]), - EquivalenceClass::new(vec![Arc::clone(&col_b), Arc::clone(&col_y)]), + let eq_group = EquivalenceGroup::new([ + EquivalenceClass::new([Arc::clone(&col_a), Arc::clone(&col_x)]), + EquivalenceClass::new([Arc::clone(&col_b), Arc::clone(&col_y)]), ]); let test_cases = vec![ @@ -868,12 +967,12 @@ mod tests { Arc::clone(&col_a), Operator::Plus, Arc::clone(&col_b), - )) as Arc, + )) as _, right: Arc::new(BinaryExpr::new( Arc::clone(&col_x), Operator::Plus, Arc::clone(&col_y), - )) as Arc, + )) as _, expected: true, description: "Binary expressions with equivalent operands should be equal", @@ -883,12 +982,12 @@ mod tests { Arc::clone(&col_a), Operator::Plus, Arc::clone(&col_b), - )) as Arc, + )) as _, right: Arc::new(BinaryExpr::new( Arc::clone(&col_x), Operator::Plus, Arc::clone(&col_a), - )) as Arc, + )) as _, expected: false, description: "Binary expressions with non-equivalent operands should not be equal", @@ -898,12 +997,12 @@ mod tests { Arc::clone(&col_a), Operator::Plus, Arc::clone(&lit_1), - )) as Arc, + )) as _, right: Arc::new(BinaryExpr::new( Arc::clone(&col_x), Operator::Plus, Arc::clone(&lit_1), - )) as Arc, + )) as _, expected: true, description: "Binary expressions with equivalent column and same literal should be equal", }, @@ -916,7 +1015,7 @@ mod tests { )), Operator::Multiply, Arc::clone(&lit_1), - )) as Arc, + )) as _, right: Arc::new(BinaryExpr::new( Arc::new(BinaryExpr::new( Arc::clone(&col_x), @@ -925,7 +1024,7 @@ mod tests { )), Operator::Multiply, Arc::clone(&lit_1), - )) as Arc, + )) as _, expected: true, description: "Nested binary expressions with equivalent operands should be equal", }, @@ -961,7 +1060,7 @@ mod tests { Field::new("c", DataType::Int32, false), ])); let mut group = EquivalenceGroup::default(); - group.add_equal_conditions(&col("a", &schema)?, &col("b", &schema)?); + group.add_equal_conditions(col("a", &schema)?, col("b", &schema)?); let projected_schema = Arc::new(Schema::new(vec![ Field::new("a+c", DataType::Int32, false), diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 75f0d77db7c5b..98f4e977c423a 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -170,15 +170,15 @@ mod tests { /// Column [a=c] (e.g they are aliases). pub fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { let test_schema = create_test_schema()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let col_f = &col("f", &test_schema)?; - let col_g = &col("g", &test_schema)?; + let col_a = col("a", &test_schema)?; + let col_b = col("b", &test_schema)?; + let col_c = col("c", &test_schema)?; + let col_d = col("d", &test_schema)?; + let col_e = col("e", &test_schema)?; + let col_f = col("f", &test_schema)?; + let col_g = col("g", &test_schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); - eq_properties.add_equal_conditions(col_a, col_c)?; + eq_properties.add_equal_conditions(Arc::clone(&col_a), Arc::clone(&col_c))?; let option_asc = SortOptions { descending: false, @@ -227,49 +227,49 @@ mod tests { ])); let mut eq_properties = EquivalenceProperties::new(schema); - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; - let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; - let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + let col_a = Arc::new(Column::new("a", 0)) as _; + let col_b = Arc::new(Column::new("b", 1)) as _; + let col_c = Arc::new(Column::new("c", 2)) as _; + let col_x = Arc::new(Column::new("x", 3)) as _; + let col_y = Arc::new(Column::new("y", 4)) as _; // a and b are aliases - eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_a), Arc::clone(&col_b))?; assert_eq!(eq_properties.eq_group().len(), 1); // This new entry is redundant, size shouldn't increase - eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_a))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 2); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); // b and c are aliases. Existing equivalence class should expand, // however there shouldn't be any new equivalence class - eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_c))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 3); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); + assert!(eq_groups.contains(&col_c)); // This is a new set of equality. Hence equivalent class count should be 2. - eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_x), Arc::clone(&col_y))?; assert_eq!(eq_properties.eq_group().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. - eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr)?; + eq_properties.add_equal_conditions(Arc::clone(&col_x), Arc::clone(&col_a))?; assert_eq!(eq_properties.eq_group().len(), 1); let eq_groups = eq_properties.eq_group().iter().next().unwrap(); assert_eq!(eq_groups.len(), 5); - assert!(eq_groups.contains(&col_a_expr)); - assert!(eq_groups.contains(&col_b_expr)); - assert!(eq_groups.contains(&col_c_expr)); - assert!(eq_groups.contains(&col_x_expr)); - assert!(eq_groups.contains(&col_y_expr)); + assert!(eq_groups.contains(&col_a)); + assert!(eq_groups.contains(&col_b)); + assert!(eq_groups.contains(&col_c)); + assert!(eq_groups.contains(&col_x)); + assert!(eq_groups.contains(&col_y)); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 876adaa6be295..fa175b8675128 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -60,8 +60,9 @@ impl OrderingEquivalenceClass { result } - /// Converts this OrderingEquivalenceClass to a vector of orderings. - pub fn into_inner(self) -> Vec { + /// Takes ownership of the vector of orderings comprising this equivalence + /// class. + pub fn take(self) -> Vec { self.orderings } @@ -647,14 +648,10 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); let orderings = convert_to_orderings(&orderings); eq_properties.add_new_orderings(orderings); - let eq_group = eq_group + let classes = eq_group .into_iter() - .map(|eq_class| { - let eq_classes = eq_class.into_iter().cloned().collect::>(); - EquivalenceClass::new(eq_classes) - }) - .collect::>(); - let eq_group = EquivalenceGroup::new(eq_group); + .map(|eq_class| EquivalenceClass::new(eq_class.into_iter().cloned())); + let eq_group = EquivalenceGroup::new(classes); eq_properties.add_equivalence_group(eq_group); let constants = constants.into_iter().map(|expr| { @@ -689,7 +686,7 @@ mod tests { }; // a=c (e.g they are aliases). let mut eq_properties = EquivalenceProperties::new(test_schema); - eq_properties.add_equal_conditions(col_a, col_c)?; + eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_c))?; let orderings = vec![ vec![(col_a, options)], diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 2215fcbfcb730..895f632d93304 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -953,7 +953,7 @@ mod tests { for (orderings, equal_columns, expected) in test_cases { let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); for (lhs, rhs) in equal_columns { - eq_properties.add_equal_conditions(lhs, rhs)?; + eq_properties.add_equal_conditions(Arc::clone(lhs), Arc::clone(rhs))?; } let orderings = convert_to_orderings(&orderings); diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index b557f14136b77..0cea1767cf19d 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -537,8 +537,6 @@ mod tests { #[test] fn test_normalize_ordering_equivalence_classes() -> Result<()> { - let sort_options = SortOptions::default(); - let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -549,28 +547,16 @@ mod tests { let col_c_expr = col("c", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr)?; + eq_properties.add_equal_conditions(col_a_expr, Arc::clone(&col_c_expr))?; eq_properties.add_new_orderings([ - vec![PhysicalSortExpr { - expr: Arc::clone(&col_b_expr), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: Arc::clone(&col_c_expr), - options: sort_options, - }], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], ]); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_new_orderings([ - vec![PhysicalSortExpr { - expr: Arc::clone(&col_b_expr), - options: sort_options, - }], - vec![PhysicalSortExpr { - expr: Arc::clone(&col_c_expr), - options: sort_options, - }], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], ]); let oeq_class = eq_properties.oeq_class().clone(); @@ -711,7 +697,7 @@ mod tests { nulls_first: false, }; // b=a (e.g they are aliases) - eq_properties.add_equal_conditions(col_b, col_a)?; + eq_properties.add_equal_conditions(Arc::clone(col_b), Arc::clone(col_a))?; // [b ASC], [d ASC] eq_properties.add_new_orderings([ vec![PhysicalSortExpr { @@ -1097,7 +1083,7 @@ mod tests { // Equal conditions before constants { let mut properties = base_properties.clone(); - for [left, right] in &case.equal_conditions { + for [left, right] in case.equal_conditions.clone() { properties.add_equal_conditions(left, right)? } properties.add_constants( @@ -1111,7 +1097,7 @@ mod tests { properties.add_constants( case.constants.iter().cloned().map(ConstExpr::from), ); - for [left, right] in &case.equal_conditions { + for [left, right] in case.equal_conditions { properties.add_equal_conditions(left, right)? } properties @@ -1164,15 +1150,14 @@ mod tests { ]); // Add equality condition c = concat(a, b) - eq_properties.add_equal_conditions(&col_c, &a_concat_b)?; + eq_properties.add_equal_conditions(Arc::clone(&col_c), a_concat_b)?; let orderings = eq_properties.oeq_class(); - let expected_ordering1 = - [PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc()].into(); + let expected_ordering1 = [PhysicalSortExpr::new_default(col_c).asc()].into(); let expected_ordering2 = [ - PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), - PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), + PhysicalSortExpr::new_default(col_a).asc(), + PhysicalSortExpr::new_default(col_b).asc(), ] .into(); @@ -1215,7 +1200,7 @@ mod tests { eq_properties.add_new_ordering(initial_ordering.clone()); // Add equality condition c = a * b - eq_properties.add_equal_conditions(&col_c, &a_times_b)?; + eq_properties.add_equal_conditions(col_c, a_times_b)?; let orderings = eq_properties.oeq_class(); @@ -1255,7 +1240,7 @@ mod tests { ]); // Add equality condition c = concat(a, b) - eq_properties.add_equal_conditions(&col_c, &a_concat_b)?; + eq_properties.add_equal_conditions(col_c, Arc::clone(&a_concat_b))?; let orderings = eq_properties.oeq_class(); @@ -1398,7 +1383,7 @@ mod tests { let col_c = col("c", &schema)?; // Make a and b equivalent - eq_properties.add_equal_conditions(&col_a, &col_b)?; + eq_properties.add_equal_conditions(Arc::clone(&col_a), Arc::clone(&col_b))?; let asc = SortOptions::default(); @@ -1495,7 +1480,7 @@ mod tests { eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_c))]); // Equality: b = d - eq_properties.add_equal_conditions(&col_b, &col_d)?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_d))?; // Orderings: [d ASC, a ASC], [e ASC] eq_properties.add_new_orderings([ diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 5d72fd5ff9be2..d415198fede2c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -41,12 +41,10 @@ pub fn join_equivalence_properties( )); let EquivalenceProperties { - constants: left_constants, oeq_class: left_oeq_class, .. } = left; let EquivalenceProperties { - constants: right_constants, oeq_class: mut right_oeq_class, .. } = right; @@ -102,15 +100,6 @@ pub fn join_equivalence_properties( [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), } - match join_type { - JoinType::LeftAnti | JoinType::LeftSemi => { - result.add_constants(left_constants); - } - JoinType::RightAnti | JoinType::RightSemi => { - result.add_constants(right_constants); - } - _ => {} - } result } @@ -265,17 +254,17 @@ mod tests { // Join Schema let schema = Schema::new(fields); - let col_a = &col("a", &schema)?; - let col_d = &col("d", &schema)?; - let col_x = &col("x", &schema)?; - let col_y = &col("y", &schema)?; - let col_z = &col("z", &schema)?; - let col_w = &col("w", &schema)?; + let col_a = col("a", &schema)?; + let col_d = col("d", &schema)?; + let col_x = col("x", &schema)?; + let col_y = col("y", &schema)?; + let col_z = col("z", &schema)?; + let col_w = col("w", &schema)?; let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); // a=x and d=w - join_eq_properties.add_equal_conditions(col_a, col_x)?; - join_eq_properties.add_equal_conditions(col_d, col_w)?; + join_eq_properties.add_equal_conditions(col_a, Arc::clone(&col_x))?; + join_eq_properties.add_equal_conditions(col_d, Arc::clone(&col_w))?; updated_right_ordering_equivalence_class( &mut right_oeq_class, diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 2554f6fd25012..65e996bd0bb37 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -32,14 +32,12 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use std::{fmt, mem}; -use crate::equivalence::class::{const_exprs_contains, AcrossPartitions}; -use crate::equivalence::{ - EquivalenceClass, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, -}; +use crate::equivalence::class::AcrossPartitions; +use crate::equivalence::{EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping}; use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; use crate::{ - physical_exprs_contains, ConstExpr, LexOrdering, LexRequirement, PhysicalExpr, - PhysicalSortExpr, PhysicalSortRequirement, + ConstExpr, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; @@ -139,12 +137,6 @@ pub struct EquivalenceProperties { eq_group: EquivalenceGroup, /// Equivalent sort expressions oeq_class: OrderingEquivalenceClass, - /// Expressions whose values are constant. These expressions are in - /// normalized form (w.r.t. the equivalence group). - /// - /// TODO: We do not need to track constants separately, they can be tracked - /// inside `eq_group` as `Literal` expressions. - constants: Vec, /// Table constraints constraints: Constraints, /// Schema associated with this object. @@ -157,7 +149,6 @@ impl EquivalenceProperties { Self { eq_group: EquivalenceGroup::default(), oeq_class: OrderingEquivalenceClass::default(), - constants: vec![], constraints: Constraints::default(), schema, } @@ -178,7 +169,6 @@ impl EquivalenceProperties { Self { eq_group: EquivalenceGroup::default(), oeq_class: OrderingEquivalenceClass::new(orderings), - constants: vec![], constraints: Constraints::default(), schema, } @@ -205,8 +195,17 @@ impl EquivalenceProperties { } /// Returns a reference to the constants within. - pub fn constants(&self) -> &[ConstExpr] { - &self.constants + pub fn constants(&self) -> Vec { + self.eq_group + .iter() + .filter_map(|c| { + c.canonical_expr().cloned().and_then(|expr| { + c.constant + .as_ref() + .map(|across| ConstExpr::new(expr, across.clone())) + }) + }) + .collect() } /// Returns a reference to the constraints within. @@ -216,9 +215,14 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - // Prune out constant expressions: let mut sort_exprs = self.oeq_class().output_ordering()?.take(); - sort_exprs.retain(|item| !const_exprs_contains(&self.constants, &item.expr)); + // Prune out constant expressions: + sort_exprs.retain(|sort_expr| { + let Some(cls) = self.eq_group.get_equivalence_class(&sort_expr.expr) else { + return true; + }; + cls.constant.is_none() + }); LexOrdering::new(sort_exprs) } @@ -239,7 +243,6 @@ impl EquivalenceProperties { pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); - self.add_constants(other.constants); self } @@ -250,10 +253,9 @@ impl EquivalenceProperties { } /// Removes constant expressions that may change across partitions. - /// This method should be used when data from different partitions are merged. + /// This method should be used when merging data from different partitions. pub fn clear_per_partition_constants(&mut self) { - self.constants - .retain(|item| matches!(item.across_partitions, AcrossPartitions::Uniform(_))) + self.eq_group.clear_per_partition_constants(); } /// Extends this `EquivalenceProperties` by adding the orderings inside the @@ -289,50 +291,27 @@ impl EquivalenceProperties { /// equivalence class to the equivalence group. pub fn add_equal_conditions( &mut self, - left: &Arc, - right: &Arc, + left: Arc, + right: Arc, ) -> Result<()> { - // Discover new constants in light of new the equality: - if self.is_expr_constant(left) { - // Left expression is constant, add right as constant - if !const_exprs_contains(&self.constants, right) { - let across_parts = self.get_expr_constant_value(left); - let const_expr = ConstExpr::new(Arc::clone(right), across_parts); - self.constants.push(const_expr); - } - } else if self.is_expr_constant(right) { - // Right expression is constant, add left as constant - if !const_exprs_contains(&self.constants, left) { - let across_parts = self.get_expr_constant_value(right); - let const_expr = ConstExpr::new(Arc::clone(left), across_parts); - self.constants.push(const_expr); - } - } - - // Add equal expressions to the state - self.eq_group.add_equal_conditions(left, right); - - // Discover any new orderings - self.discover_new_orderings(left)?; - Ok(()) + // Add equal expressions to the state: + self.eq_group.add_equal_conditions(Arc::clone(&left), right); + // Discover any new orderings: + self.discover_new_orderings(left) } /// Track/register physical expressions with constant values. pub fn add_constants(&mut self, constants: impl IntoIterator) { - let normalized_constants = constants - .into_iter() - .filter_map(|mut c| { - c.expr = self.eq_group.normalize_expr(c.expr); - (!const_exprs_contains(&self.constants, &c.expr)).then_some(c) - }) - .collect::>(); - - // Add all new normalized constants - self.constants.extend(normalized_constants); - + let c = constants.into_iter().collect::>(); + let constants = c.into_iter(); + // Add the new constant to the equivalence group: + for constant in constants { + self.eq_group.add_constant(constant); + } // Discover any new orderings based on the constants for ordering in self.normalized_oeq_class().iter() { - self.discover_new_orderings(&ordering[0].expr).unwrap(); + self.discover_new_orderings(Arc::clone(&ordering[0].expr)) + .unwrap(); } } @@ -342,8 +321,8 @@ impl EquivalenceProperties { // When constants or equivalence classes are changed, there may be new orderings // that can be discovered with the new equivalence properties. // For a discussion, see: https://github.com/apache/datafusion/issues/9812 - fn discover_new_orderings(&mut self, expr: &Arc) -> Result<()> { - let normalized_expr = self.eq_group().normalize_expr(Arc::clone(expr)); + fn discover_new_orderings(&mut self, expr: Arc) -> Result<()> { + let normalized_expr = self.eq_group().normalize_expr(expr); let eq_class = self .eq_group .iter() @@ -413,7 +392,7 @@ impl EquivalenceProperties { // Filter out constant expressions as they don't affect ordering let filtered_exprs = ordering .into_iter() - .filter(|expr| !self.is_expr_constant(&expr.expr)) + .filter(|expr| self.is_expr_constant(&expr.expr).is_none()) .collect::>(); if let Some(filtered_exprs) = LexOrdering::new(filtered_exprs) { @@ -468,7 +447,7 @@ impl EquivalenceProperties { let sort_exprs = sort_exprs .into_iter() .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) - .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)) + .filter(|order| self.is_expr_constant(&order.expr).is_none()) .collect::>(); LexOrdering::new(sort_exprs).map(|o| o.collapse()) } @@ -494,8 +473,8 @@ impl EquivalenceProperties { // Prune redundant sections in the requirement: let reqs = sort_reqs .into_iter() - .map(|req| self.eq_group.normalize_sort_requirement(req.clone())) - .filter(|order| !self.is_normalized_expr_constant(&order.expr, false)); + .map(|req| self.eq_group.normalize_sort_requirement(req)) + .filter(|order| self.is_expr_constant(&order.expr).is_none()); LexRequirement::new(reqs).map(|r| r.collapse()) } @@ -1041,56 +1020,6 @@ impl EquivalenceProperties { .collect() } - /// Projects constants based on the provided `ProjectionMapping`. - /// - /// This function takes a `ProjectionMapping` and identifies/projects - /// constants based on the existing constants and the mapping. It ensures - /// that constants are appropriately propagated through the projection. - /// - /// # Parameters - /// - /// - `mapping`: A reference to a `ProjectionMapping` representing the - /// mapping of source expressions to target expressions in the projection. - /// - /// # Returns - /// - /// Returns a `Vec>` containing the projected constants. - fn projected_constants(&self, mapping: &ProjectionMapping) -> Vec { - // First, project existing constants. For example, assume that `a + b` - // is known to be constant. If the projection were `a as a_new`, `b as b_new`, - // then we would project constant `a + b` as `a_new + b_new`. - let mut projected_constants = self - .constants - .iter() - .filter_map(|const_expr| { - self.eq_group.project_expr(mapping, &const_expr.expr).map( - |projected_expr| { - ConstExpr::new( - projected_expr, - const_expr.across_partitions.clone(), - ) - }, - ) - }) - .collect::>(); - - // Add projection expressions that are known to be constant: - for (source, target) in mapping.iter() { - if self.is_expr_constant(source) - && !const_exprs_contains(&projected_constants, target) - { - let uniform = if self.is_expr_constant_across_partitions(source) { - self.get_expr_constant_value(source) - } else { - AcrossPartitions::Heterogeneous - }; - let const_expr = ConstExpr::new(Arc::clone(target), uniform); - projected_constants.push(const_expr); - } - } - projected_constants - } - /// Projects constraints according to the given projection mapping. /// /// This function takes a projection mapping and extracts the column indices of the target columns. @@ -1121,13 +1050,11 @@ impl EquivalenceProperties { pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { let eq_group = self.eq_group.project(mapping); let oeq_class = OrderingEquivalenceClass::new(self.projected_orderings(mapping)); - let constants = self.projected_constants(mapping); let constraints = self.projected_constraints(mapping).unwrap_or_default(); Self { schema: output_schema, eq_group, oeq_class, - constants, constraints, } } @@ -1214,86 +1141,15 @@ impl EquivalenceProperties { /// /// # Returns /// - /// Returns `true` if the expression is constant according to equivalence - /// group, `false` otherwise. - pub fn is_expr_constant(&self, expr: &Arc) -> bool { - let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - self.is_normalized_expr_constant(&normalized_expr, false) - } - - /// Helper of the [`Self::is_expr_constant`] function, assumes that the - /// given expression is normalized. - fn is_normalized_expr_constant( - &self, - normalized_expr: &Arc, - across_partitions: bool, - ) -> bool { - let normalized_constants = self - .constants - .iter() - .filter(|const_expr| { - !across_partitions - || matches!( - const_expr.across_partitions, - AcrossPartitions::Uniform { .. } - ) - }) - .map(|const_expr| self.eq_group.normalize_expr(Arc::clone(&const_expr.expr))) - .collect::>(); - is_constant_recurse(&normalized_constants, normalized_expr) - } - - /// This function determines whether the provided expression is constant - /// across partitions based on the known constants. For more details, see - /// [`Self::is_expr_constant`]. - /// - /// # Parameters - /// - /// - `expr`: A reference to a `Arc` representing the - /// expression to be checked. - /// - /// # Returns - /// - /// Returns `true` if the expression is constant across all partitions according - /// to equivalence group, `false` otherwise. - pub fn is_expr_constant_across_partitions( - &self, - expr: &Arc, - ) -> bool { - let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - self.is_normalized_expr_constant(&normalized_expr, true) - } - - /// Retrieves the constant value of a given physical expression, if it exists. - /// - /// Normalizes the input expression and checks if it matches any known constants - /// in the current context. Returns whether the expression has a uniform value, - /// varies across partitions, or is not constant. - /// - /// # Parameters - /// - `expr`: A reference to the physical expression to evaluate. - /// - /// # Returns - /// - `AcrossPartitions::Uniform(value)`: If the expression has the same value across partitions. - /// - `AcrossPartitions::Heterogeneous`: If the expression varies across partitions. - /// - `None`: If the expression is not recognized as constant. - pub fn get_expr_constant_value( + /// Returns a `Some` value if the expression is constant according to + /// equivalence group, and `None` otherwise. The `Some` variant contains + /// an `AcrossPartitions` value indicating whether the expression is + /// constant across partitions, and its actual value (if available). + pub fn is_expr_constant( &self, expr: &Arc, - ) -> AcrossPartitions { - let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); - - if let Some(lit) = normalized_expr.as_any().downcast_ref::() { - return AcrossPartitions::Uniform(Some(lit.value().clone())); - } - - for const_expr in self.constants.iter() { - if normalized_expr.eq(&const_expr.expr) { - return const_expr.across_partitions.clone(); - } - } - - AcrossPartitions::Heterogeneous + ) -> Option { + self.eq_group.is_expr_constant(expr) } /// Retrieves the properties for a given physical expression. @@ -1322,7 +1178,7 @@ impl EquivalenceProperties { /// Transforms this `EquivalenceProperties` into a new `EquivalenceProperties` /// by mapping columns in the original schema to columns in the new schema /// by index. - pub fn with_new_schema(self, schema: SchemaRef) -> Result { + pub fn with_new_schema(mut self, schema: SchemaRef) -> Result { // The new schema and the original schema is aligned when they have the // same number of columns, and fields at the same index have the same // type in both schemas. @@ -1337,51 +1193,48 @@ impl EquivalenceProperties { // Rewriting equivalence properties in terms of new schema is not // safe when schemas are not aligned: return plan_err!( - "Cannot rewrite old_schema:{:?} with new schema: {:?}", + "Schemas have to be aligned to rewrite equivalences:\n Old schema: {:?}\n New schema: {:?}", self.schema, schema ); } - // Rewrite constants according to new schema: - let new_constants = self - .constants - .into_iter() - .map(|mut const_expr| { - const_expr.expr = with_new_schema(const_expr.expr, &schema)?; - Ok(const_expr) - }) - .collect::>>()?; - - // Rewrite orderings according to new schema: - let mut new_orderings = vec![]; - for ordering in self.oeq_class { - let new_ordering = ordering - .into_iter() - .map(|mut sort_expr| { - sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; - Ok(sort_expr) - }) - .collect::>>()?; - new_orderings.push(new_ordering); - } // Rewrite equivalence classes according to the new schema: let mut eq_classes = vec![]; - for eq_class in self.eq_group { - let new_eq_exprs = eq_class + for mut eq_class in self.eq_group { + eq_class.exprs = eq_class + .exprs .into_iter() .map(|expr| with_new_schema(expr, &schema)) - .collect::>>()?; - eq_classes.push(EquivalenceClass::new(new_eq_exprs)); + .collect::>()?; + // TODO: Also change the data type of the constant value if it exists. + eq_classes.push(eq_class); } - // Construct the resulting equivalence properties: - let mut result = EquivalenceProperties::new(schema); - result.constants = new_constants; - result.add_new_orderings(new_orderings); - result.add_equivalence_group(EquivalenceGroup::new(eq_classes)); + // Rewrite orderings according to new schema: + let new_orderings = self + .oeq_class + .into_iter() + .map(|ordering| { + ordering + .into_iter() + .map(|mut sort_expr| { + sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; + Ok(sort_expr) + }) + .collect::>>() + // The following `unwrap` is safe because the vector will always + // be non-empty. + .map(|v| LexOrdering::new(v).unwrap()) + }) + .collect::>()?; - Ok(result) + // Update the schema, the equivalence group and the ordering equivalence + // class: + self.schema = schema; + self.eq_group = EquivalenceGroup::new(eq_classes); + self.oeq_class = OrderingEquivalenceClass::new(new_orderings); + Ok(self) } } @@ -1393,10 +1246,7 @@ impl EquivalenceProperties { /// ``` impl Display for EquivalenceProperties { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if self.eq_group.is_empty() - && self.oeq_class.is_empty() - && self.constants.is_empty() - { + if self.eq_group.is_empty() && self.oeq_class.is_empty() { return write!(f, "No properties"); } if !self.oeq_class.is_empty() { @@ -1405,9 +1255,6 @@ impl Display for EquivalenceProperties { if !self.eq_group.is_empty() { write!(f, ", eq: {}", self.eq_group)?; } - if !self.constants.is_empty() { - write!(f, ", const: [{}]", ConstExpr::format_list(&self.constants))?; - } Ok(()) } } @@ -1454,7 +1301,7 @@ fn update_properties( .eq_group .normalize_expr(Arc::clone(&node.expr)); let oeq_class = eq_properties.normalized_oeq_class(); - if eq_properties.is_normalized_expr_constant(&normalized_expr, false) + if eq_properties.is_expr_constant(&normalized_expr).is_some() || oeq_class.is_expr_partial_const(&normalized_expr) { node.data.sort_properties = SortProperties::Singleton; @@ -1464,31 +1311,6 @@ fn update_properties( Ok(Transformed::yes(node)) } -/// This function determines whether the provided expression is constant -/// based on the known constants. -/// -/// # Parameters -/// -/// - `constants`: A `&[Arc]` containing expressions known to -/// be a constant. -/// - `expr`: A reference to a `Arc` representing the expression -/// to check. -/// -/// # Returns -/// -/// Returns `true` if the expression is constant according to equivalence -/// group, `false` otherwise. -fn is_constant_recurse( - constants: &[Arc], - expr: &Arc, -) -> bool { - if physical_exprs_contains(constants, expr) || expr.as_any().is::() { - return true; - } - let children = expr.children(); - !children.is_empty() && children.iter().all(|c| is_constant_recurse(constants, c)) -} - /// This function examines whether a referring expression directly refers to a /// given referred expression or if any of its children in the expression tree /// refer to the specified expression. @@ -1581,40 +1403,3 @@ impl Hash for ExprWrapper { self.0.hash(state); } } - -#[cfg(test)] -mod tests { - use super::*; - use crate::expressions::{col, BinaryExpr}; - - use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; - use datafusion_expr::Operator; - - #[test] - fn test_expr_consists_of_constants() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Nanosecond, None), true), - ])); - let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; - let col_d = col("d", &schema)?; - let b_plus_d = Arc::new(BinaryExpr::new( - Arc::clone(&col_b), - Operator::Plus, - Arc::clone(&col_d), - )) as _; - - let constants = vec![Arc::clone(&col_a), Arc::clone(&col_b)]; - let expr = Arc::clone(&b_plus_d); - assert!(!is_constant_recurse(&constants, &expr)); - - let constants = vec![Arc::clone(&col_a), Arc::clone(&col_b), Arc::clone(&col_d)]; - let expr = Arc::clone(&b_plus_d); - assert!(is_constant_recurse(&constants, &expr)); - Ok(()) - } -} diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index ed708d690d9ed..e389740bb2416 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -15,18 +15,17 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::{internal_err, Result}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::iter::Peekable; +use std::slice::Iter; use std::sync::Arc; +use super::EquivalenceProperties; use crate::equivalence::class::AcrossPartitions; -use crate::ConstExpr; +use crate::{ConstExpr, PhysicalSortExpr}; -use super::EquivalenceProperties; -use crate::PhysicalSortExpr; use arrow::datatypes::SchemaRef; -use std::slice::Iter; +use datafusion_common::{internal_err, Result}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Calculates the union (in the sense of `UnionExec`) `EquivalenceProperties` /// of `lhs` and `rhs` according to the schema of `lhs`. @@ -48,12 +47,13 @@ fn calculate_union_binary( // First, calculate valid constants for the union. An expression is constant // at the output of the union if it is constant in both sides with matching values. + let rhs_constants = rhs.constants(); let constants = lhs .constants() - .iter() + .into_iter() .filter_map(|lhs_const| { // Find matching constant expression in RHS - rhs.constants() + rhs_constants .iter() .find(|rhs_const| rhs_const.expr.eq(&lhs_const.expr)) .map(|rhs_const| { @@ -147,10 +147,16 @@ impl UnionEquivalentOrderingBuilder { properties: &EquivalenceProperties, ) { let constants = source.constants(); + let properties_constants = properties.constants(); for mut ordering in source.normalized_oeq_class() { // Progressively shorten the ordering to search for a satisfied prefix: loop { - ordering = match self.try_add_ordering(ordering, constants, properties) { + ordering = match self.try_add_ordering( + ordering, + &constants, + properties, + &properties_constants, + ) { AddedOrdering::Yes => break, AddedOrdering::No(ordering) => { let mut sort_exprs = ordering.take(); @@ -179,13 +185,19 @@ impl UnionEquivalentOrderingBuilder { ordering: LexOrdering, constants: &[ConstExpr], properties: &EquivalenceProperties, + properties_constants: &[ConstExpr], ) -> AddedOrdering { if properties.ordering_satisfy(ordering.clone()) { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); AddedOrdering::Yes - } else if self.try_find_augmented_ordering(&ordering, constants, properties) { + } else if self.try_find_augmented_ordering( + &ordering, + constants, + properties, + properties_constants, + ) { // Augmented with constants to match the properties. AddedOrdering::Yes } else { @@ -200,6 +212,7 @@ impl UnionEquivalentOrderingBuilder { ordering: &LexOrdering, constants: &[ConstExpr], properties: &EquivalenceProperties, + properties_constants: &[ConstExpr], ) -> bool { let mut result = false; // Can only augment if there are constants. @@ -211,7 +224,7 @@ impl UnionEquivalentOrderingBuilder { ordering, constants, existing_ordering, - &properties.constants, + properties_constants, ) { self.orderings.push(augmented_ordering); result = true; @@ -302,15 +315,25 @@ fn advance_if_matches_constant( #[cfg(test)] mod tests { use super::*; - use crate::equivalence::class::const_exprs_contains; use crate::equivalence::tests::{create_test_schema, parse_sort_expr}; use crate::expressions::col; + use crate::PhysicalExpr; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::ScalarValue; use itertools::Itertools; + /// Checks whether `expr` is among in the `const_exprs`. + fn const_exprs_contains( + const_exprs: &[ConstExpr], + expr: &Arc, + ) -> bool { + const_exprs + .iter() + .any(|const_expr| const_expr.expr.eq(expr)) + } + #[test] fn test_union_equivalence_properties_multi_children_1() { let schema = create_test_schema().unwrap(); @@ -788,9 +811,9 @@ mod tests { // Check whether constants are same let lhs_constants = lhs.constants(); let rhs_constants = rhs.constants(); - for rhs_constant in rhs_constants { + for rhs_constant in &rhs_constants { assert!( - const_exprs_contains(lhs_constants, &rhs_constant.expr), + const_exprs_contains(&lhs_constants, &rhs_constant.expr), "{err_msg}\nlhs: {lhs}\nrhs: {rhs}" ); } diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index e90872a1a0a92..42c1c6b4602be 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -41,10 +41,10 @@ pub fn add_sort_above( ) -> PlanContext { let mut sort_reqs = sort_requirements.take(); sort_reqs.retain(|sort_expr| { - !node - .plan + node.plan .equivalence_properties() .is_expr_constant(&sort_expr.expr) + .is_none() }); let sort_exprs = sort_reqs.into_iter().map(Into::into).collect::>(); let Some(ordering) = LexOrdering::new(sort_exprs) else { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 9b92e5d22bb13..b20b69bf0a36a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2352,7 +2352,7 @@ mod tests { let col_c = &col("c", &test_schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); // Columns a and b are equal. - eq_properties.add_equal_conditions(col_a, col_b)?; + eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_b))?; // Aggregate requirements are // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let order_by_exprs = vec![ diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 87a072b263966..e0dd8ca720ce6 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -227,20 +227,18 @@ impl FilterExec { if let Some(binary) = conjunction.as_any().downcast_ref::() { if binary.op() == &Operator::Eq { // Filter evaluates to single value for all partitions - if input_eqs.is_expr_constant(binary.left()) { - let (expr, across_parts) = ( - binary.right(), - input_eqs.get_expr_constant_value(binary.right()), - ); + if input_eqs.is_expr_constant(binary.left()).is_some() { + let across = input_eqs + .is_expr_constant(binary.right()) + .unwrap_or_default(); res_constants - .push(ConstExpr::new(Arc::clone(expr), across_parts)); - } else if input_eqs.is_expr_constant(binary.right()) { - let (expr, across_parts) = ( - binary.left(), - input_eqs.get_expr_constant_value(binary.left()), - ); + .push(ConstExpr::new(Arc::clone(binary.right()), across)); + } else if input_eqs.is_expr_constant(binary.right()).is_some() { + let across = input_eqs + .is_expr_constant(binary.left()) + .unwrap_or_default(); res_constants - .push(ConstExpr::new(Arc::clone(expr), across_parts)); + .push(ConstExpr::new(Arc::clone(binary.left()), across)); } } } @@ -260,7 +258,7 @@ impl FilterExec { let mut eq_properties = input.equivalence_properties().clone(); let (equal_pairs, _) = collect_columns_from_predicate(predicate); for (lhs, rhs) in equal_pairs { - eq_properties.add_equal_conditions(lhs, rhs)? + eq_properties.add_equal_conditions(Arc::clone(lhs), Arc::clone(rhs))? } // Add the columns that have only one viable value (singleton) after // filtering to constants. diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index e937b4419ad4d..4ea5a6edc1b8f 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -335,7 +335,7 @@ impl TestMemoryExec { sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() - .into_inner(); + .take(); } self.sort_information = sort_information; From 00ce540ea7dcdc298087561ab2f44c8e08aa93a5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 24 Apr 2025 02:08:44 +0300 Subject: [PATCH 106/167] Resolve logical conflict --- .../tests/physical_optimizer/enforce_distribution.rs | 10 +++------- .../functions-aggregate/src/approx_percentile_cont.rs | 2 +- datafusion/sqllogictest/test_files/aggregate.slt | 3 --- 3 files changed, 4 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 403c67216976f..8f3d731c25532 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3494,16 +3494,12 @@ fn test_replace_order_preserving_variants_with_fetch() -> Result<()> { // Create a base plan let parquet_exec = parquet_exec(); - let sort_expr = PhysicalSortExpr { - expr: Arc::new(Column::new("id", 0)), - options: SortOptions::default(), - }; - - let ordering = LexOrdering::new(vec![sort_expr]); + let sort_expr = PhysicalSortExpr::new_default(Arc::new(Column::new("id", 0))); // Create a SortPreservingMergeExec with fetch=5 let spm_exec = Arc::new( - SortPreservingMergeExec::new(ordering, parquet_exec.clone()).with_fetch(Some(5)), + SortPreservingMergeExec::new([sort_expr].into(), parquet_exec.clone()) + .with_fetch(Some(5)), ); // Create distribution context diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 41281733f5deb..03381e2e800e4 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -143,7 +143,7 @@ impl ApproxPercentileCont { let percentile = validate_input_percentile_expr(&args.exprs[1])?; let is_descending = args - .ordering_req + .order_bys .first() .map(|sort_expr| sort_expr.options.descending) .unwrap_or(false); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index df0a9044371e8..bcee1af79f2eb 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -160,9 +160,6 @@ SELECT approx_percentile_cont(c12) WITHIN GROUP (ORDER BY c12) FROM aggregate_te statement error DataFusion error: This feature is not implemented: Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be a literal SELECT approx_percentile_cont(0.95, c5) WITHIN GROUP (ORDER BY c12) FROM aggregate_test_100 -statement error DataFusion error: This feature is not implemented: Conflicting ordering requirements in aggregate functions is not supported -SELECT approx_percentile_cont(0.95) WITHIN GROUP (ORDER BY c5), approx_percentile_cont(0.2) WITHIN GROUP (ORDER BY c12) FROM aggregate_test_100 - statement error DataFusion error: Error during planning: \[IGNORE | RESPECT\] NULLS are not permitted for approx_percentile_cont SELECT approx_percentile_cont(0.95) WITHIN GROUP (ORDER BY c5) IGNORE NULLS FROM aggregate_test_100 From 052120fdcdd3ff9223b7e8e6b3b51b1b0b70ef96 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 24 Apr 2025 15:34:49 +0300 Subject: [PATCH 107/167] Remove the unusual take API, instead use the from trait --- datafusion/core/tests/memory_limit/mod.rs | 10 +--- datafusion/datasource/src/memory.rs | 2 +- .../physical-expr-common/src/sort_expr.rs | 24 +++++----- .../physical-expr/src/equivalence/class.rs | 40 +++++++++------- .../physical-expr/src/equivalence/ordering.rs | 25 +++++----- .../src/equivalence/properties/dependency.rs | 15 ++---- .../src/equivalence/properties/mod.rs | 47 +++++++++---------- .../src/equivalence/properties/union.rs | 2 +- datafusion/physical-optimizer/src/utils.rs | 2 +- datafusion/physical-plan/src/test.rs | 2 +- 10 files changed, 81 insertions(+), 88 deletions(-) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index a34279e1770a5..79db412950843 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -875,14 +875,8 @@ impl Scenario { nulls_first: false, }; let sort_information = vec![[ - PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options, - }, - PhysicalSortExpr { - expr: col("b", &schema).unwrap(), - options, - }, + PhysicalSortExpr::new(col("a", &schema).unwrap(), options), + PhysicalSortExpr::new(col("b", &schema).unwrap(), options), ] .into()]; diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index f035d266c015e..ecb459a7d07a3 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -712,7 +712,7 @@ impl MemorySourceConfig { sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() - .take(); + .into(); } self.sort_information = sort_information; diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index e5617502bbcfe..e759737b498dd 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -354,12 +354,6 @@ impl LexOrdering { self.inner.capacity() } - /// Takes ownership of the underlying vector of sort expressions and - /// returns it. - pub fn take(self) -> Vec { - self.inner - } - /// Constructs a duplicate-free `LexOrdering` by filtering out duplicate /// entries that have same physical expression inside. /// @@ -460,6 +454,12 @@ impl<'a> IntoIterator for &'a LexOrdering { } } +impl From for Vec { + fn from(ordering: LexOrdering) -> Self { + ordering.inner + } +} + ///`LexRequirement` is an struct containing a `Vec`, which /// represents a lexicographical ordering requirement. #[derive(Debug, Clone, PartialEq)] @@ -491,12 +491,6 @@ impl LexRequirement { self.inner.capacity() } - /// Takes ownership of the underlying vector of sort requirements and - /// returns it. - pub fn take(self) -> Vec { - self.inner - } - /// Constructs a duplicate-free `LexRequirement` by filtering out duplicate /// entries that have same physical expression inside. /// @@ -557,6 +551,12 @@ impl<'a> IntoIterator for &'a LexRequirement { } } +impl From for Vec { + fn from(requirement: LexRequirement) -> Self { + requirement.inner + } +} + // Cross-conversion utilities between `LexOrdering` and `LexRequirement` impl From for LexRequirement { fn from(value: LexOrdering) -> Self { diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index f54ab74400dbc..6394f19411222 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -42,6 +42,21 @@ pub enum AcrossPartitions { Uniform(Option), } +impl Display for AcrossPartitions { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + AcrossPartitions::Heterogeneous => write!(f, "(heterogeneous)"), + AcrossPartitions::Uniform(value) => { + if let Some(val) = value { + write!(f, "(uniform: {})", val) + } else { + write!(f, "(uniform: unknown)") + } + } + } + } +} + /// A structure representing a expression known to be constant in a physical /// execution plan. /// @@ -124,19 +139,7 @@ impl PartialEq for ConstExpr { impl Display for ConstExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{}", self.expr)?; - match &self.across_partitions { - AcrossPartitions::Heterogeneous => { - write!(f, "(heterogeneous)")?; - } - AcrossPartitions::Uniform(value) => { - if let Some(val) = value { - write!(f, "(uniform: {})", val)?; - } else { - write!(f, "(uniform: unknown)")?; - } - } - } - Ok(()) + write!(f, "{}", self.across_partitions) } } @@ -217,9 +220,8 @@ impl EquivalenceClass { self.constant = other.constant; } } - (Some(_), None) => {} (None, Some(_)) => self.constant = other.constant, - (None, None) => {} + (_, None) => {} } } @@ -279,8 +281,12 @@ impl IntoIterator for EquivalenceClass { impl Display for EquivalenceClass { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - // TODO: Annotate constants. - write!(f, "[{}]", format_physical_expr_list(&self.exprs)) + write!(f, "{{")?; + write!(f, "members: {}", format_physical_expr_list(&self.exprs))?; + if let Some(across) = &self.constant { + write!(f, ", constant: {}", across)?; + } + write!(f, "}}") } } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index fa175b8675128..3f09270988ab7 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -51,21 +51,18 @@ impl OrderingEquivalenceClass { self.orderings.clear(); } - /// Creates new ordering equivalence class from the given orderings - /// - /// Any redundant entries are removed - pub fn new(orderings: Vec) -> Self { - let mut result = Self { orderings }; + /// Creates a new ordering equivalence class from the given orderings + /// and removes any redundant entries (if given). + pub fn new( + orderings: impl IntoIterator>, + ) -> Self { + let mut result = Self { + orderings: orderings.into_iter().filter_map(LexOrdering::new).collect(), + }; result.remove_redundant_entries(); result } - /// Takes ownership of the vector of orderings comprising this equivalence - /// class. - pub fn take(self) -> Vec { - self.orderings - } - /// Checks whether `ordering` is a member of this equivalence class. pub fn contains(&self, ordering: &LexOrdering) -> bool { self.orderings.contains(ordering) @@ -328,6 +325,12 @@ impl Display for OrderingEquivalenceClass { } } +impl From for Vec { + fn from(oeq_class: OrderingEquivalenceClass) -> Self { + oeq_class.orderings + } +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 0cea1767cf19d..e71ff74105c91 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1437,21 +1437,12 @@ mod tests { // Initial ordering: [a ASC, b DESC] eq_properties.add_new_ordering([ - PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: asc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_b), - options: desc, - }, + PhysicalSortExpr::new(Arc::clone(&col_a), asc), + PhysicalSortExpr::new(Arc::clone(&col_b), desc), ]); // New ordering: [a DESC] - let new_order = vec![PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: desc, - }]; + let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), desc)]; let result = eq_properties.with_reorder(new_order.clone()); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 65e996bd0bb37..3b773f6fdb0bf 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -19,10 +19,6 @@ mod dependency; // Submodule containing DependencyMap and Dependencies mod joins; // Submodule containing join_equivalence_properties mod union; // Submodule containing calculate_union -use dependency::{ - construct_prefix_orderings, generate_dependency_orderings, referred_dependencies, - Dependencies, DependencyMap, -}; pub use joins::*; pub use union::*; @@ -32,6 +28,10 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use std::{fmt, mem}; +use self::dependency::{ + construct_prefix_orderings, generate_dependency_orderings, referred_dependencies, + Dependencies, DependencyMap, +}; use crate::equivalence::class::AcrossPartitions; use crate::equivalence::{EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping}; use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; @@ -129,15 +129,15 @@ use itertools::Itertools; /// PhysicalSortExpr::new_default(col_c).desc(), /// ]); /// -/// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], const: [b@1(heterogeneous)]") +/// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], eq: [{members: [b@1], constant: (heterogeneous)}]"); /// ``` #[derive(Debug, Clone)] pub struct EquivalenceProperties { - /// Distinct equivalence classes (exprs known to have the same expressions) + /// Distinct equivalence classes (i.e. expressions with the same value). eq_group: EquivalenceGroup, - /// Equivalent sort expressions + /// Equivalent sort expressions (i.e. those define the same ordering). oeq_class: OrderingEquivalenceClass, - /// Table constraints + /// Table constraints that factor in equivalence calculations. constraints: Constraints, /// Schema associated with this object. schema: SchemaRef, @@ -165,7 +165,6 @@ impl EquivalenceProperties { schema: SchemaRef, orderings: impl IntoIterator>, ) -> Self { - let orderings = orderings.into_iter().filter_map(LexOrdering::new).collect(); Self { eq_group: EquivalenceGroup::default(), oeq_class: OrderingEquivalenceClass::new(orderings), @@ -215,7 +214,7 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - let mut sort_exprs = self.oeq_class().output_ordering()?.take(); + let mut sort_exprs: Vec<_> = self.oeq_class().output_ordering()?.into(); // Prune out constant expressions: sort_exprs.retain(|sort_expr| { let Some(cls) = self.eq_group.get_equivalence_class(&sort_expr.expr) else { @@ -234,8 +233,7 @@ impl EquivalenceProperties { self.oeq_class .iter() .cloned() - .filter_map(|ordering| self.normalize_sort_exprs(ordering)) - .collect(), + .filter_map(|ordering| self.normalize_sort_exprs(ordering)), ) } @@ -616,7 +614,7 @@ impl EquivalenceProperties { }; let prefix_len = self.common_sort_prefix_length(normalized_ordering.clone()); let flag = prefix_len == normalized_ordering.len(); - let mut sort_exprs = normalized_ordering.take(); + let mut sort_exprs: Vec<_> = normalized_ordering.into(); if !flag { sort_exprs.truncate(prefix_len); } @@ -747,19 +745,21 @@ impl EquivalenceProperties { .all(|(reference, given)| given.compatible(&reference)) } - /// we substitute the ordering according to input expression type, this is a simplified version - /// In this case, we just substitute when the expression satisfy the following condition: - /// I. just have one column and is a CAST expression - /// TODO: Add one-to-ones analysis for monotonic ScalarFunctions. - /// TODO: we could precompute all the scenario that is computable, for example: atan(x + 1000) should also be substituted if - /// x is DESC or ASC - /// After substitution, we may generate more than 1 `LexOrdering`. As an example, - /// `[a ASC, b ASC]` will turn into `[a ASC, b ASC], [CAST(a) ASC, b ASC]` when projection expressions `a, b, CAST(a)` is applied. + /// Substitute the ordering according to input expression type. We substitute + /// when the expression satisfies the following conditions: + /// + /// 1. Has one column and is a `CAST` expression. + /// + /// After substitution, we may generate more than one `LexOrdering`. For + /// example, `[a ASC, b ASC]` will turn into `[CAST(a) ASC, b ASC]` and + /// `[a ASC, b ASC]` when applying projection expressions `a, b, CAST(a)`. pub fn substitute_ordering_component( &self, mapping: &ProjectionMapping, sort_expr: LexOrdering, ) -> Result> { + // TODO: Handle all scenarios that allow precomputation; e.g. when `x` + // is sorted, `atan(x + 1000)` should also be substituted. let new_orderings = sort_expr .into_iter() .map(|sort_expr| { @@ -815,8 +815,7 @@ impl EquivalenceProperties { .map(|order| self.substitute_ordering_component(mapping, order)) .collect::>>()? .into_iter() - .flatten() - .collect(); + .flatten(); self.oeq_class = OrderingEquivalenceClass::new(new_orderings); Ok(()) } @@ -1227,7 +1226,7 @@ impl EquivalenceProperties { // be non-empty. .map(|v| LexOrdering::new(v).unwrap()) }) - .collect::>()?; + .collect::>>()?; // Update the schema, the equivalence group and the ordering equivalence // class: diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index e389740bb2416..7733e7bcfa707 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -159,7 +159,7 @@ impl UnionEquivalentOrderingBuilder { ) { AddedOrdering::Yes => break, AddedOrdering::No(ordering) => { - let mut sort_exprs = ordering.take(); + let mut sort_exprs: Vec<_> = ordering.into(); sort_exprs.pop(); if let Some(ordering) = LexOrdering::new(sort_exprs) { ordering diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 42c1c6b4602be..7552d202b7a97 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -39,7 +39,7 @@ pub fn add_sort_above( sort_requirements: LexRequirement, fetch: Option, ) -> PlanContext { - let mut sort_reqs = sort_requirements.take(); + let mut sort_reqs: Vec<_> = sort_requirements.into(); sort_reqs.retain(|sort_expr| { node.plan .equivalence_properties() diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 4ea5a6edc1b8f..6184acddda631 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -335,7 +335,7 @@ impl TestMemoryExec { sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() - .take(); + .into(); } self.sort_information = sort_information; From 0b37352b9700e1bed5427f320b8742ec9eb0211b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 24 Apr 2025 17:23:02 +0300 Subject: [PATCH 108/167] Simplify projection mapping - 1 --- .../fuzz_cases/equivalence/projection.rs | 5 +- .../tests/fuzz_cases/equivalence/utils.rs | 4 +- datafusion/datasource/src/memory.rs | 19 ++-- .../src/equivalence/projection.rs | 91 ++++++++----------- .../src/equivalence/properties/dependency.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/projection.rs | 2 +- datafusion/physical-plan/src/test.rs | 19 ++-- 8 files changed, 64 insertions(+), 84 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index 775e904cd998e..76f73330828dd 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -145,8 +145,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { for proj_exprs in proj_exprs.iter().combinations(n_req) { let proj_exprs = proj_exprs .into_iter() - .map(|(expr, name)| (Arc::clone(expr), name.to_string())) - .collect::>(); + .map(|(expr, name)| (Arc::clone(expr), name.to_string())); let (projected_batch, projected_eq) = apply_projection( proj_exprs.clone(), &table_data_with_properties, @@ -154,7 +153,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { )?; let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + ProjectionMapping::try_new(proj_exprs, &test_schema)?; let projected_exprs = projection_mapping .iter() diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index a987d080fdb0b..f8aafd1839095 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -133,12 +133,12 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti // Apply projection to the input_data, return projected equivalence properties and record batch pub fn apply_projection( - proj_exprs: Vec<(Arc, String)>, + proj_exprs: impl IntoIterator, String)>, input_data: &RecordBatch, input_eq_properties: &EquivalenceProperties, ) -> Result<(RecordBatch, EquivalenceProperties)> { let input_schema = input_data.schema(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; let output_schema = output_schema(&projection_mapping, &input_schema)?; let num_rows = input_data.num_rows(); diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index ecb459a7d07a3..e601976cb9bbf 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -695,20 +695,17 @@ impl MemorySourceConfig { // If there is a projection on the source, we also need to project orderings if let Some(projection) = &self.projection { + let base_schema = self.original_schema(); + let proj_exprs = projection.iter().map(|idx| { + let name = base_schema.field(*idx).name(); + (Arc::new(Column::new(name, *idx)) as _, name.to_string()) + }); + let projection_mapping = + ProjectionMapping::try_new(proj_exprs, &base_schema)?; let base_eqp = EquivalenceProperties::new_with_orderings( - self.original_schema(), + Arc::clone(&base_schema), sort_information, ); - let proj_exprs = projection - .iter() - .map(|idx| { - let base_schema = self.original_schema(); - let name = base_schema.field(*idx).name(); - (Arc::new(Column::new(name, *idx)) as _, name.to_string()) - }) - .collect::>(); - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 895f632d93304..6c6fc65f346ef 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -48,37 +48,39 @@ impl ProjectionMapping { /// /// where `col("c + d")` means the column named `"c + d"`. pub fn try_new( - expr: &[(Arc, String)], + expr: impl IntoIterator, String)>, input_schema: &SchemaRef, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - expr.iter() + expr.into_iter() .enumerate() - .map(|(expr_idx, (expression, name))| { - let target_expr = Arc::new(Column::new(name, expr_idx)) as _; - Arc::clone(expression) - .transform_down(|e| match e.as_any().downcast_ref::() { - Some(col) => { - // Sometimes, an expression and its name in the input_schema - // doesn't match. This can cause problems, so we make sure - // that the expression name matches with the name in `input_schema`. - // Conceptually, `source_expr` and `expression` should be the same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - if col.name() != matching_input_field.name() { - return internal_err!("Input field name {} does not match with the projection expression {}", - matching_input_field.name(),col.name()) - } - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::yes(Arc::new(matching_input_column))) + .map(|(expr_idx, (expr, name))| { + let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; + expr.transform_down(|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + if col.name() != matching_input_field.name() { + return internal_err!( + "Input field name {} does not match with the projection expression {}", + matching_input_field.name(), + col.name() + ); } - None => Ok(Transformed::no(e)), - }) - .data() - .map(|source_expr| (source_expr, target_expr)) + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::no(e)), + }) + .data() + .map(|source_expr| (source_expr, target_expr)) }) - .collect::>>() + .collect::>() .map(|map| Self { map }) } @@ -87,8 +89,12 @@ impl ProjectionMapping { /// This is used when the output is a subset of the input without any /// other transformations. The indices are for columns in the schema. pub fn from_indices(indices: &[usize], schema: &SchemaRef) -> Result { - let projection_exprs = project_index_to_exprs(indices, schema); - ProjectionMapping::try_new(&projection_exprs, schema) + let projection_exprs = indices.iter().map(|index| { + let field = schema.field(*index); + let column = Arc::new(Column::new(field.name(), *index)); + (column as _, field.name().clone()) + }); + ProjectionMapping::try_new(projection_exprs, schema) } /// Iterate over pairs of (source, target) expressions @@ -119,22 +125,6 @@ impl ProjectionMapping { } } -fn project_index_to_exprs( - projection_index: &[usize], - schema: &SchemaRef, -) -> Vec<(Arc, String)> { - projection_index - .iter() - .map(|index| { - let field = schema.field(*index); - ( - Arc::new(Column::new(field.name(), *index)) as Arc, - field.name().to_owned(), - ) - }) - .collect::>() -} - #[cfg(test)] mod tests { use super::*; @@ -610,9 +600,8 @@ mod tests { let proj_exprs = proj_exprs .into_iter() - .map(|(expr, name)| (Arc::clone(expr), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + .map(|(expr, name)| (Arc::clone(expr), name)); + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let expected = expected @@ -685,9 +674,8 @@ mod tests { ]; let proj_exprs = proj_exprs .into_iter() - .map(|(expr, name)| (Arc::clone(expr), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + .map(|(expr, name)| (Arc::clone(expr), name)); + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let col_a_new = &col("a_new", &output_schema)?; @@ -866,9 +854,8 @@ mod tests { ]; let proj_exprs = proj_exprs .into_iter() - .map(|(expr, name)| (Arc::clone(expr), name)) - .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + .map(|(expr, name)| (Arc::clone(expr), name)); + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let col_a_plus_b_new = &col("a+b", &output_schema)?; diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index e71ff74105c91..12ad6833d8842 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -459,7 +459,7 @@ mod tests { (Arc::clone(&col_a), "a3".to_string()), (Arc::clone(&col_a), "a4".to_string()), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; let out_schema = output_schema(&projection_mapping, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 @@ -469,7 +469,7 @@ mod tests { (Arc::clone(&col_a), "a3".to_string()), (Arc::clone(&col_a), "a4".to_string()), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 let col_a1 = &col("a1", &out_schema)?; @@ -524,7 +524,7 @@ mod tests { (col("c", &input_schema)?, "c".to_string()), (col("d", &input_schema)?, "d".to_string()), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; let out_properties = input_properties.project(&projection_mapping, input_schema); assert_eq!( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b20b69bf0a36a..25a05ef4f6096 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -522,7 +522,7 @@ impl AggregateExec { // construct a map from the input expression to the output expression of the Aggregation group by let group_expr_mapping = - ProjectionMapping::try_new(&group_by.expr, &input.schema())?; + ProjectionMapping::try_new(group_by.expr.clone(), &input.schema())?; let cache = Self::compute_properties( &input, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 09543ddd91871..c430f1352179b 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -98,7 +98,7 @@ impl ProjectionExec { )); // Construct a map from the input expressions to the output expression of the Projection - let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(expr.clone(), &input_schema)?; let cache = Self::compute_properties(&input, &projection_mapping, Arc::clone(&schema))?; Ok(Self { diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 6184acddda631..ec38f04de6e44 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -318,20 +318,17 @@ impl TestMemoryExec { // If there is a projection on the source, we also need to project orderings if let Some(projection) = &self.projection { + let base_schema = self.original_schema(); + let proj_exprs = projection.iter().map(|idx| { + let name = base_schema.field(*idx).name(); + (Arc::new(Column::new(name, *idx)) as _, name.to_string()) + }); + let projection_mapping = + ProjectionMapping::try_new(proj_exprs, &base_schema)?; let base_eqp = EquivalenceProperties::new_with_orderings( - self.original_schema(), + Arc::clone(&base_schema), sort_information, ); - let proj_exprs = projection - .iter() - .map(|idx| { - let base_schema = self.original_schema(); - let name = base_schema.field(*idx).name(); - (Arc::new(Column::new(name, *idx)) as _, name.to_string()) - }) - .collect::>(); - let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; sort_information = base_eqp .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() From 914458e8e0ecef75675f7ef61b981e93cc4fff08 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 25 Apr 2025 08:37:13 +0300 Subject: [PATCH 109/167] Use a map instead of a vector in ProjectionMapping --- .../fuzz_cases/equivalence/projection.rs | 4 +- .../tests/fuzz_cases/equivalence/utils.rs | 35 +++---- .../physical-expr/src/equivalence/class.rs | 31 ++++--- .../physical-expr/src/equivalence/mod.rs | 34 +++---- .../src/equivalence/projection.rs | 91 ++++++++----------- .../src/equivalence/properties/mod.rs | 36 +++++--- .../physical-plan/src/aggregates/mod.rs | 9 +- 7 files changed, 114 insertions(+), 126 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index 76f73330828dd..01c493072d6b1 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -157,7 +157,9 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { let projected_exprs = projection_mapping .iter() - .map(|(_source, target)| Arc::clone(target)) + .flat_map(|(_, targets)| { + targets.iter().map(|(target, _)| Arc::clone(target)) + }) .collect::>(); for n_req in 1..=projected_exprs.len() { diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index f8aafd1839095..5ba2eeca8bd0e 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -23,7 +23,7 @@ use arrow::array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Arra use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn, SortOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::utils::{compare_rows, get_row_at_idx}; -use datafusion_common::{exec_err, plan_datafusion_err, DataFusionError, Result}; +use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, @@ -39,31 +39,26 @@ use datafusion_physical_plan::expressions::{col, Column}; use itertools::izip; use rand::prelude::*; +/// Projects the input schema based on the given projection mapping. pub fn output_schema( mapping: &ProjectionMapping, input_schema: &Arc, ) -> Result { - // Calculate output schema - let fields: Result> = mapping - .iter() - .map(|(source, target)| { - let name = target - .as_any() - .downcast_ref::() - .ok_or_else(|| plan_datafusion_err!("Expects to have column"))? - .name(); - let field = Field::new( - name, - source.data_type(input_schema)?, - source.nullable(input_schema)?, - ); - - Ok(field) - }) - .collect(); + // Calculate output schema: + let mut fields = vec![]; + for (source, targets) in mapping.iter() { + let data_type = source.data_type(input_schema)?; + let nullable = source.nullable(input_schema)?; + for (target, _) in targets { + let Some(column) = target.as_any().downcast_ref::() else { + return plan_err!("Expects to have column"); + }; + fields.push(Field::new(column.name(), data_type.clone(), nullable)); + } + } let output_schema = Arc::new(Schema::new_with_metadata( - fields?, + fields, input_schema.metadata().clone(), )); diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 6394f19411222..b3ff8c8b76c3e 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -535,9 +535,10 @@ impl EquivalenceGroup { ) -> Option> { // First, we try to project expressions with an exact match. If we are // unable to do this, we consult equivalence classes. - if let Some(target) = mapping.target_expr(expr) { + if let Some(target) = mapping.get(expr) { // If we match the source, we can project directly: - return Some(target); + let (target, _) = target.first().unwrap(); + return Some(Arc::clone(target)); } else { // If the given expression is not inside the mapping, try to project // expressions considering the equivalence classes. @@ -547,6 +548,7 @@ impl EquivalenceGroup { // and the equivalence class `(a, b)`, expression `b` projects to `a1`. let eq_class = self.get_equivalence_class(source); if eq_class.is_some_and(|group| group.contains(expr)) { + let (target, _) = target.first().unwrap(); return Some(Arc::clone(target)); } } @@ -577,21 +579,23 @@ impl EquivalenceGroup { // class that contains the corresponding target expression. let mut new_constants = vec![]; let mut new_classes = IndexMap::<_, EquivalenceClass>::new(); - for (source, target) in mapping.iter() { + for (source, targets) in mapping.iter() { // We need to find equivalent projected expressions. For example, // consider a table with columns `[a, b, c]` with `a` == `b`, and // projection `[a + c, b + c]`. To conclude that `a + c == b + c`, // we first normalize all source expressions in the mapping, then // merge all equivalent expressions into the classes. let normalized_expr = self.normalize_expr(Arc::clone(source)); - new_classes - .entry(normalized_expr) - .or_default() - .push(Arc::clone(target)); + let cls = new_classes.entry(normalized_expr).or_default(); + for (target, _) in targets { + cls.push(Arc::clone(target)); + } // Save new constants arising from the projection: if let Some(across) = self.is_expr_constant(source) { - let const_expr = ConstExpr::new(Arc::clone(target), across); - new_constants.push(const_expr); + for (target, _) in targets { + let const_expr = ConstExpr::new(Arc::clone(target), across.clone()); + new_constants.push(const_expr); + } } } @@ -1074,7 +1078,7 @@ mod tests { ])); let mapping = ProjectionMapping { - map: vec![ + map: [ ( binary( col("a", &schema)?, @@ -1082,7 +1086,7 @@ mod tests { col("c", &schema)?, &schema, )?, - col("a+c", &projected_schema)?, + vec![(col("a+c", &projected_schema)?, 0)], ), ( binary( @@ -1091,9 +1095,10 @@ mod tests { col("c", &schema)?, &schema, )?, - col("b+c", &projected_schema)?, + vec![(col("b+c", &projected_schema)?, 1)], ), - ], + ] + .into(), }; let projected = group.project(&mapping); diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 98f4e977c423a..6d4bf84b33814 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -82,7 +82,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_common::{plan_datafusion_err, Result}; + use datafusion_common::{plan_err, Result}; use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; /// Converts a string to a physical sort expression @@ -121,27 +121,21 @@ mod tests { mapping: &ProjectionMapping, input_schema: &Arc, ) -> Result { - // Calculate output schema - let fields: Result> = mapping - .iter() - .map(|(source, target)| { - let name = target - .as_any() - .downcast_ref::() - .ok_or_else(|| plan_datafusion_err!("Expects to have column"))? - .name(); - let field = Field::new( - name, - source.data_type(input_schema)?, - source.nullable(input_schema)?, - ); - - Ok(field) - }) - .collect(); + // Calculate output schema: + let mut fields = vec![]; + for (source, targets) in mapping.iter() { + let data_type = source.data_type(input_schema)?; + let nullable = source.nullable(input_schema)?; + for (target, _) in targets { + let Some(column) = target.as_any().downcast_ref::() else { + return plan_err!("Expects to have column"); + }; + fields.push(Field::new(column.name(), data_type.clone(), nullable)); + } + } let output_schema = Arc::new(Schema::new_with_metadata( - fields?, + fields, input_schema.metadata().clone(), )); diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 6c6fc65f346ef..98cd371003d81 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::ops::Deref; use std::sync::Arc; use crate::expressions::Column; @@ -24,13 +25,16 @@ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{internal_err, Result}; +use indexmap::IndexMap; + /// Stores the mapping between source expressions and target expressions for a /// projection. #[derive(Debug, Clone)] pub struct ProjectionMapping { /// Mapping between source expressions and target expressions. /// Vector indices correspond to the indices after projection. - pub map: Vec<(Arc, Arc)>, + #[allow(clippy::type_complexity)] + pub map: IndexMap, Vec<(Arc, usize)>>, } impl ProjectionMapping { @@ -52,36 +56,36 @@ impl ProjectionMapping { input_schema: &SchemaRef, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - expr.into_iter() - .enumerate() - .map(|(expr_idx, (expr, name))| { - let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; - expr.transform_down(|e| match e.as_any().downcast_ref::() { - Some(col) => { - // Sometimes, an expression and its name in the input_schema - // doesn't match. This can cause problems, so we make sure - // that the expression name matches with the name in `input_schema`. - // Conceptually, `source_expr` and `expression` should be the same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - if col.name() != matching_input_field.name() { - return internal_err!( - "Input field name {} does not match with the projection expression {}", - matching_input_field.name(), - col.name() - ); - } - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::yes(Arc::new(matching_input_column))) + let mut map = IndexMap::new(); + for (expr_idx, (expr, name)) in expr.into_iter().enumerate() { + let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; + let source_expr = expr.transform_down(|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_field = input_schema.field(idx); + let matching_name = matching_field.name(); + if col.name() != matching_name { + return internal_err!( + "Input field name {} does not match with the projection expression {}", + matching_name, + col.name() + ); } - None => Ok(Transformed::no(e)), - }) - .data() - .map(|source_expr| (source_expr, target_expr)) + let matching_column = Column::new(matching_name, idx); + Ok(Transformed::yes(Arc::new(matching_column))) + } + None => Ok(Transformed::no(e)), }) - .collect::>() - .map(|map| Self { map }) + .data()?; + map.entry(source_expr) + .or_insert_with(Vec::new) + .push((target_expr, expr_idx)); + } + Ok(Self { map }) } /// Constructs a subset mapping using the provided indices. @@ -96,32 +100,13 @@ impl ProjectionMapping { }); ProjectionMapping::try_new(projection_exprs, schema) } +} - /// Iterate over pairs of (source, target) expressions - pub fn iter( - &self, - ) -> impl Iterator, Arc)> + '_ { - self.map.iter() - } +impl Deref for ProjectionMapping { + type Target = IndexMap, Vec<(Arc, usize)>>; - /// This function returns the target expression for a given source expression. - /// - /// # Arguments - /// - /// * `expr` - Source physical expression. - /// - /// # Returns - /// - /// An `Option` containing the target for the given source expression, - /// where a `None` value means that `expr` is not inside the mapping. - pub fn target_expr( - &self, - expr: &Arc, - ) -> Option> { - self.map - .iter() - .find(|(source, _)| source.eq(expr)) - .map(|(_, target)| Arc::clone(target)) + fn deref(&self) -> &Self::Target { + &self.map } } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 3b773f6fdb0bf..9d20493b12f68 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -931,7 +931,7 @@ impl EquivalenceProperties { .map(|(source, target)| { let normalized_source = self.eq_group.normalize_expr(Arc::clone(source)); - (normalized_source, Arc::clone(target)) + (normalized_source, target.to_vec()) }) .collect(), } @@ -957,7 +957,7 @@ impl EquivalenceProperties { // Get dependency map for existing orderings: let dependency_map = self.construct_dependency_map(&mapping); - let orderings = mapping.iter().flat_map(|(source, target)| { + let orderings = mapping.iter().flat_map(|(source, targets)| { referred_dependencies(&dependency_map, source) .into_iter() .filter_map(|relevant_deps| { @@ -972,19 +972,25 @@ impl EquivalenceProperties { } }) .flat_map(|(options, relevant_deps)| { - let sort_expr = PhysicalSortExpr::new(Arc::clone(target), options); - // Generate dependent orderings (i.e. prefixes for `sort_expr`): - let mut dependency_orderings = + // Generate dependent orderings (i.e. prefixes for targets): + let dependency_orderings = generate_dependency_orderings(&relevant_deps, &dependency_map); - // Append `sort_expr` to the dependent orderings: + let sort_exprs = targets.iter().map(|(target, _)| { + PhysicalSortExpr::new(Arc::clone(target), options) + }); if dependency_orderings.is_empty() { - dependency_orderings.push([sort_expr].into()); + sort_exprs.map(|sort_expr| [sort_expr].into()).collect() } else { - for ordering in dependency_orderings.iter_mut() { - ordering.push(sort_expr.clone()); - } + sort_exprs + .flat_map(|sort_expr| { + let mut result = dependency_orderings.clone(); + for ordering in result.iter_mut() { + ordering.push(sort_expr.clone()); + } + result + }) + .collect::>() } - dependency_orderings }) }); @@ -1037,10 +1043,12 @@ impl EquivalenceProperties { fn projected_constraints(&self, mapping: &ProjectionMapping) -> Option { let indices = mapping .iter() - .filter_map(|(_, target)| target.as_any().downcast_ref::()) - .map(|col| col.index()) + .flat_map(|(_, targets)| { + targets.iter().flat_map(|(target, _)| { + target.as_any().downcast_ref::().map(|c| c.index()) + }) + }) .collect::>(); - debug_assert_eq!(mapping.map.len(), indices.len()); self.constraints.project(&indices) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 25a05ef4f6096..3039126cf9f9e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -693,11 +693,10 @@ impl AggregateExec { group_expr_mapping .map .iter() - .filter_map(|(_, target_col)| { - target_col - .as_any() - .downcast_ref::() - .map(|c| c.index()) + .flat_map(|(_, target_cols)| { + target_cols.iter().flat_map(|(expr, _)| { + expr.as_any().downcast_ref::().map(|c| c.index()) + }) }) .collect(), ); From cce61684b2b444270e1a0f260c37ea07df0da0ee Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 26 Apr 2025 11:55:52 +0300 Subject: [PATCH 110/167] Simplify DependencyMap --- .../src/equivalence/properties/dependency.rs | 158 +++++++----------- .../src/equivalence/properties/mod.rs | 105 ++++++------ 2 files changed, 113 insertions(+), 150 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 12ad6833d8842..dc20026f61675 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -16,28 +16,28 @@ // under the License. use std::fmt::{self, Display}; +use std::ops::{Deref, DerefMut}; use std::sync::Arc; +use super::expr_refers; use crate::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use indexmap::IndexSet; -use indexmap::IndexMap; +use indexmap::{IndexMap, IndexSet}; use itertools::Itertools; -use super::{expr_refers, ExprWrapper}; - // A list of sort expressions that can be calculated from a known set of /// dependencies. #[derive(Debug, Default, Clone, PartialEq, Eq)] pub struct Dependencies { - inner: IndexSet, + sort_exprs: IndexSet, } impl Display for Dependencies { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "[")?; - let mut iter = self.inner.iter(); + let mut iter = self.sort_exprs.iter(); if let Some(dep) = iter.next() { write!(f, "{}", dep)?; } @@ -49,38 +49,34 @@ impl Display for Dependencies { } impl Dependencies { - /// Create a new empty `Dependencies` instance. - fn new() -> Self { + // Creates a new `Dependencies` instance from the given sort expressions. + pub fn new(sort_exprs: impl IntoIterator) -> Self { Self { - inner: IndexSet::new(), + sort_exprs: sort_exprs.into_iter().collect(), } } +} - /// Create a new `Dependencies` from an iterator of `PhysicalSortExpr`. - pub fn new_from_iter(iter: impl IntoIterator) -> Self { - Self { - inner: iter.into_iter().collect(), - } - } +impl Deref for Dependencies { + type Target = IndexSet; - /// Insert a new dependency into the set. - pub fn insert(&mut self, sort_expr: PhysicalSortExpr) { - self.inner.insert(sort_expr); + fn deref(&self) -> &Self::Target { + &self.sort_exprs } +} - /// Iterator over dependencies in the set - pub fn iter(&self) -> impl Iterator + Clone { - self.inner.iter() +impl DerefMut for Dependencies { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.sort_exprs } +} - /// Return the inner set of dependencies - pub fn into_inner(self) -> IndexSet { - self.inner - } +impl IntoIterator for Dependencies { + type Item = PhysicalSortExpr; + type IntoIter = as IntoIterator>::IntoIter; - /// Returns true if there are no dependencies - fn is_empty(&self) -> bool { - self.inner.is_empty() + fn into_iter(self) -> Self::IntoIter { + self.sort_exprs.into_iter() } } @@ -133,26 +129,25 @@ impl<'a> DependencyEnumerator<'a> { let node = dependency_map .get(referred_sort_expr) .expect("`referred_sort_expr` should be inside `dependency_map`"); - // Since we work on intermediate nodes, we are sure `val.target_sort_expr` - // exists. - let target_sort_expr = node.target_sort_expr.as_ref().unwrap(); + // Since we work on intermediate nodes, we are sure `node.target` exists. + let target = node.target.as_ref().unwrap(); // An empty dependency means the referred_sort_expr represents a global ordering. // Return its projected version, which is the target_expression. if node.dependencies.is_empty() { - return vec![[target_sort_expr.clone()].into()]; + return vec![[target.clone()].into()]; }; node.dependencies .iter() .flat_map(|dep| { - let mut orderings = if self.insert(target_sort_expr, dep) { + let mut orderings = if self.insert(target, dep) { self.construct_orderings(dep, dependency_map) } else { vec![] }; for ordering in orderings.iter_mut() { - ordering.push(target_sort_expr.clone()) + ordering.push(target.clone()) } orderings }) @@ -178,70 +173,56 @@ impl<'a> DependencyEnumerator<'a> { /// # Note on IndexMap Rationale /// /// Using `IndexMap` (which preserves insert order) to ensure consistent results -/// across different executions for the same query. We could have used -/// `HashSet`, `HashMap` in place of them without any loss of functionality. +/// across different executions for the same query. We could have used `HashSet` +/// and `HashMap` instead without any loss of functionality. /// /// As an example, if existing orderings are /// 1. `[a ASC, b ASC]` -/// 2. `[c ASC]` for +/// 2. `[c ASC]` /// /// Then both the following output orderings are valid /// 1. `[a ASC, b ASC, c ASC]` /// 2. `[c ASC, a ASC, b ASC]` /// -/// (this are both valid as they are concatenated versions of the alternative -/// orderings). When using `HashSet`, `HashMap` it is not guaranteed to generate -/// consistent result, among the possible 2 results in the example above. -#[derive(Debug)] +/// These are both valid as they are concatenated versions of the alternative +/// orderings. Had we used `HashSet`/`HashMap`, we couldn't guarantee to generate +/// the same result among the possible two results in the example above. +#[derive(Debug, Default)] pub struct DependencyMap { - inner: IndexMap, + map: IndexMap, } impl DependencyMap { - pub fn new() -> Self { - Self { - inner: IndexMap::new(), - } - } - /// Insert a new dependency `sort_expr` --> `dependency` into the map. /// /// If `target_sort_expr` is none, a new entry is created with empty dependencies. pub fn insert( &mut self, - sort_expr: &PhysicalSortExpr, - target_sort_expr: Option<&PhysicalSortExpr>, - dependency: Option<&PhysicalSortExpr>, + sort_expr: PhysicalSortExpr, + target_sort_expr: Option, + dependency: Option, ) { - self.inner - .entry(sort_expr.clone()) - .or_insert_with(|| DependencyNode { - target_sort_expr: target_sort_expr.cloned(), - dependencies: Dependencies::new(), - }) - .insert_dependency(dependency) - } - - /// Iterator over (sort_expr, DependencyNode) pairs - pub fn iter(&self) -> impl Iterator { - self.inner.iter() + let entry = self.map.entry(sort_expr); + let node = entry.or_insert_with(|| DependencyNode { + target: target_sort_expr, + dependencies: Dependencies::default(), + }); + node.dependencies.extend(dependency); } +} - /// iterator over all sort exprs - pub fn sort_exprs(&self) -> impl Iterator { - self.inner.keys() - } +impl Deref for DependencyMap { + type Target = IndexMap; - /// Return the dependency node for the given sort expression, if any - pub fn get(&self, sort_expr: &PhysicalSortExpr) -> Option<&DependencyNode> { - self.inner.get(sort_expr) + fn deref(&self) -> &Self::Target { + &self.map } } impl Display for DependencyMap { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { writeln!(f, "DependencyMap: {{")?; - for (sort_expr, node) in self.inner.iter() { + for (sort_expr, node) in self.map.iter() { writeln!(f, " {sort_expr} --> {node}")?; } writeln!(f, "}}") @@ -256,29 +237,20 @@ impl Display for DependencyMap { /// /// # Fields /// -/// - `target_sort_expr`: An optional `PhysicalSortExpr` representing the target -/// sort expression associated with the node. It is `None` if the sort expression +/// - `target`: An optional `PhysicalSortExpr` representing the target sort +/// expression associated with the node. It is `None` if the sort expression /// cannot be projected. /// - `dependencies`: A [`Dependencies`] containing dependencies on other sort /// expressions that are referred to by the target sort expression. #[derive(Debug, Clone, PartialEq, Eq)] pub struct DependencyNode { - pub target_sort_expr: Option, - pub dependencies: Dependencies, -} - -impl DependencyNode { - /// Insert dependency to the state (if exists). - fn insert_dependency(&mut self, dependency: Option<&PhysicalSortExpr>) { - if let Some(dep) = dependency { - self.dependencies.insert(dep.clone()); - } - } + pub(crate) target: Option, + pub(crate) dependencies: Dependencies, } impl Display for DependencyNode { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if let Some(target) = &self.target_sort_expr { + if let Some(target) = &self.target { write!(f, "(target: {}, ", target)?; } else { write!(f, "(")?; @@ -307,12 +279,12 @@ pub fn referred_dependencies( source: &Arc, ) -> Vec { // Associate `PhysicalExpr`s with `PhysicalSortExpr`s that contain them: - let mut expr_to_sort_exprs = IndexMap::::new(); + let mut expr_to_sort_exprs = IndexMap::<_, Dependencies>::new(); for sort_expr in dependency_map - .sort_exprs() + .keys() .filter(|sort_expr| expr_refers(source, &sort_expr.expr)) { - let key = ExprWrapper(Arc::clone(&sort_expr.expr)); + let key = Arc::clone(&sort_expr.expr); expr_to_sort_exprs .entry(key) .or_default() @@ -322,16 +294,10 @@ pub fn referred_dependencies( // Generate all valid dependencies for the source. For example, if the source // is `a + b` and the map is `[a -> (a ASC, a DESC), b -> (b ASC)]`, we get // `vec![HashSet(a ASC, b ASC), HashSet(a DESC, b ASC)]`. - let dependencies = expr_to_sort_exprs + expr_to_sort_exprs .into_values() - .map(Dependencies::into_inner) - .collect::>(); - dependencies - .iter() .multi_cartesian_product() - .map(|referred_deps| { - Dependencies::new_from_iter(referred_deps.into_iter().cloned()) - }) + .map(Dependencies::new) .collect() } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 9d20493b12f68..fc8d3ebf9cd85 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -24,7 +24,6 @@ pub use union::*; use std::collections::VecDeque; use std::fmt::Display; -use std::hash::{Hash, Hasher}; use std::sync::Arc; use std::{fmt, mem}; @@ -832,7 +831,7 @@ impl EquivalenceProperties { /// ``` /// /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to - /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. + /// `Some(a1 + b1)` and `d` to `None`, meaning that it is not projectable. pub fn project_expr( &self, expr: &Arc, @@ -871,42 +870,58 @@ impl EquivalenceProperties { /// c ASC: Node {None, HashSet{a ASC}} /// ``` fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { - let mut dependency_map = DependencyMap::new(); - for ordering in self.normalized_oeq_class().iter() { - for (idx, sort_expr) in ordering.iter().enumerate() { - let target_sort_expr = - self.project_expr(&sort_expr.expr, mapping).map(|expr| { - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }); - let is_projected = target_sort_expr.is_some(); - if is_projected - || mapping - .iter() - .any(|(source, _)| expr_refers(source, &sort_expr.expr)) - { - // Previous ordering is a dependency. Note that there is no, - // dependency for a leading ordering (i.e. the first sort - // expression). - let dependency = idx.checked_sub(1).map(|a| &ordering[a]); - // Add sort expressions that can be projected or referred to - // by any of the projection expressions to the dependency map: - dependency_map.insert( - sort_expr, - target_sort_expr.as_ref(), - dependency, - ); - } - if !is_projected { - // If we can not project, stop constructing the dependency - // map as remaining dependencies will be invalid after projection. + let mut map = DependencyMap::default(); + for ordering in self.normalized_oeq_class().into_iter() { + // Previous expression is a dependency. Note that there is no + // dependency for the leading expression. + if !self.insert_to_dependency_map( + mapping, + ordering[0].clone(), + None, + &mut map, + ) { + continue; + } + for (dependency, sort_expr) in ordering.into_iter().tuple_windows() { + if !self.insert_to_dependency_map( + mapping, + sort_expr, + Some(dependency), + &mut map, + ) { + // If we can't project, stop constructing the dependency map + // as remaining dependencies will be invalid post projection. break; } } } - dependency_map + map + } + + /// Projects the sort expression according to the projection mapping and + /// inserts it into the dependency map with the given dependency. Returns + /// a boolean flag indicating whether the given expression is projectable. + fn insert_to_dependency_map( + &self, + mapping: &ProjectionMapping, + sort_expr: PhysicalSortExpr, + dependency: Option, + map: &mut DependencyMap, + ) -> bool { + let target_sort_expr = self + .project_expr(&sort_expr.expr, mapping) + .map(|expr| PhysicalSortExpr::new(expr, sort_expr.options)); + let projectable = target_sort_expr.is_some(); + if projectable + || mapping + .iter() + .any(|(source, _)| expr_refers(source, &sort_expr.expr)) + { + // Add sort expressions that can be projected or referred to + // by any of the projection expressions to the dependency map: + map.insert(sort_expr, target_sort_expr, dependency); + } + projectable } /// Returns a new `ProjectionMapping` where source expressions are normalized. @@ -1004,13 +1019,13 @@ impl EquivalenceProperties { if prefixes.is_empty() { // If prefix is empty, there is no dependency. Insert // empty ordering: - if let Some(target) = &node.target_sort_expr { + if let Some(target) = &node.target { prefixes.push([target.clone()].into()); } } else { // Append current ordering on top its dependencies: for ordering in prefixes.iter_mut() { - if let Some(target) = &node.target_sort_expr { + if let Some(target) = &node.target { ordering.push(target.clone()) } } @@ -1392,21 +1407,3 @@ fn get_expr_properties( expr.get_properties(&child_states) } } - -/// Wrapper struct for `Arc` to use them as keys in a hash map. -#[derive(Debug, Clone)] -struct ExprWrapper(Arc); - -impl PartialEq for ExprWrapper { - fn eq(&self, other: &Self) -> bool { - self.0.eq(&other.0) - } -} - -impl Eq for ExprWrapper {} - -impl Hash for ExprWrapper { - fn hash(&self, state: &mut H) { - self.0.hash(state); - } -} From 8094ab05af98c32cd8d96134178d250a1d9367ce Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 26 Apr 2025 19:20:40 +0300 Subject: [PATCH 111/167] Simplify DependencyMap - 2 --- .../src/equivalence/properties/mod.rs | 132 ++++++++---------- datafusion/physical-plan/src/projection.rs | 10 +- 2 files changed, 64 insertions(+), 78 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index fc8d3ebf9cd85..5da5d3e400f67 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -22,7 +22,6 @@ mod union; // Submodule containing calculate_union pub use joins::*; pub use union::*; -use std::collections::VecDeque; use std::fmt::Display; use std::sync::Arc; use std::{fmt, mem}; @@ -41,9 +40,7 @@ use crate::{ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{ - internal_datafusion_err, plan_err, Constraint, Constraints, HashMap, Result, -}; +use datafusion_common::{plan_err, Constraint, Constraints, HashMap, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_physical_expr_common::utils::ExprPropertiesNode; @@ -744,79 +741,64 @@ impl EquivalenceProperties { .all(|(reference, given)| given.compatible(&reference)) } - /// Substitute the ordering according to input expression type. We substitute - /// when the expression satisfies the following conditions: + /// Modify existing orderings by substituting sort expressions with appropriate + /// targets from the projection mapping. We substitute a sort expression when + /// its physical expression has a one-to-one functional relationship with a + /// target expression in the mapping. /// - /// 1. Has one column and is a `CAST` expression. + /// After substitution, we may generate more than one `LexOrdering` for each + /// existing equivalent ordering. For example, `[a ASC, b ASC]` will turn + /// into `[CAST(a) ASC, b ASC]` and `[a ASC, b ASC]` when applying projection + /// expressions `a, b, CAST(a)`. /// - /// After substitution, we may generate more than one `LexOrdering`. For - /// example, `[a ASC, b ASC]` will turn into `[CAST(a) ASC, b ASC]` and - /// `[a ASC, b ASC]` when applying projection expressions `a, b, CAST(a)`. - pub fn substitute_ordering_component( + /// TODO: Handle all scenarios that allow substitution; e.g. when `x` is + /// sorted, `atan(x + 1000)` should also be substituted. For now, we + /// only consider single-column `CAST` expressions. + fn substitute_oeq_class( &self, + oeq_class: OrderingEquivalenceClass, mapping: &ProjectionMapping, - sort_expr: LexOrdering, - ) -> Result> { - // TODO: Handle all scenarios that allow precomputation; e.g. when `x` - // is sorted, `atan(x + 1000)` should also be substituted. - let new_orderings = sort_expr - .into_iter() - .map(|sort_expr| { - let referring_exprs = mapping - .iter() - .map(|(source, _target)| source) - .filter(|source| expr_refers(source, &sort_expr.expr)) - .cloned(); - let mut result = VecDeque::new(); - let expr_type = sort_expr.expr.data_type(&self.schema)?; - // TODO: Add one-to-one analysis for ScalarFunctions. - for r_expr in referring_exprs { - // We check whether this expression is substitutable. - if let Some(cast_expr) = r_expr.as_any().downcast_ref::() { - // For casts, we need to know whether the cast expression matches: - if cast_expr.expr.eq(&sort_expr.expr) - && cast_expr.is_bigger_cast(&expr_type) + ) -> OrderingEquivalenceClass { + let new_orderings = oeq_class.into_iter().flat_map(|order| { + // Modify/expand existing orderings by substituting sort + // expressions with appropriate targets from the mapping: + order + .into_iter() + .map(|sort_expr| { + let referring_exprs = mapping + .iter() + .map(|(source, _target)| source) + .filter(|source| expr_refers(source, &sort_expr.expr)) + .cloned(); + let mut result = vec![]; + // The sort expression comes from this schema, so the + // following call to `unwrap` is safe. + let expr_type = sort_expr.expr.data_type(&self.schema).unwrap(); + // TODO: Add one-to-one analysis for ScalarFunctions. + for r_expr in referring_exprs { + // We check whether this expression is substitutable. + if let Some(cast_expr) = + r_expr.as_any().downcast_ref::() { - result.push_back(PhysicalSortExpr { - expr: r_expr, - options: sort_expr.options, - }); + // For casts, we need to know whether the cast + // expression matches: + if cast_expr.expr.eq(&sort_expr.expr) + && cast_expr.is_bigger_cast(&expr_type) + { + result.push(PhysicalSortExpr::new( + r_expr, + sort_expr.options, + )); + } } } - } - result.push_front(sort_expr); - Ok(result) - }) - .collect::>>()?; - // Generate all valid orderings, given substituted expressions. - new_orderings - .into_iter() - .multi_cartesian_product() - .map(|o| { - LexOrdering::new(o).ok_or_else(|| { - internal_datafusion_err!( - "Expected a non-empty list of sort expressions" - ) + result.push(sort_expr); + result }) - }) - .collect() - } - - /// In projection, supposed we have a input function 'A DESC B DESC' and the output shares the same expression - /// with A and B, we could surely use the ordering of the original ordering, However, if the A has been changed, - /// for example, A-> Cast(A, Int64) or any other form, it is invalid if we continue using the original ordering - /// Since it would cause bug in dependency constructions, we should substitute the input order in order to get correct - /// dependency map, happen in issue 8838: - pub fn substitute_oeq_class(&mut self, mapping: &ProjectionMapping) -> Result<()> { - let oeq_class = mem::take(&mut self.oeq_class); - let new_orderings = oeq_class - .into_iter() - .map(|order| self.substitute_ordering_component(mapping, order)) - .collect::>>()? - .into_iter() - .flatten(); - self.oeq_class = OrderingEquivalenceClass::new(new_orderings); - Ok(()) + // Generate all valid orderings given substituted expressions: + .multi_cartesian_product() + }); + OrderingEquivalenceClass::new(new_orderings) } /// Projects argument `expr` according to `projection_mapping`, taking @@ -869,9 +851,13 @@ impl EquivalenceProperties { /// b ASC: Node {Some(b_new ASC), HashSet{a ASC}} /// c ASC: Node {None, HashSet{a ASC}} /// ``` - fn construct_dependency_map(&self, mapping: &ProjectionMapping) -> DependencyMap { + fn construct_dependency_map( + &self, + oeq_class: OrderingEquivalenceClass, + mapping: &ProjectionMapping, + ) -> DependencyMap { let mut map = DependencyMap::default(); - for ordering in self.normalized_oeq_class().into_iter() { + for ordering in oeq_class.into_iter() { // Previous expression is a dependency. Note that there is no // dependency for the leading expression. if !self.insert_to_dependency_map( @@ -971,7 +957,9 @@ impl EquivalenceProperties { let mapping = self.normalized_mapping(mapping); // Get dependency map for existing orderings: - let dependency_map = self.construct_dependency_map(&mapping); + let mut oeq_class = self.normalized_oeq_class(); + oeq_class = self.substitute_oeq_class(oeq_class, &mapping); + let dependency_map = self.construct_dependency_map(oeq_class, &mapping); let orderings = mapping.iter().flat_map(|(source, targets)| { referred_dependencies(&dependency_map, source) .into_iter() diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index c430f1352179b..e2fb9040b23c7 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -127,14 +127,12 @@ impl ProjectionExec { schema: SchemaRef, ) -> Result { // Calculate equivalence properties: - let mut input_eq_properties = input.equivalence_properties().clone(); - input_eq_properties.substitute_oeq_class(projection_mapping)?; + let input_eq_properties = input.equivalence_properties(); let eq_properties = input_eq_properties.project(projection_mapping, schema); - // Calculate output partitioning, which needs to respect aliases: - let input_partition = input.output_partitioning(); - let output_partitioning = - input_partition.project(projection_mapping, &input_eq_properties); + let output_partitioning = input + .output_partitioning() + .project(projection_mapping, input_eq_properties); Ok(PlanProperties::new( eq_properties, From 98ff7d2bbe343147d730449957c0e780461dc593 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 00:11:07 +0300 Subject: [PATCH 112/167] Simplify DependencyMap - 3 --- .../physical-expr/src/equivalence/ordering.rs | 101 +++++++----------- .../src/equivalence/properties/dependency.rs | 5 +- .../src/equivalence/properties/joins.rs | 4 +- .../src/equivalence/properties/mod.rs | 10 +- 4 files changed, 49 insertions(+), 71 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 3f09270988ab7..43170e07b1844 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -17,6 +17,7 @@ use std::fmt::Display; use std::hash::Hash; +use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; @@ -27,18 +28,21 @@ use arrow::compute::SortOptions; use datafusion_common::HashSet; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -/// An `OrderingEquivalenceClass` object keeps track of different alternative -/// orderings than can describe a schema. For example, consider the following table: +/// An `OrderingEquivalenceClass` keeps track of distinct alternative orderings +/// than can describe a table. For example, consider the following table: /// /// ```text -/// |a|b|c|d| -/// |1|4|3|1| -/// |2|3|3|2| -/// |3|1|2|2| -/// |3|2|1|3| +/// ┌───┬───┬───┬───┐ +/// │ a │ b │ c │ d │ +/// ├───┼───┼───┼───┤ +/// │ 1 │ 4 │ 3 │ 1 │ +/// │ 2 │ 3 │ 3 │ 2 │ +/// │ 3 │ 1 │ 2 │ 2 │ +/// │ 3 │ 2 │ 1 │ 3 │ +/// └───┴───┴───┴───┘ /// ``` /// -/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table +/// Here, both `[a ASC, b ASC]` and `[c DESC, d ASC]` describe the table /// ordering. In this case, we say that these orderings are equivalent. #[derive(Clone, Debug, Default, Eq, PartialEq, Hash)] pub struct OrderingEquivalenceClass { @@ -63,37 +67,14 @@ impl OrderingEquivalenceClass { result } - /// Checks whether `ordering` is a member of this equivalence class. - pub fn contains(&self, ordering: &LexOrdering) -> bool { - self.orderings.contains(ordering) - } - - /// Checks whether this ordering equivalence class is empty. - pub fn is_empty(&self) -> bool { - self.orderings.is_empty() - } - - /// Returns an iterator over the equivalent orderings in this class. - /// - /// Note this class also implements [`IntoIterator`] to return an iterator - /// over owned [`LexOrdering`]s. - pub fn iter(&self) -> impl Iterator { - self.orderings.iter() - } - - /// Returns how many equivalent orderings there are in this class. - pub fn len(&self) -> usize { - self.orderings.len() - } - - /// Extend this ordering equivalence class with the `other` class. - pub fn extend(&mut self, other: Self) { - self.orderings.extend(other.orderings); + /// Extend this ordering equivalence class with the given orderings. + pub fn extend(&mut self, orderings: impl IntoIterator) { + self.orderings.extend(orderings); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); } - /// Adds new orderings into this ordering equivalence class + /// Adds new orderings into this ordering equivalence class. pub fn add_new_orderings( &mut self, sort_exprs: impl IntoIterator>, @@ -104,15 +85,7 @@ impl OrderingEquivalenceClass { self.remove_redundant_entries(); } - /// Adds a single ordering to the existing ordering equivalence class. - pub fn add_new_ordering( - &mut self, - ordering: impl IntoIterator, - ) { - self.add_new_orderings(std::iter::once(ordering)); - } - - /// Removes redundant orderings from this equivalence class. + /// Removes redundant orderings from this ordering equivalence class. /// /// For instance, if we already have the ordering `[a ASC, b ASC, c DESC]`, /// then there is no need to keep ordering `[a ASC, b ASC]` in the state. @@ -177,20 +150,14 @@ impl OrderingEquivalenceClass { .map(|o| o.collapse()) } - // Append orderings in `other` to all existing orderings in this equivalence - // class. + // Append orderings in `other` to all existing orderings in this ordering + // equivalence class. pub fn join_suffix(mut self, other: &Self) -> Self { let n_ordering = self.orderings.len(); - // Replicate entries before cross product + // Replicate entries before cross product: let n_cross = std::cmp::max(n_ordering, other.len() * n_ordering); - self.orderings = self - .orderings - .iter() - .cloned() - .cycle() - .take(n_cross) - .collect(); - // Suffix orderings of other to the current orderings. + self.orderings = self.orderings.into_iter().cycle().take(n_cross).collect(); + // Append sort expressions of `other` to the current orderings: for (outer_idx, ordering) in other.iter().enumerate() { let base = outer_idx * n_ordering; // Use the cross product index: @@ -300,7 +267,23 @@ impl OrderingEquivalenceClass { } } -/// Convert the `OrderingEquivalenceClass` into an iterator of LexOrderings +impl Deref for OrderingEquivalenceClass { + type Target = [LexOrdering]; + + fn deref(&self) -> &Self::Target { + self.orderings.as_slice() + } +} + +impl From> for OrderingEquivalenceClass { + fn from(orderings: Vec) -> Self { + let mut result = Self { orderings }; + result.remove_redundant_entries(); + result + } +} + +/// Convert the `OrderingEquivalenceClass` into an iterator of `LexOrdering`s. impl IntoIterator for OrderingEquivalenceClass { type Item = LexOrdering; type IntoIter = IntoIter; @@ -320,8 +303,7 @@ impl Display for OrderingEquivalenceClass { for ordering in iter { write!(f, ", [{}]", ordering)?; } - write!(f, "]")?; - Ok(()) + write!(f, "]") } } @@ -957,8 +939,7 @@ mod tests { for (orderings, expected) in test_cases { let orderings = convert_to_orderings(&orderings); let expected = convert_to_orderings(&expected); - let actual = OrderingEquivalenceClass::new(orderings.clone()); - let actual = actual.orderings; + let actual = OrderingEquivalenceClass::from(orderings.clone()); let err_msg = format!( "orderings: {:?}, expected: {:?}, actual :{:?}", orderings, expected, actual diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index dc20026f61675..1f9a0874c2a3e 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -193,9 +193,8 @@ pub struct DependencyMap { } impl DependencyMap { - /// Insert a new dependency `sort_expr` --> `dependency` into the map. - /// - /// If `target_sort_expr` is none, a new entry is created with empty dependencies. + /// Insert a new dependency of `sort_expr` (i.e. `dependency`) into the map + /// along with its target sort expression. pub fn insert( &mut self, sort_expr: PhysicalSortExpr, diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index d415198fede2c..d27ff2ca5c166 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -243,7 +243,7 @@ mod tests { ]; let orderings = convert_to_orderings(&orderings); // Right child ordering equivalences - let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); + let mut right_oeq_class = OrderingEquivalenceClass::from(orderings); let left_columns_len = 4; @@ -280,7 +280,7 @@ mod tests { vec![(col_z, option_asc), (col_w, option_asc)], ]; let orderings = convert_to_orderings(&orderings); - let expected = OrderingEquivalenceClass::new(orderings); + let expected = OrderingEquivalenceClass::from(orderings); assert_eq!(result, expected); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 5da5d3e400f67..b71bcc6ac708c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -399,7 +399,7 @@ impl EquivalenceProperties { if new_orderings.is_empty() { new_orderings.push(filtered_exprs); } - self.oeq_class = OrderingEquivalenceClass::new(new_orderings); + self.oeq_class = new_orderings.into(); } self } @@ -924,8 +924,6 @@ impl EquivalenceProperties { /// /// A new `ProjectionMapping` with normalized source expressions. fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { - // Construct the mapping where source expressions are normalized. In this way - // In the algorithms below we can work on exact equalities ProjectionMapping { map: mapping .iter() @@ -954,8 +952,8 @@ impl EquivalenceProperties { /// /// A vector of `LexOrdering` containing all valid orderings after projection. fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + // Normalize source expressions in the mapping: let mapping = self.normalized_mapping(mapping); - // Get dependency map for existing orderings: let mut oeq_class = self.normalized_oeq_class(); oeq_class = self.substitute_oeq_class(oeq_class, &mapping); @@ -1059,7 +1057,7 @@ impl EquivalenceProperties { /// and `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { let eq_group = self.eq_group.project(mapping); - let oeq_class = OrderingEquivalenceClass::new(self.projected_orderings(mapping)); + let oeq_class = self.projected_orderings(mapping).into(); let constraints = self.projected_constraints(mapping).unwrap_or_default(); Self { schema: output_schema, @@ -1243,7 +1241,7 @@ impl EquivalenceProperties { // class: self.schema = schema; self.eq_group = EquivalenceGroup::new(eq_classes); - self.oeq_class = OrderingEquivalenceClass::new(new_orderings); + self.oeq_class = new_orderings.into(); Ok(self) } } From 564ea229decbfd2fb872b881215fe0a579ed593c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 13:47:53 +0300 Subject: [PATCH 113/167] Incorporate Jay's suggestions --- .../tests/fuzz_cases/equivalence/utils.rs | 4 +- .../sort_preserving_repartition_fuzz.rs | 2 +- datafusion/datasource/src/memory.rs | 12 ++-- .../physical-expr/src/equivalence/class.rs | 28 ++++----- .../physical-expr/src/equivalence/mod.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 6 +- .../src/equivalence/projection.rs | 6 +- .../src/equivalence/properties/dependency.rs | 34 +++++------ .../src/equivalence/properties/joins.rs | 14 ++--- .../src/equivalence/properties/mod.rs | 49 ++++++++-------- .../src/equivalence/properties/union.rs | 2 +- .../physical-expr/src/window/standard.rs | 4 +- .../src/output_requirements.rs | 15 +++-- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 4 +- datafusion/physical-plan/src/test.rs | 16 +++--- datafusion/physical-plan/src/union.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 26 ++++----- .../sqllogictest/test_files/my_group_by.slt | 57 ------------------- 20 files changed, 110 insertions(+), 177 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/my_group_by.slt diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 5ba2eeca8bd0e..1fa20c5144586 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -120,7 +120,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti options: options_asc, }); - eq_properties.add_new_ordering(ordering); + eq_properties.add_ordering(ordering); } Ok((test_schema, eq_properties)) @@ -345,7 +345,7 @@ pub fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { ], ]; let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); Ok((test_schema, eq_properties)) } diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 30aa1799b7670..1ceddb6f83ab0 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -104,7 +104,7 @@ mod sp_repartition_fuzz_tests { options: options_asc, }); - eq_properties.add_new_ordering(ordering); + eq_properties.add_ordering(ordering); } Ok((test_schema, eq_properties)) diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index e601976cb9bbf..0fae3d362fa94 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -41,7 +41,9 @@ use datafusion_common::{ internal_err, plan_err, project_schema, Constraints, Result, ScalarValue, }; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::ProjectionMapping; +use datafusion_physical_expr::equivalence::{ + OrderingEquivalenceClass, ProjectionMapping, +}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; @@ -706,10 +708,10 @@ impl MemorySourceConfig { Arc::clone(&base_schema), sort_information, ); - sort_information = base_eqp - .project(&projection_mapping, Arc::clone(&self.projected_schema)) - .into_oeq_class() - .into(); + let proj_eqp = + base_eqp.project(&projection_mapping, Arc::clone(&self.projected_schema)); + let oeq_class: OrderingEquivalenceClass = proj_eqp.into(); + sort_information = oeq_class.into(); } self.sort_information = sort_information; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index b3ff8c8b76c3e..66f05b9da9814 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -16,6 +16,7 @@ // under the License. use std::fmt::Display; +use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; @@ -312,21 +313,6 @@ impl EquivalenceGroup { result } - /// Returns how many equivalence classes there are in this group. - pub fn len(&self) -> usize { - self.classes.len() - } - - /// Checks whether this equivalence group is empty. - pub fn is_empty(&self) -> bool { - self.classes.is_empty() - } - - /// Returns an iterator over the equivalence classes in this group. - pub fn iter(&self) -> impl Iterator { - self.classes.iter() - } - /// Returns an iterator over the equivalence classes in this group. pub fn iter_mut(&mut self) -> impl Iterator { self.classes.iter_mut() @@ -440,9 +426,9 @@ impl EquivalenceGroup { /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { - // Remove duplicate entries from each equivalence class: + // First, remove trivial equivalence classes: self.classes.retain(|cls| !cls.is_trivial()); - // Unify/bridge groups that have common expressions: + // Then, unify/bridge groups that have common expressions: self.bridge_classes() } @@ -753,6 +739,14 @@ impl EquivalenceGroup { } } +impl Deref for EquivalenceGroup { + type Target = [EquivalenceClass]; + + fn deref(&self) -> &Self::Target { + &self.classes + } +} + impl IntoIterator for EquivalenceGroup { type Item = EquivalenceClass; type IntoIter = IntoIter; diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 6d4bf84b33814..b5c1db40dfc73 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -195,7 +195,7 @@ mod tests { ], ]; let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); Ok((test_schema, eq_properties)) } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 43170e07b1844..d546bae1b6962 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -75,7 +75,7 @@ impl OrderingEquivalenceClass { } /// Adds new orderings into this ordering equivalence class. - pub fn add_new_orderings( + pub fn add_orderings( &mut self, sort_exprs: impl IntoIterator>, ) { @@ -632,7 +632,7 @@ mod tests { format!("error in test orderings: {orderings:?}, eq_group: {eq_group:?}, constants: {constants:?}, reqs: {reqs:?}, expected: {expected:?}"); let mut eq_properties = EquivalenceProperties::new(Arc::clone(&test_schema)); let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); let classes = eq_group .into_iter() .map(|eq_class| EquivalenceClass::new(eq_class.into_iter().cloned())); @@ -681,7 +681,7 @@ mod tests { let orderings = convert_to_orderings(&orderings); // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); // First entry in the tuple is required ordering, second entry is the expected flag // that indicates whether this required ordering is satisfied. diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 98cd371003d81..c6fd70fc111be 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -581,7 +581,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); let proj_exprs = proj_exprs .into_iter() @@ -784,7 +784,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); let orderings = convert_to_orderings(orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); let expected = convert_to_orderings(expected); @@ -929,7 +929,7 @@ mod tests { } let orderings = convert_to_orderings(&orderings); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); let expected = convert_to_orderings(&expected); diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 1f9a0874c2a3e..3644a838b49b4 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -467,7 +467,7 @@ mod tests { let mut input_properties = EquivalenceProperties::new(Arc::clone(&input_schema)); // add equivalent ordering [a, b, c, d] - input_properties.add_new_ordering([ + input_properties.add_ordering([ parse_sort_expr("a", &input_schema), parse_sort_expr("b", &input_schema), parse_sort_expr("c", &input_schema), @@ -475,7 +475,7 @@ mod tests { ]); // add equivalent ordering [a, c, b, d] - input_properties.add_new_ordering([ + input_properties.add_ordering([ parse_sort_expr("a", &input_schema), parse_sort_expr("c", &input_schema), parse_sort_expr("b", &input_schema), // NB b and c are swapped @@ -513,13 +513,13 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); eq_properties.add_equal_conditions(col_a_expr, Arc::clone(&col_c_expr))?; - eq_properties.add_new_orderings([ + eq_properties.add_orderings([ vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], ]); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); - expected_eqs.add_new_orderings([ + expected_eqs.add_orderings([ vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], ]); @@ -544,7 +544,7 @@ mod tests { let col_b = &col("b", &schema)?; let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -579,7 +579,7 @@ mod tests { let col_b = &col("b", &schema)?; let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([ + eq_properties.add_orderings([ vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, @@ -623,7 +623,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); // not satisfied orders - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -664,7 +664,7 @@ mod tests { // b=a (e.g they are aliases) eq_properties.add_equal_conditions(Arc::clone(col_b), Arc::clone(col_a))?; // [b ASC], [d ASC] - eq_properties.add_new_orderings([ + eq_properties.add_orderings([ vec![PhysicalSortExpr { expr: Arc::clone(col_b), options: option_asc, @@ -747,7 +747,7 @@ mod tests { nulls_first: true, }; // [d ASC, h DESC] also satisfies schema. - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, @@ -1108,7 +1108,7 @@ mod tests { // Assume existing ordering is [c ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr::new_default(Arc::clone(&col_c)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), @@ -1162,7 +1162,7 @@ mod tests { ] .into(); - eq_properties.add_new_ordering(initial_ordering.clone()); + eq_properties.add_ordering(initial_ordering.clone()); // Add equality condition c = a * b eq_properties.add_equal_conditions(col_c, a_times_b)?; @@ -1198,7 +1198,7 @@ mod tests { // Assume existing ordering is [concat(a, b) ASC, a ASC, b ASC] let mut eq_properties = EquivalenceProperties::new(Arc::clone(&schema)); - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr::new_default(Arc::clone(&a_concat_b)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_a)).asc(), PhysicalSortExpr::new_default(Arc::clone(&col_b)).asc(), @@ -1301,7 +1301,7 @@ mod tests { }; // Initial ordering: [a ASC, b DESC, c ASC] - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, @@ -1353,7 +1353,7 @@ mod tests { let asc = SortOptions::default(); // Initial ordering: [a ASC, c ASC] - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr { expr: Arc::clone(&col_a), options: asc, @@ -1401,7 +1401,7 @@ mod tests { }; // Initial ordering: [a ASC, b DESC] - eq_properties.add_new_ordering([ + eq_properties.add_ordering([ PhysicalSortExpr::new(Arc::clone(&col_a), asc), PhysicalSortExpr::new(Arc::clone(&col_b), desc), ]); @@ -1439,7 +1439,7 @@ mod tests { eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_d))?; // Orderings: [d ASC, a ASC], [e ASC] - eq_properties.add_new_orderings([ + eq_properties.add_orderings([ vec![ PhysicalSortExpr { expr: Arc::clone(&col_d), @@ -1625,7 +1625,7 @@ mod tests { expr: col(col_name, schema).unwrap(), options: SortOptions::default(), }); - eq_properties.add_new_ordering(base_ordering); + eq_properties.add_ordering(base_ordering); // Add constraints eq_properties = diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index d27ff2ca5c166..0a542aaaa2022 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -68,9 +68,9 @@ pub fn join_equivalence_properties( // then we should add `a ASC, b ASC` to the ordering equivalences // of the join output. let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); - result.add_ordering_equivalence_class(out_oeq_class); + result.extend_orderings(out_oeq_class); } else { - result.add_ordering_equivalence_class(left_oeq_class); + result.extend_orderings(left_oeq_class); } } [false, true] => { @@ -91,9 +91,9 @@ pub fn join_equivalence_properties( // then we should add `b ASC, a ASC` to the ordering equivalences // of the join output. let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); - result.add_ordering_equivalence_class(out_oeq_class); + result.extend_orderings(out_oeq_class); } else { - result.add_ordering_equivalence_class(right_oeq_class); + result.extend_orderings(right_oeq_class); } } [false, false] => {} @@ -193,8 +193,8 @@ mod tests { let left_orderings = convert_to_orderings(&left_orderings); let right_orderings = convert_to_orderings(&right_orderings); let expected = convert_to_orderings(&expected); - left_eq_properties.add_new_orderings(left_orderings); - right_eq_properties.add_new_orderings(right_orderings); + left_eq_properties.add_orderings(left_orderings); + right_eq_properties.add_orderings(right_orderings); let join_eq = join_equivalence_properties( left_eq_properties, right_eq_properties, @@ -271,7 +271,7 @@ mod tests { &join_type, left_columns_len, ); - join_eq_properties.add_ordering_equivalence_class(right_oeq_class); + join_eq_properties.extend_orderings(right_oeq_class); let result = join_eq_properties.oeq_class().clone(); // [x ASC, y ASC], [z ASC, w ASC] diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index e97b5c30b45a0..c36e9c4035cf1 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -179,35 +179,29 @@ impl EquivalenceProperties { &self.oeq_class } - /// Return the inner OrderingEquivalenceClass, consuming self - pub fn into_oeq_class(self) -> OrderingEquivalenceClass { - self.oeq_class - } - /// Returns a reference to the equivalence group within. pub fn eq_group(&self) -> &EquivalenceGroup { &self.eq_group } - /// Returns a reference to the constants within. + /// Returns a reference to the constraints within. + pub fn constraints(&self) -> &Constraints { + &self.constraints + } + + /// Returns all the known constants expressions. pub fn constants(&self) -> Vec { self.eq_group .iter() .filter_map(|c| { - c.canonical_expr().cloned().and_then(|expr| { - c.constant - .as_ref() - .map(|across| ConstExpr::new(expr, across.clone())) + c.constant.as_ref().and_then(|across| { + c.canonical_expr() + .map(|expr| ConstExpr::new(Arc::clone(expr), across.clone())) }) }) .collect() } - /// Returns a reference to the constraints within. - pub fn constraints(&self) -> &Constraints { - &self.constraints - } - /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { let mut sort_exprs: Vec<_> = self.oeq_class().output_ordering()?.into(); @@ -252,26 +246,23 @@ impl EquivalenceProperties { self.eq_group.clear_per_partition_constants(); } - /// Extends this `EquivalenceProperties` by adding the orderings inside the - /// ordering equivalence class `other`. - pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { + /// Extends this `EquivalenceProperties` by adding the orderings inside + /// collection `other`. + pub fn extend_orderings(&mut self, other: impl IntoIterator) { self.oeq_class.extend(other); } /// Adds new orderings into the existing ordering equivalence class. - pub fn add_new_orderings( + pub fn add_orderings( &mut self, orderings: impl IntoIterator>, ) { - self.oeq_class.add_new_orderings(orderings); + self.oeq_class.add_orderings(orderings); } /// Adds a single ordering to the existing ordering equivalence class. - pub fn add_new_ordering( - &mut self, - ordering: impl IntoIterator, - ) { - self.add_new_orderings(std::iter::once(ordering)); + pub fn add_ordering(&mut self, ordering: impl IntoIterator) { + self.add_orderings(std::iter::once(ordering)); } /// Incorporates the given equivalence group to into the existing @@ -371,7 +362,7 @@ impl EquivalenceProperties { } } - self.oeq_class.add_new_orderings(new_orderings); + self.oeq_class.add_orderings(new_orderings); Ok(()) } @@ -1246,6 +1237,12 @@ impl EquivalenceProperties { } } +impl From for OrderingEquivalenceClass { + fn from(eq_properties: EquivalenceProperties) -> Self { + eq_properties.oeq_class + } +} + /// More readable display version of the `EquivalenceProperties`. /// /// Format: diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 7733e7bcfa707..7d69d669a3992 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -77,7 +77,7 @@ fn calculate_union_binary( let mut eq_properties = EquivalenceProperties::new(lhs.schema); eq_properties.add_constants(constants); - eq_properties.add_new_orderings(orderings); + eq_properties.add_orderings(orderings); Ok(eq_properties) } diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 009c55c793498..26deee65925b1 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -282,7 +282,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( ) { if partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eqp.add_new_ordering([expr]); + eqp.add_ordering([expr]); } else { // If we have a PARTITION BY, standard functions can not introduce // a global ordering unless the existing ordering is compatible @@ -293,7 +293,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( let (mut ordering, _) = eqp.find_longest_permutation(partition_by); if ordering.len() == partition_by.len() { ordering.push(expr); - eqp.add_new_ordering(ordering); + eqp.add_ordering(ordering); } } } diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index d2a6179171ead..4103f2a929db9 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -308,10 +308,11 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - // In case of constant columns, output ordering of SortExec would give an empty set. - // Therefore; we check the sort expression field of the SortExec to assign the requirements. + // In case of constant columns, output ordering of the `SortExec` would + // be an empty set. Therefore; we check the sort expression field to + // assign the requirements. let req_ordering = sort_exec.expr(); - let req_dist = sort_exec.required_input_distribution()[0].clone(); + let req_dist = sort_exec.required_input_distribution().swap_remove(0); let reqs = OrderingRequirements::from(req_ordering.clone()); Ok(( Arc::new(OutputRequirementExec::new(plan, Some(reqs), req_dist)) as _, @@ -328,11 +329,9 @@ fn require_top_ordering_helper( true, )) } else if plan.maintains_input_order()[0] - && (plan.required_input_ordering()[0].is_none() - || matches!( - plan.required_input_ordering()[0].clone().unwrap(), - OrderingRequirements::Soft(_) - )) + && (plan.required_input_ordering()[0] + .as_ref() + .is_none_or(|o| matches!(o, OrderingRequirements::Soft(_)))) { // Keep searching for a `SortExec` as long as ordering is maintained, // and on-the-way operators do not themselves require an ordering. diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3039126cf9f9e..c72d6691abf28 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -654,7 +654,7 @@ impl AggregateExec { return false; } // ensure no ordering is required on the input - if let Some(requirement) = self.required_input_ordering()[0].clone() { + if let Some(requirement) = self.required_input_ordering().swap_remove(0) { return matches!(requirement, OrderingRequirements::Hard(_)); } true diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 1d06db3892612..f3ca011300d12 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1396,7 +1396,7 @@ mod tests { impl SortedUnboundedExec { fn compute_properties(schema: SchemaRef) -> PlanProperties { let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_ordering([PhysicalSortExpr::new_default(Arc::new( + eq_properties.add_ordering([PhysicalSortExpr::new_default(Arc::new( Column::new("c1", 0), ))]); PlanProperties::new( diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 6c5dd7cfaea20..f68a85de13a28 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -159,7 +159,7 @@ impl SortPreservingMergeExec { ) -> PlanProperties { let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_per_partition_constants(); - eq_properties.add_new_ordering(ordering); + eq_properties.add_ordering(ordering); PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning @@ -1307,7 +1307,7 @@ mod tests { .map(|(i, f)| Arc::new(Column::new(f.name(), i)) as Arc) .collect::>(); let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_ordering( + eq_properties.add_ordering( columns .iter() .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))), diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index ec38f04de6e44..e4789de1417ff 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -40,10 +40,12 @@ use datafusion_common::{ config::ConfigOptions, internal_err, project_schema, Result, Statistics, }; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{ - equivalence::ProjectionMapping, expressions::Column, utils::collect_columns, - EquivalenceProperties, LexOrdering, Partitioning, +use datafusion_physical_expr::equivalence::{ + OrderingEquivalenceClass, ProjectionMapping, }; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, Partitioning}; use futures::{Future, FutureExt}; @@ -329,10 +331,10 @@ impl TestMemoryExec { Arc::clone(&base_schema), sort_information, ); - sort_information = base_eqp - .project(&projection_mapping, Arc::clone(&self.projected_schema)) - .into_oeq_class() - .into(); + let proj_eqp = + base_eqp.project(&projection_mapping, Arc::clone(&self.projected_schema)); + let oeq_class: OrderingEquivalenceClass = proj_eqp.into(); + sort_information = oeq_class.into(); } self.sort_information = sort_information; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index b0975312abd5d..da09b37c3a28b 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -852,7 +852,7 @@ mod tests { ))); let mut union_expected_eq = EquivalenceProperties::new(Arc::clone(&schema)); - union_expected_eq.add_new_orderings(union_expected_orderings); + union_expected_eq.add_orderings(union_expected_orderings); let union = UnionExec::new(vec![child1, child2]); let union_eq_properties = union.properties().equivalence_properties(); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 2cb2aa5091b3f..ad9fecd4b4544 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -405,7 +405,7 @@ pub(crate) fn window_equivalence_properties( existing }) }); - window_eq_properties.add_new_orderings(new_lexs); + window_eq_properties.add_orderings(new_lexs); } } else { // The window frame is ever expanding, so set monotonicity comes @@ -431,7 +431,7 @@ pub(crate) fn window_equivalence_properties( let window_col = Column::new(expr.name(), i + input_schema_len); if no_partitioning { // Reverse set-monotonic cases with no partitioning: - window_eq_properties.add_new_ordering([PhysicalSortExpr::new( + window_eq_properties.add_ordering([PhysicalSortExpr::new( Arc::new(window_col), SortOptions::new(increasing, true), )]); @@ -442,7 +442,7 @@ pub(crate) fn window_equivalence_properties( Arc::new(window_col.clone()), SortOptions::new(increasing, true), )); - window_eq_properties.add_new_ordering(lex); + window_eq_properties.add_ordering(lex); } } } @@ -471,19 +471,15 @@ pub(crate) fn window_equivalence_properties( set_monotonicity.eq(&SetMonotonicity::Increasing); let window_col = Column::new(expr.name(), i + input_schema_len); if increasing && (asc || no_partitioning) { - window_eq_properties.add_new_ordering([ - PhysicalSortExpr::new( - Arc::new(window_col), - SortOptions::new(false, false), - ), - ]); + window_eq_properties.add_ordering([PhysicalSortExpr::new( + Arc::new(window_col), + SortOptions::new(false, false), + )]); } else if !increasing && (!asc || no_partitioning) { - window_eq_properties.add_new_ordering([ - PhysicalSortExpr::new( - Arc::new(window_col), - SortOptions::new(true, false), - ), - ]); + window_eq_properties.add_ordering([PhysicalSortExpr::new( + Arc::new(window_col), + SortOptions::new(true, false), + )]); }; } } diff --git a/datafusion/sqllogictest/test_files/my_group_by.slt b/datafusion/sqllogictest/test_files/my_group_by.slt deleted file mode 100644 index 29e1cd425cf22..0000000000000 --- a/datafusion/sqllogictest/test_files/my_group_by.slt +++ /dev/null @@ -1,57 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# create a table for testing -statement ok -CREATE TABLE sales_global (zip_code INT, - country VARCHAR(3), - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount FLOAT - ) as VALUES - (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), - (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), - (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) - -query TT -EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, - FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, - LAST_VALUE(amount ORDER BY amount DESC) AS fv2 - FROM sales_global - GROUP BY country - ORDER BY country ----- -logical_plan -01)Sort: sales_global.country ASC NULLS LAST -02)--Projection: sales_global.country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 -03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -04)------TableScan: sales_global projection=[country, amount] -physical_plan -01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] -02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] -09)----------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -10)------------------DataSourceExec: partitions=1, partition_sizes=[1] From 0fc3465d3b1674f8c3a831eb9e6873736b5057b5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 14:55:28 +0300 Subject: [PATCH 114/167] Simplifications --- .../src/equivalence/properties/mod.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index c36e9c4035cf1..62bb68d6cb892 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -207,10 +207,9 @@ impl EquivalenceProperties { let mut sort_exprs: Vec<_> = self.oeq_class().output_ordering()?.into(); // Prune out constant expressions: sort_exprs.retain(|sort_expr| { - let Some(cls) = self.eq_group.get_equivalence_class(&sort_expr.expr) else { - return true; - }; - cls.constant.is_none() + self.eq_group + .get_equivalence_class(&sort_expr.expr) + .is_none_or(|cls| cls.constant.is_none()) }); LexOrdering::new(sort_exprs) } @@ -219,12 +218,12 @@ impl EquivalenceProperties { /// Normalization removes constants and duplicates as well as standardizing /// expressions according to the equivalence group within. pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { - OrderingEquivalenceClass::new( - self.oeq_class - .iter() - .cloned() - .filter_map(|ordering| self.normalize_sort_exprs(ordering)), - ) + self.oeq_class + .iter() + .cloned() + .filter_map(|ordering| self.normalize_sort_exprs(ordering)) + .collect::>() + .into() } /// Extends this `EquivalenceProperties` with the `other` object. From 41718877cecac96f582a1d8114d66ec5de81d081 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 16:01:15 +0300 Subject: [PATCH 115/167] Fix doctest --- datafusion/physical-expr/src/equivalence/properties/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 62bb68d6cb892..e1bd19f074fd3 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -120,7 +120,7 @@ use itertools::Itertools; /// // with a single constant value of b /// let mut eq_properties = EquivalenceProperties::new(schema); /// eq_properties.add_constants(vec![ConstExpr::from(col_b)]); -/// eq_properties.add_new_ordering([ +/// eq_properties.add_ordering([ /// PhysicalSortExpr::new_default(col_a).asc(), /// PhysicalSortExpr::new_default(col_c).desc(), /// ]); From 47d5c726608daf5fb249bbc1f5662bad2f819489 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 16:06:46 +0300 Subject: [PATCH 116/167] Improve docstrings --- datafusion/physical-expr/src/equivalence/class.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 66f05b9da9814..d15cf2debba65 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -84,10 +84,9 @@ impl Display for AcrossPartitions { /// ``` #[derive(Clone, Debug)] pub struct ConstExpr { - /// The expression that is known to be constant (e.g. a `Column`) + /// The expression that is known to be constant (e.g. a `Column`). pub expr: Arc, - /// Does the constant have the same value across all partitions? See - /// struct docs for more details + /// Indicates whether the constant have the same value across all partitions. pub across_partitions: AcrossPartitions, } From ece9d5f93a5edc08f2c3a25c8008fc91a01f8449 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 28 Apr 2025 17:11:58 +0300 Subject: [PATCH 117/167] Update/cast the constant value accordingly when schema changes --- datafusion/physical-expr/src/equivalence/class.rs | 5 ++++- .../physical-expr/src/equivalence/properties/mod.rs | 12 +++++++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index d15cf2debba65..237bef6876e90 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -89,6 +89,10 @@ pub struct ConstExpr { /// Indicates whether the constant have the same value across all partitions. pub across_partitions: AcrossPartitions, } +// TODO: The `ConstExpr` definition above can be in an inconsistent state where +// `expr` is a literal but `across_partitions` is not `Uniform`. Consider +// a refactor to ensure that `ConstExpr` is always in a consistent state +// (either by changing type definition, or by API constraints). impl ConstExpr { /// Create a new constant expression from a physical expression, specifying @@ -102,7 +106,6 @@ impl ConstExpr { let mut result = ConstExpr::from(expr); // Override the across partitions specification if the expression is not // a literal. - // TODO: Consider dropping across partitions from the constructor. if result.across_partitions == AcrossPartitions::Heterogeneous { result.across_partitions = across_partitions; } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index e1bd19f074fd3..bee802dc218db 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1200,12 +1200,22 @@ impl EquivalenceProperties { // Rewrite equivalence classes according to the new schema: let mut eq_classes = vec![]; for mut eq_class in self.eq_group { + // Rewrite the expressions in the equivalence class: eq_class.exprs = eq_class .exprs .into_iter() .map(|expr| with_new_schema(expr, &schema)) .collect::>()?; - // TODO: Also change the data type of the constant value if it exists. + // Rewrite the constant value (if available and known): + let data_type = eq_class + .canonical_expr() + .map(|e| e.data_type(&schema)) + .transpose()?; + if let (Some(data_type), Some(AcrossPartitions::Uniform(Some(value)))) = + (data_type, &mut eq_class.constant) + { + *value = value.cast_to(&data_type)?; + } eq_classes.push(eq_class); } From be45b96cd43bf81903ff1b930b5e885dbaea4c6a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 29 Apr 2025 12:10:33 +0300 Subject: [PATCH 118/167] Improve ProjectionMapping --- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 59 +++++++++-------- .../physical-expr/src/equivalence/mod.rs | 2 +- .../src/equivalence/projection.rs | 64 ++++++++++++++++--- .../src/equivalence/properties/mod.rs | 17 ++--- .../physical-plan/src/aggregates/mod.rs | 3 +- 6 files changed, 94 insertions(+), 53 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 1fa20c5144586..8f6b55ab1cf50 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -49,7 +49,7 @@ pub fn output_schema( for (source, targets) in mapping.iter() { let data_type = source.data_type(input_schema)?; let nullable = source.nullable(input_schema)?; - for (target, _) in targets { + for (target, _) in targets.iter() { let Some(column) = target.as_any().downcast_ref::() else { return plan_err!("Expects to have column"); }; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 237bef6876e90..c765683a28027 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -523,20 +523,20 @@ impl EquivalenceGroup { ) -> Option> { // First, we try to project expressions with an exact match. If we are // unable to do this, we consult equivalence classes. - if let Some(target) = mapping.get(expr) { + if let Some(targets) = mapping.get(expr) { // If we match the source, we can project directly: - let (target, _) = target.first().unwrap(); + let (target, _) = targets.first(); return Some(Arc::clone(target)); } else { // If the given expression is not inside the mapping, try to project // expressions considering the equivalence classes. - for (source, target) in mapping.iter() { + for (source, targets) in mapping.iter() { // If we match an equivalent expression to `source`, then we can // project. For example, if we have the mapping `(a as a1, a + c)` // and the equivalence class `(a, b)`, expression `b` projects to `a1`. let eq_class = self.get_equivalence_class(source); if eq_class.is_some_and(|group| group.contains(expr)) { - let (target, _) = target.first().unwrap(); + let (target, _) = targets.first(); return Some(Arc::clone(target)); } } @@ -544,7 +544,7 @@ impl EquivalenceGroup { // Project a non-leaf expression by projecting its children. let children = expr.children(); if children.is_empty() { - // Leaf expression should be inside mapping. + // A leaf expression should be inside the mapping. return None; } children @@ -575,12 +575,12 @@ impl EquivalenceGroup { // merge all equivalent expressions into the classes. let normalized_expr = self.normalize_expr(Arc::clone(source)); let cls = new_classes.entry(normalized_expr).or_default(); - for (target, _) in targets { + for (target, _) in targets.iter() { cls.push(Arc::clone(target)); } // Save new constants arising from the projection: if let Some(across) = self.is_expr_constant(source) { - for (target, _) in targets { + for (target, _) in targets.iter() { let const_expr = ConstExpr::new(Arc::clone(target), across.clone()); new_constants.push(const_expr); } @@ -1073,29 +1073,28 @@ mod tests { Field::new("b+c", DataType::Int32, false), ])); - let mapping = ProjectionMapping { - map: [ - ( - binary( - col("a", &schema)?, - Operator::Plus, - col("c", &schema)?, - &schema, - )?, - vec![(col("a+c", &projected_schema)?, 0)], - ), - ( - binary( - col("b", &schema)?, - Operator::Plus, - col("c", &schema)?, - &schema, - )?, - vec![(col("b+c", &projected_schema)?, 1)], - ), - ] - .into(), - }; + let mapping = [ + ( + binary( + col("a", &schema)?, + Operator::Plus, + col("c", &schema)?, + &schema, + )?, + vec![(col("a+c", &projected_schema)?, 0)].into(), + ), + ( + binary( + col("b", &schema)?, + Operator::Plus, + col("c", &schema)?, + &schema, + )?, + vec![(col("b+c", &projected_schema)?, 1)].into(), + ), + ] + .into_iter() + .collect::(); let projected = group.project(&mapping); diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index b5c1db40dfc73..3e30e76acc34d 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -126,7 +126,7 @@ mod tests { for (source, targets) in mapping.iter() { let data_type = source.data_type(input_schema)?; let nullable = source.nullable(input_schema)?; - for (target, _) in targets { + for (target, _) in targets.iter() { let Some(column) = target.as_any().downcast_ref::() else { return plan_err!("Expects to have column"); }; diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index c6fd70fc111be..4559ade54cd7a 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::ops::Deref; +use std::ops::{Deref, DerefMut}; use std::sync::Arc; use crate::expressions::Column; @@ -27,14 +27,50 @@ use datafusion_common::{internal_err, Result}; use indexmap::IndexMap; +/// Stores target expressions, along with their indices, that associate with a +/// source expression in a projection mapping. +#[derive(Clone, Debug, Default)] +pub struct ProjectionTargets { + /// A non-empty vector of pairs of target expressions and their indices. + /// Consider using a special non-empty collection type in the future (e.g. + /// if Rust provides one in the standard library). + exprs_indices: Vec<(Arc, usize)>, +} + +impl ProjectionTargets { + pub fn first(&self) -> &(Arc, usize) { + // Since the vector is non-empty, we can safely unwrap: + self.exprs_indices.first().unwrap() + } +} + +impl Deref for ProjectionTargets { + type Target = Vec<(Arc, usize)>; + + fn deref(&self) -> &Self::Target { + &self.exprs_indices + } +} + +impl DerefMut for ProjectionTargets { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.exprs_indices + } +} + +impl From, usize)>> for ProjectionTargets { + fn from(exprs_indices: Vec<(Arc, usize)>) -> Self { + Self { exprs_indices } + } +} + /// Stores the mapping between source expressions and target expressions for a /// projection. -#[derive(Debug, Clone)] +#[derive(Clone, Debug)] pub struct ProjectionMapping { /// Mapping between source expressions and target expressions. /// Vector indices correspond to the indices after projection. - #[allow(clippy::type_complexity)] - pub map: IndexMap, Vec<(Arc, usize)>>, + map: IndexMap, ProjectionTargets>, } impl ProjectionMapping { @@ -46,8 +82,8 @@ impl ProjectionMapping { /// projection mapping would be: /// /// ```text - /// [0]: (c + d, col("c + d")) - /// [1]: (a + b, col("a + b")) + /// [0]: (c + d, [(col("c + d"), 0)]) + /// [1]: (a + b, [(col("a + b"), 1)]) /// ``` /// /// where `col("c + d")` means the column named `"c + d"`. @@ -56,7 +92,7 @@ impl ProjectionMapping { input_schema: &SchemaRef, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - let mut map = IndexMap::new(); + let mut map = IndexMap::<_, ProjectionTargets>::new(); for (expr_idx, (expr, name)) in expr.into_iter().enumerate() { let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; let source_expr = expr.transform_down(|e| match e.as_any().downcast_ref::() { @@ -82,7 +118,7 @@ impl ProjectionMapping { }) .data()?; map.entry(source_expr) - .or_insert_with(Vec::new) + .or_default() .push((target_expr, expr_idx)); } Ok(Self { map }) @@ -103,13 +139,23 @@ impl ProjectionMapping { } impl Deref for ProjectionMapping { - type Target = IndexMap, Vec<(Arc, usize)>>; + type Target = IndexMap, ProjectionTargets>; fn deref(&self) -> &Self::Target { &self.map } } +impl FromIterator<(Arc, ProjectionTargets)> for ProjectionMapping { + fn from_iter, ProjectionTargets)>>( + iter: T, + ) -> Self { + Self { + map: IndexMap::from_iter(iter), + } + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index bee802dc218db..0205c6e752560 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -914,16 +914,13 @@ impl EquivalenceProperties { /// /// A new `ProjectionMapping` with normalized source expressions. fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { - ProjectionMapping { - map: mapping - .iter() - .map(|(source, target)| { - let normalized_source = - self.eq_group.normalize_expr(Arc::clone(source)); - (normalized_source, target.to_vec()) - }) - .collect(), - } + mapping + .iter() + .map(|(source, target)| { + let normalized_source = self.eq_group.normalize_expr(Arc::clone(source)); + (normalized_source, target.clone()) + }) + .collect::() } /// Computes projected orderings based on a given projection mapping. diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c72d6691abf28..32e6c90f5e665 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -676,7 +676,7 @@ impl AggregateExec { // If the group by is empty, then we ensure that the operator will produce // only one row, and mark the generated result as a constant value. - if group_expr_mapping.map.is_empty() { + if group_expr_mapping.is_empty() { let mut constants = eq_properties.constants().to_vec(); let new_constants = aggr_exprs.iter().enumerate().map(|(idx, func)| { let column = Arc::new(Column::new(func.name(), idx)); @@ -691,7 +691,6 @@ impl AggregateExec { let mut constraints = eq_properties.constraints().to_vec(); let new_constraint = Constraint::Unique( group_expr_mapping - .map .iter() .flat_map(|(_, target_cols)| { target_cols.iter().flat_map(|(expr, _)| { From 7f2b0db4c166f3974f5c0472257ad8a12a50eeeb Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 29 Apr 2025 12:57:11 +0300 Subject: [PATCH 119/167] Remove DerefMut from ProjectionTargets to preserve non-emptiness --- .../physical-expr/src/equivalence/projection.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 4559ade54cd7a..f4045e59da56e 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::ops::{Deref, DerefMut}; +use std::ops::Deref; use std::sync::Arc; use crate::expressions::Column; @@ -42,22 +42,20 @@ impl ProjectionTargets { // Since the vector is non-empty, we can safely unwrap: self.exprs_indices.first().unwrap() } + + pub fn push(&mut self, target: (Arc, usize)) { + self.exprs_indices.push(target); + } } impl Deref for ProjectionTargets { - type Target = Vec<(Arc, usize)>; + type Target = [(Arc, usize)]; fn deref(&self) -> &Self::Target { &self.exprs_indices } } -impl DerefMut for ProjectionTargets { - fn deref_mut(&mut self) -> &mut Self::Target { - &mut self.exprs_indices - } -} - impl From, usize)>> for ProjectionTargets { fn from(exprs_indices: Vec<(Arc, usize)>) -> Self { Self { exprs_indices } From cb0972db2b232cd153f510b0cf689670cdedba8a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 29 Apr 2025 17:14:43 +0300 Subject: [PATCH 120/167] Docstring --- datafusion/physical-expr/src/equivalence/projection.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index f4045e59da56e..493e9d12bd964 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -38,11 +38,13 @@ pub struct ProjectionTargets { } impl ProjectionTargets { + /// Returns the first target expression and its index. pub fn first(&self) -> &(Arc, usize) { // Since the vector is non-empty, we can safely unwrap: self.exprs_indices.first().unwrap() } + /// Adds a target expression and its index to the list of targets. pub fn push(&mut self, target: (Arc, usize)) { self.exprs_indices.push(target); } From 9f898ec25c3a74840ab9f3c93e7f016d8b235bba Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 30 Apr 2025 11:13:44 +0300 Subject: [PATCH 121/167] Optimize project_expr by fetching equivalence classes only once --- .../physical-expr/src/equivalence/class.rs | 80 +++++++++++++------ 1 file changed, 54 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index c765683a28027..8bd4184bef8c6 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -20,6 +20,7 @@ use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; +use super::projection::ProjectionTargets; use super::{add_offset_to_expr, ProjectionMapping}; use crate::expressions::{Column, Literal}; use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; @@ -511,6 +512,50 @@ impl EquivalenceGroup { sort_requirement } + /// Perform an indirect projection of `expr` by consulting the equivalence + /// classes. + fn project_expr_indirect( + aug_mapping: &IndexMap< + &Arc, + (&ProjectionTargets, Option<&EquivalenceClass>), + >, + expr: &Arc, + ) -> Option> { + // The given expression is not inside the mapping, so we try to project + // indirectly using equivalence classes. + for (targets, eq_class) in aug_mapping.values() { + // If we match an equivalent expression to a source expression in + // the mapping, then we can project. For example, if we have the + // mapping `(a as a1, a + c)` and the equivalence `a == b`, + // expression `b` projects to `a1`. + if eq_class.as_ref().is_some_and(|group| group.contains(expr)) { + let (target, _) = targets.first(); + return Some(Arc::clone(target)); + } + } + // Project a non-leaf expression by projecting its children. + let children = expr.children(); + if children.is_empty() { + // A leaf expression should be inside the mapping. + return None; + } + children + .into_iter() + .map(|child| { + // First, we try to project children with an exact match. If + // we are unable to do this, we consult equivalence classes. + if let Some((targets, _)) = aug_mapping.get(child) { + // If we match the source, we can project directly: + let (target, _) = targets.first(); + Some(Arc::clone(target)) + } else { + Self::project_expr_indirect(aug_mapping, child) + } + }) + .collect::>>() + .map(|children| Arc::clone(expr).with_new_children(children).unwrap()) + } + /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. /// @@ -521,37 +566,20 @@ impl EquivalenceGroup { mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - // First, we try to project expressions with an exact match. If we are - // unable to do this, we consult equivalence classes. if let Some(targets) = mapping.get(expr) { // If we match the source, we can project directly: let (target, _) = targets.first(); - return Some(Arc::clone(target)); + Some(Arc::clone(target)) } else { - // If the given expression is not inside the mapping, try to project - // expressions considering the equivalence classes. - for (source, targets) in mapping.iter() { - // If we match an equivalent expression to `source`, then we can - // project. For example, if we have the mapping `(a as a1, a + c)` - // and the equivalence class `(a, b)`, expression `b` projects to `a1`. - let eq_class = self.get_equivalence_class(source); - if eq_class.is_some_and(|group| group.contains(expr)) { - let (target, _) = targets.first(); - return Some(Arc::clone(target)); - } - } - } - // Project a non-leaf expression by projecting its children. - let children = expr.children(); - if children.is_empty() { - // A leaf expression should be inside the mapping. - return None; + let aug_mapping = mapping + .iter() + .map(|(k, v)| { + let eq_class = self.get_equivalence_class(k); + (k, (v, eq_class)) + }) + .collect(); + Self::project_expr_indirect(&aug_mapping, expr) } - children - .into_iter() - .map(|child| self.project_expr(mapping, child)) - .collect::>>() - .map(|children| Arc::clone(expr).with_new_children(children).unwrap()) } /// Projects this equivalence group according to the given projection mapping. From 74f74b3eb253566e39ae791a5976f234519d1b60 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 30 Apr 2025 12:02:44 +0300 Subject: [PATCH 122/167] Project multiple expressions more efficiently at once --- .../physical-expr/src/equivalence/class.rs | 66 ++++++++++++++----- .../src/equivalence/properties/mod.rs | 22 +++++-- datafusion/physical-expr/src/partitioning.rs | 18 ++--- 3 files changed, 74 insertions(+), 32 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 8bd4184bef8c6..514eaaa59b3bb 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -300,6 +300,11 @@ impl From for Vec> { } } +type AugmentedMapping<'a> = IndexMap< + &'a Arc, + (&'a ProjectionTargets, Option<&'a EquivalenceClass>), +>; + /// A collection of distinct `EquivalenceClass`es #[derive(Clone, Debug, Default)] pub struct EquivalenceGroup { @@ -515,10 +520,7 @@ impl EquivalenceGroup { /// Perform an indirect projection of `expr` by consulting the equivalence /// classes. fn project_expr_indirect( - aug_mapping: &IndexMap< - &Arc, - (&ProjectionTargets, Option<&EquivalenceClass>), - >, + aug_mapping: &AugmentedMapping, expr: &Arc, ) -> Option> { // The given expression is not inside the mapping, so we try to project @@ -556,11 +558,21 @@ impl EquivalenceGroup { .map(|children| Arc::clone(expr).with_new_children(children).unwrap()) } + fn augment_projection_mapping<'a>( + &'a self, + mapping: &'a ProjectionMapping, + ) -> AugmentedMapping<'a> { + mapping + .iter() + .map(|(k, v)| { + let eq_class = self.get_equivalence_class(k); + (k, (v, eq_class)) + }) + .collect() + } + /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. - /// - /// TODO: Write a multiple `expr` version to avoid searching for equivalence - /// classes for every source expression in `mapping` multiple times. pub fn project_expr( &self, mapping: &ProjectionMapping, @@ -571,24 +583,42 @@ impl EquivalenceGroup { let (target, _) = targets.first(); Some(Arc::clone(target)) } else { - let aug_mapping = mapping - .iter() - .map(|(k, v)| { - let eq_class = self.get_equivalence_class(k); - (k, (v, eq_class)) - }) - .collect(); + let aug_mapping = self.augment_projection_mapping(mapping); Self::project_expr_indirect(&aug_mapping, expr) } } + /// Projects `expressions` according to the given projection mapping. + /// This function is similar to [`Self::project_expr`], but projects multiple + /// expressions at once more efficiently than calling `project_expr` for each + /// expression. + pub fn project_expressions<'a>( + &self, + mapping: &ProjectionMapping, + expressions: impl IntoIterator>, + ) -> Vec>> { + let mut aug_mapping = None; + expressions + .into_iter() + .map(|expr| { + if let Some(targets) = mapping.get(expr) { + // If we match the source, we can project directly: + let (target, _) = targets.first(); + Some(Arc::clone(target)) + } else { + let aug_mapping = aug_mapping + .get_or_insert_with(|| self.augment_projection_mapping(mapping)); + Self::project_expr_indirect(aug_mapping, expr) + } + }) + .collect() + } + /// Projects this equivalence group according to the given projection mapping. pub fn project(&self, mapping: &ProjectionMapping) -> Self { let projected_classes = self.iter().map(|cls| { - let new_exprs = cls - .iter() - .filter_map(|expr| self.project_expr(mapping, expr)); - EquivalenceClass::new(new_exprs) + let new_exprs = self.project_expressions(mapping, cls.iter()); + EquivalenceClass::new(new_exprs.into_iter().flatten()) }); // The key is the source expression, and the value is the equivalence diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 0205c6e752560..0a55b9eafbe0a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -791,11 +791,11 @@ impl EquivalenceProperties { OrderingEquivalenceClass::new(new_orderings) } - /// Projects argument `expr` according to `projection_mapping`, taking - /// equivalences into account. + /// Projects argument `expr` according to the projection described by + /// `mapping`, taking equivalences into account. /// /// For example, assume that columns `a` and `c` are always equal, and that - /// `projection_mapping` encodes following mapping: + /// the projection described by `mapping` encodes the following: /// /// ```text /// a -> a1 @@ -807,9 +807,21 @@ impl EquivalenceProperties { pub fn project_expr( &self, expr: &Arc, - projection_mapping: &ProjectionMapping, + mapping: &ProjectionMapping, ) -> Option> { - self.eq_group.project_expr(projection_mapping, expr) + self.eq_group.project_expr(mapping, expr) + } + + /// Projects the given `expressions` according to the projection described + /// by `mapping`, taking equivalences into account. This function is similar + /// to [`Self::project_expr`], but projects multiple expressions at once + /// more efficiently than calling `project_expr` for each expression. + pub fn project_expressions<'a>( + &self, + expressions: impl IntoIterator>, + mapping: &ProjectionMapping, + ) -> Vec>> { + self.eq_group.project_expressions(mapping, expressions) } /// Constructs a dependency map based on existing orderings referred to in diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index eb7e1ea6282bb..53bbd3be5dc27 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -199,18 +199,18 @@ impl Partitioning { /// Calculate the output partitioning after applying the given projection. pub fn project( &self, - projection_mapping: &ProjectionMapping, + mapping: &ProjectionMapping, input_eq_properties: &EquivalenceProperties, ) -> Self { if let Partitioning::Hash(exprs, part) = self { - let normalized_exprs = exprs - .iter() - .map(|expr| { - input_eq_properties - .project_expr(expr, projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) + let proj_exprs = input_eq_properties.project_expressions(exprs, mapping); + let normalized_exprs = proj_exprs + .into_iter() + .zip(exprs) + .map(|(proj_expr, expr)| { + proj_expr.unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) }) .collect(); Partitioning::Hash(normalized_exprs, *part) From 4839e6bd385d32bda8f291cfa204013b4f2fdefc Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 1 May 2025 11:19:42 +0300 Subject: [PATCH 123/167] Project multiple expressions more efficiently at once - 2 --- .../physical-expr/src/equivalence/class.rs | 35 +++++++++---------- .../src/equivalence/properties/mod.rs | 8 ++--- datafusion/physical-expr/src/partitioning.rs | 5 ++- 3 files changed, 22 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 514eaaa59b3bb..7afa9f0134e9a 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -593,32 +593,29 @@ impl EquivalenceGroup { /// expressions at once more efficiently than calling `project_expr` for each /// expression. pub fn project_expressions<'a>( - &self, - mapping: &ProjectionMapping, - expressions: impl IntoIterator>, - ) -> Vec>> { + &'a self, + mapping: &'a ProjectionMapping, + expressions: impl IntoIterator> + 'a, + ) -> impl Iterator>> + 'a { let mut aug_mapping = None; - expressions - .into_iter() - .map(|expr| { - if let Some(targets) = mapping.get(expr) { - // If we match the source, we can project directly: - let (target, _) = targets.first(); - Some(Arc::clone(target)) - } else { - let aug_mapping = aug_mapping - .get_or_insert_with(|| self.augment_projection_mapping(mapping)); - Self::project_expr_indirect(aug_mapping, expr) - } - }) - .collect() + expressions.into_iter().map(move |expr| { + if let Some(targets) = mapping.get(expr) { + // If we match the source, we can project directly: + let (target, _) = targets.first(); + Some(Arc::clone(target)) + } else { + let aug_mapping = aug_mapping + .get_or_insert_with(|| self.augment_projection_mapping(mapping)); + Self::project_expr_indirect(aug_mapping, expr) + } + }) } /// Projects this equivalence group according to the given projection mapping. pub fn project(&self, mapping: &ProjectionMapping) -> Self { let projected_classes = self.iter().map(|cls| { let new_exprs = self.project_expressions(mapping, cls.iter()); - EquivalenceClass::new(new_exprs.into_iter().flatten()) + EquivalenceClass::new(new_exprs.flatten()) }); // The key is the source expression, and the value is the equivalence diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 0a55b9eafbe0a..d5ccff1179f1d 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -817,10 +817,10 @@ impl EquivalenceProperties { /// to [`Self::project_expr`], but projects multiple expressions at once /// more efficiently than calling `project_expr` for each expression. pub fn project_expressions<'a>( - &self, - expressions: impl IntoIterator>, - mapping: &ProjectionMapping, - ) -> Vec>> { + &'a self, + expressions: impl IntoIterator> + 'a, + mapping: &'a ProjectionMapping, + ) -> impl Iterator>> + 'a { self.eq_group.project_expressions(mapping, expressions) } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 53bbd3be5dc27..d6b2b1b046f75 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -203,9 +203,8 @@ impl Partitioning { input_eq_properties: &EquivalenceProperties, ) -> Self { if let Partitioning::Hash(exprs, part) = self { - let proj_exprs = input_eq_properties.project_expressions(exprs, mapping); - let normalized_exprs = proj_exprs - .into_iter() + let normalized_exprs = input_eq_properties + .project_expressions(exprs, mapping) .zip(exprs) .map(|(proj_expr, expr)| { proj_expr.unwrap_or_else(|| { From 96ec3b42b9b0888ce36ae9c9f2684d682a96358f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 1 May 2025 11:48:29 +0300 Subject: [PATCH 124/167] Project multiple expressions more efficiently at once - 3 --- datafusion/common/src/functional_dependencies.rs | 14 +++++++------- .../src/equivalence/properties/mod.rs | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 2d2d15dfbd2e6..37c9c23ce570e 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -54,13 +54,13 @@ impl Constraints { Self { inner: constraints } } - /// Check whether constraints is empty - pub fn is_empty(&self) -> bool { - self.inner.is_empty() + /// Extends the current constraints with the given `other` constraints. + pub fn extend(&mut self, other: Constraints) { + self.inner.extend(other.inner); } - /// Projects constraints using the given projection indices. - /// Returns None if any of the constraint columns are not included in the projection. + /// Projects constraints using the given projection indices. Returns `None` + /// if any of the constraint columns are not included in the projection. pub fn project(&self, proj_indices: &[usize]) -> Option { let projected = self .inner @@ -70,14 +70,14 @@ impl Constraints { Constraint::PrimaryKey(indices) => { let new_indices = update_elements_with_matching_indices(indices, proj_indices); - // Only keep constraint if all columns are preserved + // Only keep the constraint if all columns are preserved: (new_indices.len() == indices.len()) .then_some(Constraint::PrimaryKey(new_indices)) } Constraint::Unique(indices) => { let new_indices = update_elements_with_matching_indices(indices, proj_indices); - // Only keep constraint if all columns are preserved + // Only keep the constraint if all columns are preserved: (new_indices.len() == indices.len()) .then_some(Constraint::Unique(new_indices)) } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index d5ccff1179f1d..c9f778efbea94 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -230,6 +230,7 @@ impl EquivalenceProperties { pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); + self.constraints.extend(other.constraints); self } @@ -431,8 +432,7 @@ impl EquivalenceProperties { let sort_exprs = sort_exprs .into_iter() .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) - .filter(|order| self.is_expr_constant(&order.expr).is_none()) - .collect::>(); + .filter(|order| self.is_expr_constant(&order.expr).is_none()); LexOrdering::new(sort_exprs).map(|o| o.collapse()) } From 82486ae9116053c6bf66f2b7684cf3b2b7ae1588 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 1 May 2025 12:21:43 +0300 Subject: [PATCH 125/167] Project multiple expressions more efficiently at once - 4 --- datafusion/physical-expr/src/equivalence/properties/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index c9f778efbea94..96e6541d23be5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -204,7 +204,7 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - let mut sort_exprs: Vec<_> = self.oeq_class().output_ordering()?.into(); + let mut sort_exprs: Vec<_> = self.oeq_class.output_ordering()?.into(); // Prune out constant expressions: sort_exprs.retain(|sort_expr| { self.eq_group From 51e7e806ad3708fb952dbd767024850bf4ee3410 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 3 May 2025 13:32:06 +0300 Subject: [PATCH 126/167] Move normalization of sort expressions to equivalence group --- .../physical-expr-common/src/sort_expr.rs | 104 +++++++++--------- .../physical-expr/src/equivalence/class.rs | 65 +++++++++-- .../src/equivalence/properties/mod.rs | 51 ++------- 3 files changed, 112 insertions(+), 108 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 97601af3b36ba..cf427c3dd9909 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -32,6 +32,8 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_expr_common::columnar_value::ColumnarValue; +use itertools::Itertools; + /// Represents Sort operation for a column in a RecordBatch /// /// Example: @@ -329,46 +331,42 @@ impl From for PhysicalSortExpr { /// descending order. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LexOrdering { - inner: Vec, + exprs: Vec, } impl LexOrdering { /// Creates a new [`LexOrdering`] from the given vector of sort expressions. /// If the vector is empty, returns `None`. - pub fn new(inner: impl IntoIterator) -> Option { - let inner = inner.into_iter().collect::>(); - (!inner.is_empty()).then(|| Self { inner }) + pub fn new(exprs: impl IntoIterator) -> Option { + let exprs = exprs.into_iter().collect::>(); + (!exprs.is_empty()).then(|| Self { exprs }) } /// Appends an element to the back of the `LexOrdering`. pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { - self.inner.push(physical_sort_expr) + self.exprs.push(physical_sort_expr) } /// Add all elements from `iter` to the `LexOrdering`. pub fn extend(&mut self, iter: impl IntoIterator) { - self.inner.extend(iter) + self.exprs.extend(iter) } /// Returns the number of elements that can be stored in the `LexOrdering` /// without reallocating. pub fn capacity(&self) -> usize { - self.inner.capacity() + self.exprs.capacity() } - /// Constructs a duplicate-free `LexOrdering` by filtering out duplicate - /// entries that have same physical expression inside. - /// - /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a - /// Some(ASC)]`. + /// Constructs a duplicate-free `LexOrdering` by filtering out entries with + /// the same physical expression inside. For example, `[a ASC, a DESC]` + /// collapses to `[a ASC]`. pub fn collapse(mut self) -> Self { - let mut orderings = Vec::::new(); - for element in self.inner { - if !orderings.iter().any(|item| item.expr.eq(&element.expr)) { - orderings.push(element); - } - } - self.inner = orderings; + self.exprs = self + .exprs + .into_iter() + .unique_by(|s| Arc::clone(&s.expr)) + .collect(); self } @@ -378,10 +376,10 @@ impl LexOrdering { /// to the number of expressions inside this `LexOrdering`, making truncation /// a no-op, or (2) when `len` is `0`, making truncation impossible. pub fn truncate(&mut self, len: usize) -> bool { - if len == 0 || len >= self.inner.len() { + if len == 0 || len >= self.exprs.len() { return false; } - self.inner.truncate(len); + self.exprs.truncate(len); true } } @@ -404,7 +402,7 @@ impl From<[PhysicalSortExpr; N]> for LexOrdering { // Rust supports it. assert!(N > 0); Self { - inner: value.to_vec(), + exprs: value.to_vec(), } } } @@ -413,20 +411,20 @@ impl Deref for LexOrdering { type Target = [PhysicalSortExpr]; fn deref(&self) -> &Self::Target { - self.inner.as_slice() + self.exprs.as_slice() } } impl DerefMut for LexOrdering { fn deref_mut(&mut self) -> &mut Self::Target { - self.inner.as_mut_slice() + self.exprs.as_mut_slice() } } impl Display for LexOrdering { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut first = true; - for sort_expr in &self.inner { + for sort_expr in &self.exprs { if first { first = false; } else { @@ -443,7 +441,7 @@ impl IntoIterator for LexOrdering { type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { - self.inner.into_iter() + self.exprs.into_iter() } } @@ -452,13 +450,13 @@ impl<'a> IntoIterator for &'a LexOrdering { type IntoIter = std::slice::Iter<'a, PhysicalSortExpr>; fn into_iter(self) -> Self::IntoIter { - self.inner.iter() + self.exprs.iter() } } impl From for Vec { fn from(ordering: LexOrdering) -> Self { - ordering.inner + ordering.exprs } } @@ -466,46 +464,42 @@ impl From for Vec { /// represents a lexicographical ordering requirement. #[derive(Debug, Clone, PartialEq)] pub struct LexRequirement { - inner: Vec, + reqs: Vec, } impl LexRequirement { /// Creates a new [`LexRequirement`] from the given vector of sort expressions. /// If the vector is empty, returns `None`. - pub fn new(inner: impl IntoIterator) -> Option { - let inner = inner.into_iter().collect::>(); - (!inner.is_empty()).then(|| Self { inner }) + pub fn new(reqs: impl IntoIterator) -> Option { + let reqs = reqs.into_iter().collect::>(); + (!reqs.is_empty()).then(|| Self { reqs }) } /// Appends an element to the back of the `LexRequirement`. pub fn push(&mut self, requirement: PhysicalSortRequirement) { - self.inner.push(requirement) + self.reqs.push(requirement) } /// Add all elements from `iter` to the `LexRequirement`. pub fn extend(&mut self, iter: impl IntoIterator) { - self.inner.extend(iter) + self.reqs.extend(iter) } /// Returns the number of elements that can be stored in the `LexRequirement` /// without reallocating. pub fn capacity(&self) -> usize { - self.inner.capacity() + self.reqs.capacity() } - /// Constructs a duplicate-free `LexRequirement` by filtering out duplicate - /// entries that have same physical expression inside. - /// - /// For example, `vec![a Some(ASC), a Some(DESC)]` collapses to `vec![a - /// Some(ASC)]`. + /// Constructs a duplicate-free `LexRequirement` by filtering out entries + /// with the same physical expression inside. For example, the requirement + /// `[a Some(ASC), a None]` collapses to `[a Some(ASC)]`. pub fn collapse(mut self) -> Self { - let mut reqs = Vec::::new(); - for element in self.inner { - if !reqs.iter().any(|item| item.expr.eq(&element.expr)) { - reqs.push(element); - } - } - self.inner = reqs; + self.reqs = self + .reqs + .into_iter() + .unique_by(|r| Arc::clone(&r.expr)) + .collect(); self } } @@ -516,7 +510,7 @@ impl From<[PhysicalSortRequirement; N]> for LexRequirement { // Rust supports it. assert!(N > 0); Self { - inner: value.to_vec(), + reqs: value.to_vec(), } } } @@ -525,13 +519,13 @@ impl Deref for LexRequirement { type Target = [PhysicalSortRequirement]; fn deref(&self) -> &Self::Target { - self.inner.as_slice() + self.reqs.as_slice() } } impl DerefMut for LexRequirement { fn deref_mut(&mut self) -> &mut Self::Target { - self.inner.as_mut_slice() + self.reqs.as_mut_slice() } } @@ -540,7 +534,7 @@ impl IntoIterator for LexRequirement { type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { - self.inner.into_iter() + self.reqs.into_iter() } } @@ -549,13 +543,13 @@ impl<'a> IntoIterator for &'a LexRequirement { type IntoIter = std::slice::Iter<'a, PhysicalSortRequirement>; fn into_iter(self) -> Self::IntoIter { - self.inner.iter() + self.reqs.iter() } } impl From for Vec { fn from(requirement: LexRequirement) -> Self { - requirement.inner + requirement.reqs } } @@ -564,7 +558,7 @@ impl From for LexRequirement { fn from(value: LexOrdering) -> Self { // Can construct directly as `value` is non-degenerate: Self { - inner: value.into_iter().map(Into::into).collect(), + reqs: value.into_iter().map(Into::into).collect(), } } } @@ -573,7 +567,7 @@ impl From for LexOrdering { fn from(value: LexRequirement) -> Self { // Can construct directly as `value` is non-degenerate: Self { - inner: value.into_iter().map(Into::into).collect(), + exprs: value.into_iter().map(Into::into).collect(), } } } diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7afa9f0134e9a..3805514a845d0 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -30,6 +30,7 @@ use datafusion_common::{JoinType, ScalarValue}; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; +use itertools::Itertools; /// Represents whether a constant expression's value is uniform or varies across /// partitions. Has two variants: @@ -470,8 +471,8 @@ impl EquivalenceGroup { self.remove_redundant_entries(); } - /// Normalizes the given physical expression according to this group. - /// The expression is replaced with the first expression in the equivalence + /// Normalizes the given physical expression according to this group. The + /// expression is replaced with the first expression in the equivalence /// class it matches with (if any). pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.transform(|expr| { @@ -491,11 +492,11 @@ impl EquivalenceGroup { // The unwrap above is safe because the closure always returns `Ok`. } - /// Normalizes the given sort expression according to this group. - /// The underlying physical expression is replaced with the first expression - /// in the equivalence class it matches with (if any). If the underlying - /// expression does not belong to any equivalence class in this group, returns - /// the sort expression as is. + /// Normalizes the given sort expression according to this group. The + /// underlying physical expression is replaced with the first expression in + /// the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, + /// returns the sort expression as is. pub fn normalize_sort_expr( &self, mut sort_expr: PhysicalSortExpr, @@ -504,11 +505,31 @@ impl EquivalenceGroup { sort_expr } - /// Normalizes the given sort requirement according to this group. - /// The underlying physical expression is replaced with the first expression - /// in the equivalence class it matches with (if any). If the underlying - /// expression does not belong to any equivalence class in this group, returns - /// the given sort requirement as is. + /// Normalizes the given sort expressions (i.e. `sort_exprs`) by: + /// - Removing expressions that have a constant value. + /// - Replacing sections that belong to some equivalence class in the + /// with the first entry in the matching equivalence class. + /// - Removing duplicate sort expressions. + /// + /// If columns `a` and `b` are known to be equal, `d` is known to be a + /// constant, and `sort_exprs` is `[b ASC, d DESC, c ASC, a ASC]`, this + /// function would return `[a ASC, c ASC, a ASC]`. + pub fn normalize_sort_exprs<'a>( + &'a self, + sort_exprs: impl IntoIterator + 'a, + ) -> impl Iterator + 'a { + sort_exprs + .into_iter() + .map(|sort_expr| self.normalize_sort_expr(sort_expr)) + .filter(|sort_expr| self.is_expr_constant(&sort_expr.expr).is_none()) + .unique_by(|sort_expr| Arc::clone(&sort_expr.expr)) + } + + /// Normalizes the given sort requirement according to this group. The + /// underlying physical expression is replaced with the first expression in + /// the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, + /// returns the given sort requirement as is. pub fn normalize_sort_requirement( &self, mut sort_requirement: PhysicalSortRequirement, @@ -517,6 +538,26 @@ impl EquivalenceGroup { sort_requirement } + /// Normalizes the given sort requirements (i.e. `sort_reqs`) by: + /// - Removing expressions that have a constant value. + /// - Replacing sections that belong to some equivalence class in the + /// with the first entry in the matching equivalence class. + /// - Removing duplicate sort expressions. + /// + /// If columns `a` and `b` are known to be equal, `d` is known to be a + /// constant, and `sort_reqs` is `[b ASC, d DESC, c ASC, a ASC]`, this + /// function would return `[a ASC, c ASC, a ASC]`. + pub fn normalize_sort_requirements<'a>( + &'a self, + sort_reqs: impl IntoIterator + 'a, + ) -> impl Iterator + 'a { + sort_reqs + .into_iter() + .map(|req| self.normalize_sort_requirement(req)) + .filter(|req| self.is_expr_constant(&req.expr).is_none()) + .unique_by(|req| Arc::clone(&req.expr)) + } + /// Perform an indirect projection of `expr` by consulting the equivalence /// classes. fn project_expr_indirect( diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 96e6541d23be5..ac7e58ac81fa9 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -374,11 +374,10 @@ impl EquivalenceProperties { mut self, ordering: impl IntoIterator, ) -> Self { - // Filter out constant expressions as they don't affect ordering + // Filter out constant expressions as they don't affect ordering: let filtered_exprs = ordering .into_iter() - .filter(|expr| self.is_expr_constant(&expr.expr).is_none()) - .collect::>(); + .filter(|expr| self.is_expr_constant(&expr.expr).is_none()); if let Some(filtered_exprs) = LexOrdering::new(filtered_exprs) { // Preserve valid suffixes from existing orderings: @@ -411,55 +410,25 @@ impl EquivalenceProperties { } /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the - /// equivalence group and the ordering equivalence class within. Returns - /// a `LexOrdering` instance if the expressions define a proper lexicographical - /// ordering. It works by: - /// - Removing expressions that have a constant value from the given expressions. - /// - Replacing sections that belong to some equivalence class in the equivalence - /// group with the first entry in the matching equivalence class. - /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. Also - /// assume that `self.oeq_class` contains equivalent orderings `d ASC` and - /// `a ASC, c ASC` (in the sense that both describe the ordering of the - /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function - /// would return `[a ASC, c ASC]`. Internally, it would first normalize to - /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. + /// equivalence group within. Returns a `LexOrdering` instance if the + /// expressions define a proper lexicographical ordering. For more details, + /// see [`EquivalenceGroup::normalize_sort_exprs`]. pub fn normalize_sort_exprs( &self, sort_exprs: impl IntoIterator, ) -> Option { - // Prune redundant sections in the ordering: - let sort_exprs = sort_exprs - .into_iter() - .map(|sort_expr| self.eq_group.normalize_sort_expr(sort_expr)) - .filter(|order| self.is_expr_constant(&order.expr).is_none()); - LexOrdering::new(sort_exprs).map(|o| o.collapse()) + LexOrdering::new(self.eq_group.normalize_sort_exprs(sort_exprs)) } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the - /// equivalence group and the ordering equivalence class within. Returns - /// a `LexRequirement` instance if the expressions define a proper lexicographical - /// ordering requirement. It works by: - /// - Removing expressions that have a constant value from the given requirements. - /// - Replacing sections that belong to some equivalence class in the equivalence - /// group with the first entry in the matching equivalence class. - /// - /// Assume that `self.eq_group` states column `a` and `b` are aliases. Also - /// assume that `self.oeq_class` contains equivalent orderings `d ASC` and - /// `a ASC, c ASC` (in the sense that both describe the ordering of the - /// table). If `sort_exprs` were `[b ASC, c ASC, a ASC]`, then this function - /// would return `[a ASC, c ASC]`. Internally, it would first normalize to - /// `[a ASC, c ASC, a ASC]` and end up with the final result after deduplication. + /// equivalence group within. Returns a `LexRequirement` instance if the + /// expressions define a proper lexicographical requirement. For more + /// details, see [`EquivalenceGroup::normalize_sort_exprs`]. fn normalize_sort_requirements( &self, sort_reqs: impl IntoIterator, ) -> Option { - // Prune redundant sections in the requirement: - let reqs = sort_reqs - .into_iter() - .map(|req| self.eq_group.normalize_sort_requirement(req)) - .filter(|order| self.is_expr_constant(&order.expr).is_none()); - LexRequirement::new(reqs).map(|r| r.collapse()) + LexRequirement::new(self.eq_group.normalize_sort_requirements(sort_reqs)) } /// Checks whether the given ordering is satisfied by any of the existing From 725289882c8aec56b5dc82fa57513e5b78d6dc7c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 3 May 2025 21:31:58 +0300 Subject: [PATCH 127/167] Improve comments --- datafusion/physical-expr-common/src/sort_expr.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index cf427c3dd9909..0e8a5693a3e83 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -323,10 +323,10 @@ impl From for PhysicalSortExpr { } } -///`LexOrdering` contains a `Vec`, which represents -/// a lexicographical ordering. +/// This object represents a lexicographical ordering and contains a vector +/// of `PhysicalSortExpr` objects. /// -/// For example, `vec![a ASC, b DESC]` represents a lexicographical ordering +/// For example, a `vec![a ASC, b DESC]` represents a lexicographical ordering /// that first sorts by column `a` in ascending order, then by column `b` in /// descending order. #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -460,8 +460,8 @@ impl From for Vec { } } -///`LexRequirement` is an struct containing a `Vec`, which -/// represents a lexicographical ordering requirement. +/// This object represents a lexicographical ordering requirement and contains +/// a vector of `PhysicalSortRequirement` objects. #[derive(Debug, Clone, PartialEq)] pub struct LexRequirement { reqs: Vec, From 95c89b53ef7c36d7598b0328587f860a60bdb22c Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 4 May 2025 12:59:45 +0300 Subject: [PATCH 128/167] Improve display for EquivalenceProperties --- .../src/equivalence/properties/mod.rs | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index ac7e58ac81fa9..8b77ce60795dd 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1234,18 +1234,21 @@ impl From for OrderingEquivalenceClass { /// /// Format: /// ```text -/// order: [[a ASC, b ASC], [a ASC, c ASC]], eq: [[a = b], [a = c]], const: [a = 1] +/// order: [[b@1 ASC NULLS LAST]], eq: [{members: [a@0], constant: (heterogeneous)}] /// ``` impl Display for EquivalenceProperties { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - if self.eq_group.is_empty() && self.oeq_class.is_empty() { - return write!(f, "No properties"); - } - if !self.oeq_class.is_empty() { + let empty_eq_group = self.eq_group.is_empty(); + let empty_oeq_class = self.oeq_class.is_empty(); + if empty_oeq_class && empty_eq_group { + write!(f, "No properties")?; + } else if !empty_oeq_class { write!(f, "order: {}", self.oeq_class)?; - } - if !self.eq_group.is_empty() { - write!(f, ", eq: {}", self.eq_group)?; + if !empty_eq_group { + write!(f, ", eq: {}", self.eq_group)?; + } + } else { + write!(f, "eq: {}", self.eq_group)?; } Ok(()) } From 0cfb36427ce218cd22c55476b105cb30465d3de7 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 6 May 2025 13:16:15 +0300 Subject: [PATCH 129/167] More idiomatic code --- .../physical-expr-common/src/sort_expr.rs | 3 +- .../physical-expr/src/equivalence/class.rs | 31 ++++++++------ .../src/equivalence/properties/mod.rs | 41 ++++++++----------- 3 files changed, 38 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 0e8a5693a3e83..d317aea22c088 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -587,9 +587,10 @@ impl OrderingRequirements { /// Creates a new instance from the given alternatives. If an empty list of /// alternatives are given, returns `None`. pub fn new_alternatives( - alternatives: Vec, + alternatives: impl IntoIterator, soft: bool, ) -> Option { + let alternatives = alternatives.into_iter().collect::>(); (!alternatives.is_empty()).then(|| { if soft { Self::Soft(alternatives) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 3805514a845d0..b3ebabba0d0ec 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -379,14 +379,15 @@ impl EquivalenceGroup { } } - /// Adds the equality `left` = `right` to this equivalence group. - /// New equality conditions often arise after steps like `Filter(a = b)`, - /// `Alias(a, a as b)` etc. + /// Adds the equality `left` = `right` to this equivalence group. New + /// equality conditions often arise after steps like `Filter(a = b)`, + /// `Alias(a, a as b)` etc. Returns whether the given equality was + /// material (i.e. it was not already known). pub fn add_equal_conditions( &mut self, left: Arc, right: Arc, - ) { + ) -> bool { let mut idx = 0; let size = self.classes.len(); let mut first_class = None; @@ -405,17 +406,20 @@ impl EquivalenceGroup { (Some(mut first_idx), Some(mut second_idx)) => { // If the given left and right sides belong to different classes, // we should unify/bridge these classes. - if first_idx != second_idx { - // By convention, make sure `second_idx` is larger than `first_idx`. - if first_idx > second_idx { - (first_idx, second_idx) = (second_idx, first_idx); + match first_idx.cmp(&second_idx) { + // The equality is already known, return and signal this: + std::cmp::Ordering::Equal => return false, + // Swap indices to ensure `first_idx` is the lesser index. + std::cmp::Ordering::Greater => { + std::mem::swap(&mut first_idx, &mut second_idx); } - // Remove the class at `second_idx` and merge its values with - // the class at `first_idx`. The convention above makes sure - // that `first_idx` is still valid after removing `second_idx`. - let other_class = self.classes.swap_remove(second_idx); - self.classes[first_idx].extend(other_class); + _ => {} } + // Remove the class at `second_idx` and merge its values with + // the class at `first_idx`. The convention above makes sure + // that `first_idx` is still valid after removing `second_idx`. + let other_class = self.classes.swap_remove(second_idx); + self.classes[first_idx].extend(other_class); } (Some(group_idx), None) => { // Right side is new, extend left side's class: @@ -431,6 +435,7 @@ impl EquivalenceGroup { self.classes.push(EquivalenceClass::new([left, right])); } } + true } /// Removes redundant entries from this group. diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 8b77ce60795dd..176d0695e7fc1 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -280,32 +280,31 @@ impl EquivalenceProperties { right: Arc, ) -> Result<()> { // Add equal expressions to the state: - self.eq_group.add_equal_conditions(Arc::clone(&left), right); - // Discover any new orderings: - self.discover_new_orderings(left) + if self.eq_group.add_equal_conditions(Arc::clone(&left), right) { + // Discover any new orderings: + self.discover_new_orderings(left)?; + } + Ok(()) } /// Track/register physical expressions with constant values. pub fn add_constants(&mut self, constants: impl IntoIterator) { - let c = constants.into_iter().collect::>(); - let constants = c.into_iter(); // Add the new constant to the equivalence group: for constant in constants { self.eq_group.add_constant(constant); } - // Discover any new orderings based on the constants + // Discover any new orderings based on the constants: for ordering in self.normalized_oeq_class().iter() { - self.discover_new_orderings(Arc::clone(&ordering[0].expr)) - .unwrap(); + let leading = Arc::clone(&ordering[0].expr); + self.discover_new_orderings(leading).unwrap(); } } - // Discover new valid orderings in light of a new equality. - // Accepts a single argument (`expr`) which is used to determine - // which orderings should be updated. - // When constants or equivalence classes are changed, there may be new orderings - // that can be discovered with the new equivalence properties. - // For a discussion, see: https://github.com/apache/datafusion/issues/9812 + /// Discover new valid orderings in light of a new equality. Accepts a single + /// argument (`expr`) which is used to determine the orderings to update. + /// When constants or equivalence classes change, there may be new orderings + /// that can be discovered with the new equivalence properties. + /// For a discussion, see: fn discover_new_orderings(&mut self, expr: Arc) -> Result<()> { let normalized_expr = self.eq_group().normalize_expr(expr); let eq_class = self @@ -424,7 +423,7 @@ impl EquivalenceProperties { /// equivalence group within. Returns a `LexRequirement` instance if the /// expressions define a proper lexicographical requirement. For more /// details, see [`EquivalenceGroup::normalize_sort_exprs`]. - fn normalize_sort_requirements( + pub fn normalize_sort_requirements( &self, sort_reqs: impl IntoIterator, ) -> Option { @@ -470,10 +469,8 @@ impl EquivalenceProperties { } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); let satisfy = match sort_properties { SortProperties::Ordered(options) => { - let sort_expr = PhysicalSortExpr { - expr: Arc::clone(&element.expr), - options, - }; + let sort_expr = + PhysicalSortExpr::new(Arc::clone(&element.expr), options); sort_expr.satisfy(&element, schema) } // Singleton expressions satisfies any requirement. @@ -520,10 +517,8 @@ impl EquivalenceProperties { } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); let satisfy = match sort_properties { SortProperties::Ordered(options) => { - let sort_expr = PhysicalSortExpr { - expr: Arc::clone(&element.expr), - options, - }; + let sort_expr = + PhysicalSortExpr::new(Arc::clone(&element.expr), options); sort_expr.satisfy_expr(&element, schema) } // Singleton expressions satisfies any ordering. From 277f8679299d93154859e68a937bf47819577851 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 6 May 2025 13:29:26 +0300 Subject: [PATCH 130/167] More succinct code --- .../src/equivalence/properties/mod.rs | 72 +++++++++---------- 1 file changed, 35 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 176d0695e7fc1..7f31b28149997 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -348,13 +348,13 @@ impl EquivalenceProperties { let expr_properties = equivalent_expr.get_properties(&child_properties)?; if expr_properties.preserves_lex_ordering - && SortProperties::Ordered(leading_ordering_options) - == expr_properties.sort_properties + && expr_properties.sort_properties + == SortProperties::Ordered(leading_ordering_options) { - // Assume existing ordering is `[c ASC, a ASC, b ASC]`. When - // equality `c = f(a, b)` is given, the ordering `[a ASC, b ASC]`, - // implies the ordering `[f(a, b) ASC]`. Thus, we can deduce that - // ordering `[a ASC, b ASC]` is also valid. + // Assume that `[c ASC, a ASC, b ASC]` is among existing + // orderings. If equality `c = f(a, b)` is given, ordering + // `[a ASC, b ASC]` implies the ordering `[c ASC]`. Thus, + // ordering `[a ASC, b ASC]` is also a valid ordering. new_orderings.push(ordering[1..].to_vec()); break; } @@ -365,9 +365,9 @@ impl EquivalenceProperties { Ok(()) } - /// Updates the ordering equivalence group within assuming that the table - /// is re-sorted according to the argument `ordering`. Note that constants - /// and equivalence classes are unchanged as they are unaffected by a re-sort. + /// Updates the ordering equivalence class within assuming that the table + /// is re-sorted according to the argument `ordering`. Note that equivalence + /// classes (and constants) do not change as they are unaffected by a re-sort. /// If the given ordering is already satisfied, the function does nothing. pub fn with_reorder( mut self, @@ -393,19 +393,14 @@ impl EquivalenceProperties { self } - /// Checks if the new ordering matches a prefix of the existing ordering - /// (considering expression equivalences) - fn is_prefix_of(&self, new_order: &LexOrdering, existing: &LexOrdering) -> bool { - // Check if new order is longer than existing - can't be a prefix - if new_order.len() > existing.len() { - return false; - } - - // Check if new order matches existing prefix (considering equivalences) - new_order.iter().zip(existing).all(|(new, existing)| { - self.eq_group.exprs_equal(&new.expr, &existing.expr) - && new.options == existing.options - }) + /// Checks if the ordering `given` matches a prefix of the ordering + /// `reference` (considering expression equivalences). + fn is_prefix_of(&self, given: &LexOrdering, reference: &LexOrdering) -> bool { + given.len() <= reference.len() + && given.iter().zip(reference).all(|(new, existing)| { + self.eq_group.exprs_equal(&new.expr, &existing.expr) + && new.options == existing.options + }) } /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the @@ -709,9 +704,9 @@ impl EquivalenceProperties { /// sorted, `atan(x + 1000)` should also be substituted. For now, we /// only consider single-column `CAST` expressions. fn substitute_oeq_class( - &self, - oeq_class: OrderingEquivalenceClass, + schema: &SchemaRef, mapping: &ProjectionMapping, + oeq_class: OrderingEquivalenceClass, ) -> OrderingEquivalenceClass { let new_orderings = oeq_class.into_iter().flat_map(|order| { // Modify/expand existing orderings by substituting sort @@ -727,7 +722,7 @@ impl EquivalenceProperties { let mut result = vec![]; // The sort expression comes from this schema, so the // following call to `unwrap` is safe. - let expr_type = sort_expr.expr.data_type(&self.schema).unwrap(); + let expr_type = sort_expr.expr.data_type(schema).unwrap(); // TODO: Add one-to-one analysis for ScalarFunctions. for r_expr in referring_exprs { // We check whether this expression is substitutable. @@ -910,16 +905,21 @@ impl EquivalenceProperties { /// /// - `mapping`: A reference to the `ProjectionMapping` that defines the /// relationship between source and target expressions. + /// - `oeq_class`: The `OrderingEquivalenceClass` containing the orderings + /// to project. /// /// # Returns /// /// A vector of `LexOrdering` containing all valid orderings after projection. - fn projected_orderings(&self, mapping: &ProjectionMapping) -> Vec { + fn projected_orderings( + &self, + mapping: &ProjectionMapping, + mut oeq_class: OrderingEquivalenceClass, + ) -> OrderingEquivalenceClass { // Normalize source expressions in the mapping: let mapping = self.normalized_mapping(mapping); // Get dependency map for existing orderings: - let mut oeq_class = self.normalized_oeq_class(); - oeq_class = self.substitute_oeq_class(oeq_class, &mapping); + oeq_class = Self::substitute_oeq_class(&self.schema, &mapping, oeq_class); let dependency_map = self.construct_dependency_map(oeq_class, &mapping); let orderings = mapping.iter().flat_map(|(source, targets)| { referred_dependencies(&dependency_map, source) @@ -986,7 +986,8 @@ impl EquivalenceProperties { orderings .chain(projected_orderings) .map(|lex_ordering| lex_ordering.collapse()) - .collect() + .collect::>() + .into() } /// Projects constraints according to the given projection mapping. @@ -1016,17 +1017,14 @@ impl EquivalenceProperties { self.constraints.project(&indices) } - /// Projects the equivalences within according to `mapping` - /// and `output_schema`. + /// Projects the equivalences within according to `mapping` and + /// `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { - let eq_group = self.eq_group.project(mapping); - let oeq_class = self.projected_orderings(mapping).into(); - let constraints = self.projected_constraints(mapping).unwrap_or_default(); Self { + eq_group: self.eq_group.project(mapping), + oeq_class: self.projected_orderings(mapping, self.normalized_oeq_class()), + constraints: self.projected_constraints(mapping).unwrap_or_default(), schema: output_schema, - eq_group, - oeq_class, - constraints, } } From 806016251f2df4ca95b2993cf7f0c0a75571d9bd Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 6 May 2025 13:45:19 +0300 Subject: [PATCH 131/167] Remove extend_orderings from EquivalenceProperties --- .../src/equivalence/properties/joins.rs | 10 +++++----- .../physical-expr/src/equivalence/properties/mod.rs | 12 +++--------- 2 files changed, 8 insertions(+), 14 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 0a542aaaa2022..e2f1171ca316b 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -68,9 +68,9 @@ pub fn join_equivalence_properties( // then we should add `a ASC, b ASC` to the ordering equivalences // of the join output. let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); - result.extend_orderings(out_oeq_class); + result.add_orderings(out_oeq_class); } else { - result.extend_orderings(left_oeq_class); + result.add_orderings(left_oeq_class); } } [false, true] => { @@ -91,9 +91,9 @@ pub fn join_equivalence_properties( // then we should add `b ASC, a ASC` to the ordering equivalences // of the join output. let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); - result.extend_orderings(out_oeq_class); + result.add_orderings(out_oeq_class); } else { - result.extend_orderings(right_oeq_class); + result.add_orderings(right_oeq_class); } } [false, false] => {} @@ -271,7 +271,7 @@ mod tests { &join_type, left_columns_len, ); - join_eq_properties.extend_orderings(right_oeq_class); + join_eq_properties.add_orderings(right_oeq_class); let result = join_eq_properties.oeq_class().clone(); // [x ASC, y ASC], [z ASC, w ASC] diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 7f31b28149997..aa967c3695ae0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -228,9 +228,9 @@ impl EquivalenceProperties { /// Extends this `EquivalenceProperties` with the `other` object. pub fn extend(mut self, other: Self) -> Self { - self.eq_group.extend(other.eq_group); - self.oeq_class.extend(other.oeq_class); self.constraints.extend(other.constraints); + self.add_equivalence_group(other.eq_group); + self.add_orderings(other.oeq_class); self } @@ -246,12 +246,6 @@ impl EquivalenceProperties { self.eq_group.clear_per_partition_constants(); } - /// Extends this `EquivalenceProperties` by adding the orderings inside - /// collection `other`. - pub fn extend_orderings(&mut self, other: impl IntoIterator) { - self.oeq_class.extend(other); - } - /// Adds new orderings into the existing ordering equivalence class. pub fn add_orderings( &mut self, @@ -361,7 +355,7 @@ impl EquivalenceProperties { } } - self.oeq_class.add_orderings(new_orderings); + self.add_orderings(new_orderings); Ok(()) } From 703db4b3c30e0f0540d15830c7d7ac0e98a141aa Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 6 May 2025 16:21:19 +0300 Subject: [PATCH 132/167] Simplify with_reorder --- .../src/equivalence/properties/mod.rs | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index aa967c3695ae0..e64af14776b69 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -367,22 +367,21 @@ impl EquivalenceProperties { mut self, ordering: impl IntoIterator, ) -> Self { - // Filter out constant expressions as they don't affect ordering: - let filtered_exprs = ordering + // Normalize the given ordering: + let normalized_ordering = ordering .into_iter() .filter(|expr| self.is_expr_constant(&expr.expr).is_none()); - - if let Some(filtered_exprs) = LexOrdering::new(filtered_exprs) { + if let Some(normalized_ordering) = LexOrdering::new(normalized_ordering) { // Preserve valid suffixes from existing orderings: - let oeq_class = mem::take(&mut self.oeq_class); - let mut new_orderings = oeq_class - .into_iter() - .filter(|existing| self.is_prefix_of(&filtered_exprs, existing)) - .collect::>(); - if new_orderings.is_empty() { - new_orderings.push(filtered_exprs); + let mut orderings: Vec<_> = mem::take(&mut self.oeq_class).into(); + orderings.retain(|existing| { + // Check if the existing ordering is a prefix of the new ordering: + self.is_prefix_of(&normalized_ordering, existing) + }); + if orderings.is_empty() { + orderings.push(normalized_ordering); } - self.oeq_class = new_orderings.into(); + self.oeq_class = OrderingEquivalenceClass::new(orderings); } self } From 2d38c1444a57b30bed471f3c54175d9ba73deb7f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 7 May 2025 17:29:29 +0300 Subject: [PATCH 133/167] Store normalized orderings - 1 --- datafusion/core/tests/dataframe/mod.rs | 2 +- .../physical_optimizer/enforce_sorting.rs | 15 +- .../src/equivalence/properties/dependency.rs | 254 +++++------------- .../src/equivalence/properties/joins.rs | 8 +- .../src/equivalence/properties/mod.rs | 89 +++--- .../src/equivalence/properties/union.rs | 2 +- .../src/enforce_sorting/sort_pushdown.rs | 129 +++++---- datafusion/sqllogictest/test_files/topk.slt | 4 +- 8 files changed, 213 insertions(+), 290 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 1855a512048d6..4daa6a526a470 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2525,7 +2525,7 @@ async fn test_count_wildcard_on_sort() -> Result<()> { | | TableScan: t1 projection=[b] | | physical_plan | ProjectionExec: expr=[b@0 as b, count(*)@1 as count(*)] | | | SortPreservingMergeExec: [count(Int64(1))@2 ASC NULLS LAST] | - | | SortExec: expr=[count(Int64(1))@2 ASC NULLS LAST], preserve_partitioning=[true] | + | | SortExec: expr=[count(*)@1 ASC NULLS LAST], preserve_partitioning=[true] | | | ProjectionExec: expr=[b@0 as b, count(Int64(1))@1 as count(*), count(Int64(1))@1 as count(Int64(1))] | | | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[count(Int64(1))] | | | CoalesceBatchesExec: target_batch_size=8192 | diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 68ae6de899313..80cfdb9e24902 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -1335,7 +1335,7 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { ]; // can not push down the sort requirements, need to add SortExec let expected_optimized = [ - "SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[col_b@3 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", @@ -1359,14 +1359,13 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ]; - // can not push down the sort requirements, need to add SortExec + // Can push down the sort requirements since col_a = nullable_col let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + "SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 6690a2b7dd262..4b79d688ad5f6 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -512,22 +512,20 @@ mod tests { let col_c_expr = col("c", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - eq_properties.add_equal_conditions(col_a_expr, Arc::clone(&col_c_expr))?; + eq_properties + .add_equal_conditions(Arc::clone(&col_a_expr), Arc::clone(&col_c_expr))?; eq_properties.add_orderings([ vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], - vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], + vec![PhysicalSortExpr::new_default(col_c_expr)], ]); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_orderings([ - vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], - vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], + vec![PhysicalSortExpr::new_default(col_b_expr)], + vec![PhysicalSortExpr::new_default(col_a_expr)], ]); - let oeq_class = eq_properties.oeq_class().clone(); - let expected = expected_eqs.oeq_class(); - assert!(oeq_class.eq(expected)); - + assert!(eq_properties.oeq_class().eq(expected_eqs.oeq_class())); Ok(()) } @@ -540,33 +538,21 @@ mod tests { Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let required_columns = [Arc::clone(&col_b), Arc::clone(&col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_ordering([ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, + PhysicalSortExpr::new(Arc::new(Column::new("b", 1)), sort_options_not), + PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0, 1]); assert_eq!( result, vec![ - PhysicalSortExpr { - expr: Arc::clone(col_b), - options: sort_options_not - }, - PhysicalSortExpr { - expr: Arc::clone(col_a), - options: sort_options - } + PhysicalSortExpr::new(col_b, sort_options_not), + PhysicalSortExpr::new(col_a, sort_options), ] ); @@ -575,24 +561,18 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let required_columns = [Arc::clone(&col_b), Arc::clone(&col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_orderings([ - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }], + vec![PhysicalSortExpr::new( + Arc::new(Column::new("c", 2)), + sort_options, + )], vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, + PhysicalSortExpr::new(Arc::new(Column::new("b", 1)), sort_options_not), + PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ], ]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); @@ -600,14 +580,8 @@ mod tests { assert_eq!( result, vec![ - PhysicalSortExpr { - expr: Arc::clone(col_b), - options: sort_options_not - }, - PhysicalSortExpr { - expr: Arc::clone(col_a), - options: sort_options - } + PhysicalSortExpr::new(col_b, sort_options_not), + PhysicalSortExpr::new(col_a, sort_options), ] ); @@ -624,18 +598,9 @@ mod tests { // not satisfied orders eq_properties.add_ordering([ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: sort_options, - }, + PhysicalSortExpr::new(Arc::new(Column::new("b", 1)), sort_options_not), + PhysicalSortExpr::new(Arc::new(Column::new("c", 2)), sort_options), + PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ]); let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0]); @@ -653,49 +618,35 @@ mod tests { ]); let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let col_c = col("c", &schema)?; + let col_d = col("d", &schema)?; let option_asc = SortOptions { descending: false, nulls_first: false, }; // b=a (e.g they are aliases) - eq_properties.add_equal_conditions(Arc::clone(col_b), Arc::clone(col_a))?; + eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_a))?; // [b ASC], [d ASC] eq_properties.add_orderings([ - vec![PhysicalSortExpr { - expr: Arc::clone(col_b), - options: option_asc, - }], - vec![PhysicalSortExpr { - expr: Arc::clone(col_d), - options: option_asc, - }], + vec![PhysicalSortExpr::new(Arc::clone(&col_b), option_asc)], + vec![PhysicalSortExpr::new(Arc::clone(&col_d), option_asc)], ]); let test_cases = vec![ // d + b ( - Arc::new(BinaryExpr::new( - Arc::clone(col_d), - Operator::Plus, - Arc::clone(col_b), - )) as _, + Arc::new(BinaryExpr::new(col_d, Operator::Plus, Arc::clone(&col_b))) as _, SortProperties::Ordered(option_asc), ), // b - (Arc::clone(col_b), SortProperties::Ordered(option_asc)), + (col_b, SortProperties::Ordered(option_asc)), // a - (Arc::clone(col_a), SortProperties::Ordered(option_asc)), + (Arc::clone(&col_a), SortProperties::Ordered(option_asc)), // a + c ( - Arc::new(BinaryExpr::new( - Arc::clone(col_a), - Operator::Plus, - Arc::clone(col_c), - )), + Arc::new(BinaryExpr::new(col_a, Operator::Plus, col_c)), SortProperties::Unordered, ), ]; @@ -736,7 +687,7 @@ mod tests { Arc::clone(col_a), Operator::Plus, Arc::clone(col_d), - )) as Arc; + )) as _; let option_asc = SortOptions { descending: false, @@ -748,14 +699,8 @@ mod tests { }; // [d ASC, h DESC] also satisfies schema. eq_properties.add_ordering([ - PhysicalSortExpr { - expr: Arc::clone(col_d), - options: option_asc, - }, - PhysicalSortExpr { - expr: Arc::clone(col_h), - options: option_desc, - }, + PhysicalSortExpr::new(Arc::clone(col_d), option_asc), + PhysicalSortExpr::new(Arc::clone(col_h), option_desc), ]); let test_cases = vec![ // TEST CASE 1 @@ -980,12 +925,8 @@ mod tests { Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) - .with_reorder(["a", "b", "c"].into_iter().map(|c| PhysicalSortExpr { - expr: col(c, schema.as_ref()).unwrap(), - options: SortOptions { - descending: false, - nulls_first: true, - }, + .with_reorder(["a", "b", "c"].into_iter().map(|c| { + PhysicalSortExpr::new_default(col(c, schema.as_ref()).unwrap()) })); struct TestCase { @@ -999,17 +940,14 @@ mod tests { let col_a = col("a", schema.as_ref())?; let col_b = col("b", schema.as_ref())?; let col_c = col("c", schema.as_ref())?; - let cast_c = Arc::new(CastExpr::new(col_c, DataType::Date32, None)); + let cast_c = Arc::new(CastExpr::new(col_c, DataType::Date32, None)) as _; let cases = vec![ TestCase { name: "(a, b, c) -> (c)", // b is constant, so it should be removed from the sort order constants: vec![Arc::clone(&col_b)], - equal_conditions: vec![[ - Arc::clone(&cast_c) as Arc, - Arc::clone(&col_a), - ]], + equal_conditions: vec![[Arc::clone(&cast_c), Arc::clone(&col_a)]], sort_columns: &["c"], should_satisfy_ordering: true, }, @@ -1019,10 +957,7 @@ mod tests { name: "(a, b, c) -> (c)", // b is constant, so it should be removed from the sort order constants: vec![col_b], - equal_conditions: vec![[ - Arc::clone(&col_a), - Arc::clone(&cast_c) as Arc, - ]], + equal_conditions: vec![[Arc::clone(&col_a), Arc::clone(&cast_c)]], sort_columns: &["c"], should_satisfy_ordering: true, }, @@ -1031,10 +966,7 @@ mod tests { // b is not constant anymore constants: vec![], // a and c are still compatible, but this is irrelevant since the original ordering is (a, b, c) - equal_conditions: vec![[ - Arc::clone(&cast_c) as Arc, - Arc::clone(&col_a), - ]], + equal_conditions: vec![[Arc::clone(&cast_c), Arc::clone(&col_a)]], sort_columns: &["c"], should_satisfy_ordering: false, }, @@ -1205,18 +1137,18 @@ mod tests { ]); // Add equality condition c = concat(a, b) - eq_properties.add_equal_conditions(col_c, Arc::clone(&a_concat_b))?; + eq_properties.add_equal_conditions(Arc::clone(&col_c), a_concat_b)?; let orderings = eq_properties.oeq_class(); - let expected_ordering1 = [PhysicalSortExpr::new_default(a_concat_b).asc()].into(); + let expected_ordering1 = [PhysicalSortExpr::new_default(col_c).asc()].into(); let expected_ordering2 = [ PhysicalSortExpr::new_default(col_a).asc(), PhysicalSortExpr::new_default(col_b).asc(), ] .into(); - // The ordering should be [concat(a, b) ASC] and [a ASC, b ASC] + // The ordering should be [c ASC] and [a ASC, b ASC] assert_eq!(orderings.len(), 2); assert!(orderings.contains(&expected_ordering1)); assert!(orderings.contains(&expected_ordering2)); @@ -1264,14 +1196,8 @@ mod tests { eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_a))]); let sort_exprs = vec![ - PhysicalSortExpr { - expr: col_a, - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::clone(&col_b), - options: SortOptions::default(), - }, + PhysicalSortExpr::new_default(col_a), + PhysicalSortExpr::new_default(Arc::clone(&col_b)), ]; let result = eq_properties.with_reorder(sort_exprs); @@ -1302,25 +1228,13 @@ mod tests { // Initial ordering: [a ASC, b DESC, c ASC] eq_properties.add_ordering([ - PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: asc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_b), - options: desc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_c), - options: asc, - }, + PhysicalSortExpr::new(Arc::clone(&col_a), asc), + PhysicalSortExpr::new(Arc::clone(&col_b), desc), + PhysicalSortExpr::new(Arc::clone(&col_c), asc), ]); // New ordering: [a ASC] - let new_order = vec![PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: asc, - }]; + let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), asc)]; let result = eq_properties.with_reorder(new_order); @@ -1350,25 +1264,14 @@ mod tests { // Make a and b equivalent eq_properties.add_equal_conditions(Arc::clone(&col_a), Arc::clone(&col_b))?; - let asc = SortOptions::default(); - // Initial ordering: [a ASC, c ASC] eq_properties.add_ordering([ - PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: asc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_c), - options: asc, - }, + PhysicalSortExpr::new_default(Arc::clone(&col_a)), + PhysicalSortExpr::new_default(Arc::clone(&col_c)), ]); // New ordering: [b ASC] - let new_order = vec![PhysicalSortExpr { - expr: Arc::clone(&col_b), - options: asc, - }]; + let new_order = vec![PhysicalSortExpr::new_default(Arc::clone(&col_b))]; let result = eq_properties.with_reorder(new_order); @@ -1376,6 +1279,7 @@ mod tests { assert_eq!(result.oeq_class().len(), 1); // Verify orderings + let asc = SortOptions::default(); let ordering = result.oeq_class().iter().next().unwrap(); assert_eq!(ordering.len(), 2); assert!(ordering[0].expr.eq(&col_a) || ordering[0].expr.eq(&col_b)); @@ -1430,8 +1334,6 @@ mod tests { let col_d = col("d", &schema)?; let col_e = col("e", &schema)?; - let asc = SortOptions::default(); - // Constants: c is constant eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_c))]); @@ -1441,31 +1343,16 @@ mod tests { // Orderings: [d ASC, a ASC], [e ASC] eq_properties.add_orderings([ vec![ - PhysicalSortExpr { - expr: Arc::clone(&col_d), - options: asc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_a), - options: asc, - }, + PhysicalSortExpr::new_default(Arc::clone(&col_d)), + PhysicalSortExpr::new_default(Arc::clone(&col_a)), ], - vec![PhysicalSortExpr { - expr: Arc::clone(&col_e), - options: asc, - }], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_e))], ]); // Initial ordering: [b ASC, c ASC] let new_order = vec![ - PhysicalSortExpr { - expr: Arc::clone(&col_b), - options: asc, - }, - PhysicalSortExpr { - expr: Arc::clone(&col_c), - options: asc, - }, + PhysicalSortExpr::new_default(Arc::clone(&col_b)), + PhysicalSortExpr::new_default(Arc::clone(&col_c)), ]; let result = eq_properties.with_reorder(new_order); @@ -1476,6 +1363,7 @@ mod tests { assert_eq!(ordering.len(), 2); // First expression should be either b or d (they're equivalent) + let asc = SortOptions::default(); assert!( ordering[0].expr.eq(&col_b) || ordering[0].expr.eq(&col_d), "Expected b or d as first expression, got {:?}", @@ -1591,9 +1479,8 @@ mod tests { .iter() .map(|cols| { cols.iter() - .map(|col_name| PhysicalSortExpr { - expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), + .map(|col_name| { + PhysicalSortExpr::new_default(col(col_name, schema).unwrap()) }) .collect::>() }) @@ -1603,9 +1490,8 @@ mod tests { .iter() .map(|cols| { cols.iter() - .map(|col_name| PhysicalSortExpr { - expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), + .map(|col_name| { + PhysicalSortExpr::new_default(col(col_name, schema).unwrap()) }) .collect::>() }) diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index e2f1171ca316b..f345d480f3509 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -263,8 +263,8 @@ mod tests { let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); // a=x and d=w - join_eq_properties.add_equal_conditions(col_a, Arc::clone(&col_x))?; - join_eq_properties.add_equal_conditions(col_d, Arc::clone(&col_w))?; + join_eq_properties.add_equal_conditions(Arc::clone(&col_a), col_x)?; + join_eq_properties.add_equal_conditions(Arc::clone(&col_d), col_w)?; updated_right_ordering_equivalence_class( &mut right_oeq_class, @@ -276,8 +276,8 @@ mod tests { // [x ASC, y ASC], [z ASC, w ASC] let orderings = vec![ - vec![(col_x, option_asc), (col_y, option_asc)], - vec![(col_z, option_asc), (col_w, option_asc)], + vec![(col_a, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_d, option_asc)], ]; let orderings = convert_to_orderings(&orderings); let expected = OrderingEquivalenceClass::from(orderings); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index e64af14776b69..cf67b4d8d557b 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -204,26 +204,7 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - let mut sort_exprs: Vec<_> = self.oeq_class.output_ordering()?.into(); - // Prune out constant expressions: - sort_exprs.retain(|sort_expr| { - self.eq_group - .get_equivalence_class(&sort_expr.expr) - .is_none_or(|cls| cls.constant.is_none()) - }); - LexOrdering::new(sort_exprs) - } - - /// Returns the normalized version of the ordering equivalence class within. - /// Normalization removes constants and duplicates as well as standardizing - /// expressions according to the equivalence group within. - pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { - self.oeq_class - .iter() - .cloned() - .filter_map(|ordering| self.normalize_sort_exprs(ordering)) - .collect::>() - .into() + self.oeq_class.output_ordering() } /// Extends this `EquivalenceProperties` with the `other` object. @@ -251,7 +232,12 @@ impl EquivalenceProperties { &mut self, orderings: impl IntoIterator>, ) { - self.oeq_class.add_orderings(orderings); + // TODO: Normalize `orderings` before adding: + self.oeq_class.add_orderings( + orderings + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)), + ); } /// Adds a single ordering to the existing ordering equivalence class. @@ -263,6 +249,13 @@ impl EquivalenceProperties { /// equivalence group within. pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { self.eq_group.extend(other_eq_group); + // TODO: Renormalize the orderings after adding new equivalences: + let oeq_class = mem::take(&mut self.oeq_class); + self.oeq_class = OrderingEquivalenceClass::new( + oeq_class + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)), + ); } /// Adds a new equality condition into the existing equivalence group. @@ -275,6 +268,13 @@ impl EquivalenceProperties { ) -> Result<()> { // Add equal expressions to the state: if self.eq_group.add_equal_conditions(Arc::clone(&left), right) { + // TODO: Renormalize the orderings after adding the new equality: + let oeq_class = mem::take(&mut self.oeq_class); + self.oeq_class = OrderingEquivalenceClass::new( + oeq_class + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)), + ); // Discover any new orderings: self.discover_new_orderings(left)?; } @@ -287,8 +287,15 @@ impl EquivalenceProperties { for constant in constants { self.eq_group.add_constant(constant); } + // TODO: Renormalize the orderings after adding new constants: + let oeq_class = mem::take(&mut self.oeq_class); + self.oeq_class = OrderingEquivalenceClass::new( + oeq_class + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)), + ); // Discover any new orderings based on the constants: - for ordering in self.normalized_oeq_class().iter() { + for ordering in self.oeq_class.clone() { let leading = Arc::clone(&ordering[0].expr); self.discover_new_orderings(leading).unwrap(); } @@ -300,16 +307,17 @@ impl EquivalenceProperties { /// that can be discovered with the new equivalence properties. /// For a discussion, see: fn discover_new_orderings(&mut self, expr: Arc) -> Result<()> { - let normalized_expr = self.eq_group().normalize_expr(expr); + let normalized_expr = self.eq_group.normalize_expr(expr); let eq_class = self .eq_group - .iter() - .find(|class| class.contains(&normalized_expr)) - .map(|class| class.clone().into()) - .unwrap_or_else(|| vec![Arc::clone(&normalized_expr)]); + .get_equivalence_class(&normalized_expr) + .map_or_else( + || vec![Arc::clone(&normalized_expr)], + |class| class.clone().into(), + ); let mut new_orderings = vec![]; - for ordering in self.normalized_oeq_class() { + for ordering in self.oeq_class.iter() { if !ordering[0].expr.eq(&normalized_expr) { continue; } @@ -367,11 +375,8 @@ impl EquivalenceProperties { mut self, ordering: impl IntoIterator, ) -> Self { - // Normalize the given ordering: - let normalized_ordering = ordering - .into_iter() - .filter(|expr| self.is_expr_constant(&expr.expr).is_none()); - if let Some(normalized_ordering) = LexOrdering::new(normalized_ordering) { + // Normalize the given ordering and process: + if let Some(normalized_ordering) = self.normalize_sort_exprs(ordering) { // Preserve valid suffixes from existing orderings: let mut orderings: Vec<_> = mem::take(&mut self.oeq_class).into(); orderings.retain(|existing| { @@ -1013,12 +1018,22 @@ impl EquivalenceProperties { /// Projects the equivalences within according to `mapping` and /// `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { - Self { + let mut proj_eqp = Self { eq_group: self.eq_group.project(mapping), - oeq_class: self.projected_orderings(mapping, self.normalized_oeq_class()), + oeq_class: self.projected_orderings(mapping, self.oeq_class.clone()), constraints: self.projected_constraints(mapping).unwrap_or_default(), schema: output_schema, - } + }; + // TODO: Remove renormalization after fixing `projected_orderings` to + // take the new eq. group as an argument and emit normalized + // orderings. + proj_eqp.oeq_class = OrderingEquivalenceClass::new( + proj_eqp + .oeq_class + .into_iter() + .map(|o| proj_eqp.eq_group.normalize_sort_exprs(o)), + ); + proj_eqp } /// Returns the longest (potentially partial) permutation satisfying the @@ -1281,7 +1296,7 @@ fn update_properties( let normalized_expr = eq_properties .eq_group .normalize_expr(Arc::clone(&node.expr)); - let oeq_class = eq_properties.normalized_oeq_class(); + let oeq_class = &eq_properties.oeq_class; if eq_properties.is_expr_constant(&normalized_expr).is_some() || oeq_class.is_expr_partial_const(&normalized_expr) { diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 7d69d669a3992..73077c76492e4 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -148,7 +148,7 @@ impl UnionEquivalentOrderingBuilder { ) { let constants = source.constants(); let properties_constants = properties.constants(); - for mut ordering in source.normalized_oeq_class() { + for mut ordering in source.oeq_class.clone() { // Progressively shorten the ordering to search for a satisfied prefix: loop { ordering = match self.try_add_ordering( diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index af7836e0de4bf..9ddc1ad54c6bf 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -28,6 +28,7 @@ use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortRequirement, }; @@ -280,24 +281,23 @@ fn pushdown_requirement_to_children( Ok(Some(vec![Some(parent_required); plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = LexOrdering::from(parent_required.first().clone()); - match expr_source_side(&parent_required_expr, smj.join_type(), left_columns_len) { - Some(JoinSide::Left) => try_pushdown_requirements_to_join( + let parent_ordering = LexOrdering::from(parent_required.first().clone()); + let eqp = smj.properties().equivalence_properties(); + match expr_source_side(eqp, parent_ordering, smj.join_type(), left_columns_len) { + Some((JoinSide::Left, ordering)) => try_pushdown_requirements_to_join( smj, parent_required.into_single(), - &parent_required_expr, + ordering, JoinSide::Left, ), - Some(JoinSide::Right) => { + Some((JoinSide::Right, mut ordering)) => { let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); - let new_right_required = - shift_right_required(parent_required.first().clone(), right_offset)?; - let new_right_required_expr = LexOrdering::from(new_right_required); + ordering = shift_right_ordering(ordering, right_offset)?; try_pushdown_requirements_to_join( smj, parent_required.into_single(), - &new_right_required_expr, + ordering, JoinSide::Right, ) } @@ -395,7 +395,7 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirement, - sort_expr: &LexOrdering, + ordering: LexOrdering, push_side: JoinSide, ) -> Result>>> { let mut smj_required_orderings = smj.required_input_ordering(); @@ -406,7 +406,7 @@ fn try_pushdown_requirements_to_join( .left() .equivalence_properties() .clone() - .with_reorder(sort_expr.clone()); + .with_reorder(ordering.clone()); let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; @@ -416,14 +416,14 @@ fn try_pushdown_requirements_to_join( return Ok(None); } // After re-ordering requirement is still satisfied - (Some(sort_expr), smj.right().output_ordering()) + (Some(&ordering), smj.right().output_ordering()) } JoinSide::Right => { let right_eq_properties = smj .right() .equivalence_properties() .clone() - .with_reorder(sort_expr.clone()); + .with_reorder(ordering.clone()); let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); @@ -434,7 +434,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); } // After re-ordering requirement is still satisfied - (smj.left().output_ordering(), Some(sort_expr)) + (smj.left().output_ordering(), Some(&ordering)) } JoinSide::None => return Ok(None), }; @@ -457,7 +457,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(OrderingRequirements::from(sort_expr.clone())); + let new_req = Some(OrderingRequirements::from(ordering)); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; @@ -472,62 +472,85 @@ fn try_pushdown_requirements_to_join( } fn expr_source_side( - required_exprs: &LexOrdering, + eqp: &EquivalenceProperties, + mut ordering: LexOrdering, join_type: JoinType, left_columns_len: usize, -) -> Option { +) -> Option<(JoinSide, LexOrdering)> { + // TODO: Handle the case where a prefix of the ordering comes from the left + // and a suffix from the right. match join_type { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full | JoinType::LeftMark => { - let all_column_sides = required_exprs - .iter() - .filter_map(|r| { - r.expr.as_any().downcast_ref::().map(|col| { - if col.index() < left_columns_len { - JoinSide::Left - } else { - JoinSide::Right + let eq_group = eqp.eq_group(); + let mut right_ordering = ordering.clone(); + let (mut valid_left, mut valid_right) = (true, true); + for (left, right) in ordering.iter_mut().zip(right_ordering.iter_mut()) { + let col = left.expr.as_any().downcast_ref::()?; + let eq_class = eq_group.get_equivalence_class(&left.expr); + if col.index() < left_columns_len { + if valid_right { + valid_right = eq_class.is_some_and(|cls| { + for expr in cls.iter() { + if expr + .as_any() + .downcast_ref::() + .is_some_and(|c| c.index() >= left_columns_len) + { + right.expr = Arc::clone(expr); + return true; + } + } + false + }); + } + } else if valid_left { + valid_left = eq_class.is_some_and(|cls| { + for expr in cls.iter() { + if expr + .as_any() + .downcast_ref::() + .is_some_and(|c| c.index() < left_columns_len) + { + left.expr = Arc::clone(expr); + return true; + } } - }) - }) - .collect::>(); - - // If the exprs are all coming from one side, the requirements can be pushed down - if all_column_sides.len() != required_exprs.len() { - None - } else if all_column_sides - .iter() - .all(|side| matches!(side, JoinSide::Left)) - { - Some(JoinSide::Left) - } else if all_column_sides - .iter() - .all(|side| matches!(side, JoinSide::Right)) - { - Some(JoinSide::Right) + false + }); + }; + if !(valid_left || valid_right) { + return None; + } + } + if valid_left { + Some((JoinSide::Left, ordering)) + } else if valid_right { + Some((JoinSide::Right, right_ordering)) } else { + // TODO: Handle the case where we can push down to both sides. None } } - JoinType::LeftSemi | JoinType::LeftAnti => required_exprs + JoinType::LeftSemi | JoinType::LeftAnti => ordering .iter() - .all(|e| e.expr.as_any().downcast_ref::().is_some()) - .then_some(JoinSide::Left), - JoinType::RightSemi | JoinType::RightAnti => required_exprs + .all(|e| e.expr.as_any().is::()) + .then_some((JoinSide::Left, ordering)), + JoinType::RightSemi | JoinType::RightAnti => ordering .iter() - .all(|e| e.expr.as_any().downcast_ref::().is_some()) - .then_some(JoinSide::Right), + .all(|e| e.expr.as_any().is::()) + .then_some((JoinSide::Right, ordering)), } } -fn shift_right_required( - mut parent_required: LexRequirement, +fn shift_right_ordering( + mut parent_ordering: LexOrdering, left_columns_len: usize, -) -> Result { - for req in parent_required.iter_mut() { +) -> Result { + for req in parent_ordering.iter_mut() { let Some(col) = req.expr.as_any().downcast_ref::() else { return plan_err!( "Expect to shift all the parent required column indexes for SortMergeJoin" @@ -536,7 +559,7 @@ fn shift_right_required( let offset = col.index() - left_columns_len; req.expr = Arc::new(Column::new(col.name(), offset)); } - Ok(parent_required) + Ok(parent_ordering) } /// Handles the custom pushdown of parent-required sorting requirements down to diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index ce23fe26528c3..26fef6d666b10 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -370,7 +370,7 @@ query TT explain select number, letter, age, number as column4, letter as column5 from partial_sorted order by number desc, column4 desc, letter asc, column5 asc, age desc limit 3; ---- physical_plan -01)SortExec: TopK(fetch=3), expr=[number@0 DESC, column4@3 DESC, letter@1 ASC NULLS LAST, column5@4 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] +01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[number@0 as number, letter@1 as letter, age@2 as age, number@0 as column4, letter@1 as column5] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet @@ -380,7 +380,7 @@ explain select number + 1 as number_plus, number, number + 1 as other_number_plu ---- physical_plan 01)SortPreservingMergeExec: [number_plus@0 DESC, number@1 DESC, other_number_plus@2 DESC, age@3 ASC NULLS LAST], fetch=3 -02)--SortExec: TopK(fetch=3), expr=[number_plus@0 DESC, number@1 DESC, other_number_plus@2 DESC, age@3 ASC NULLS LAST], preserve_partitioning=[true], sort_prefix=[number_plus@0 DESC, number@1 DESC] +02)--SortExec: TopK(fetch=3), expr=[number_plus@0 DESC, number@1 DESC, age@3 ASC NULLS LAST], preserve_partitioning=[true], sort_prefix=[number_plus@0 DESC, number@1 DESC] 03)----ProjectionExec: expr=[__common_expr_1@0 as number_plus, number@1 as number, __common_expr_1@0 as other_number_plus, age@2 as age] 04)------ProjectionExec: expr=[CAST(number@0 AS Int64) + 1 as __common_expr_1, number@0 as number, age@1 as age] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 From 653ff8776a950a7e6763e142a3741c269c33262e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 May 2025 16:03:36 +0300 Subject: [PATCH 134/167] Reduce time complexity of normalization w.r.t. number of equivalence classes --- .../physical-expr/src/equivalence/class.rs | 164 +++++++++++------- 1 file changed, 103 insertions(+), 61 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index b3ebabba0d0ec..60bc76b6dd816 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -26,7 +26,7 @@ use crate::expressions::{Column, Literal}; use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{JoinType, ScalarValue}; +use datafusion_common::{HashMap, JoinType, ScalarValue}; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; @@ -306,57 +306,71 @@ type AugmentedMapping<'a> = IndexMap< (&'a ProjectionTargets, Option<&'a EquivalenceClass>), >; -/// A collection of distinct `EquivalenceClass`es +/// A collection of distinct `EquivalenceClass`es. This object supports fast +/// lookups of expressions and their equivalence classes. #[derive(Clone, Debug, Default)] pub struct EquivalenceGroup { + /// A mapping from expressions to their equivalence class key. + map: HashMap, usize>, + /// The equivalence classes in this group. classes: Vec, } impl EquivalenceGroup { /// Creates an equivalence group from the given equivalence classes. pub fn new(classes: impl IntoIterator) -> Self { + let classes = classes.into_iter().collect::>(); let mut result = Self { - classes: classes.into_iter().collect(), + map: classes + .iter() + .enumerate() + .flat_map(|(idx, cls)| { + cls.iter().map(move |expr| (Arc::clone(expr), idx)) + }) + .collect(), + classes, }; result.remove_redundant_entries(); result } - /// Returns an iterator over the equivalence classes in this group. - pub fn iter_mut(&mut self) -> impl Iterator { - self.classes.iter_mut() - } - /// Adds `expr` as a constant expression to this equivalence group. pub fn add_constant(&mut self, const_expr: ConstExpr) { - for cls in self.classes.iter_mut() { - if cls.contains(&const_expr.expr) { - // If the expression is already in an equivalence class, we - // should adjust the constant-ness of the class if necessary: - if let Some(across) = cls.constant.as_mut() { - // TODO: Return an error if constant values do not agree. - if *across == AcrossPartitions::Heterogeneous { - *across = const_expr.across_partitions; - } - } else { - cls.constant = Some(const_expr.across_partitions); + // If the expression is already in an equivalence class, we should + // adjust the constant-ness of the class if necessary: + if let Some(idx) = self.map.get(&const_expr.expr) { + let cls = &mut self.classes[*idx]; + if let Some(across) = cls.constant.as_mut() { + // TODO: Return an error if constant values do not agree. + if *across == AcrossPartitions::Heterogeneous { + *across = const_expr.across_partitions; } - return; - } else if let Some(across @ AcrossPartitions::Uniform(_)) = &cls.constant { - // If the expression is not in some equivalence class, but has - // the same constant value with it, add it to that class: - if const_expr.across_partitions.eq(across) { + } else { + cls.constant = Some(const_expr.across_partitions); + } + return; + } + // If the expression is not in any equivalence class, but has the same + // constant value with some class, add it to that class: + if let AcrossPartitions::Uniform(_) = &const_expr.across_partitions { + for (idx, cls) in self.classes.iter_mut().enumerate() { + if cls + .constant + .as_ref() + .is_some_and(|across| const_expr.across_partitions.eq(across)) + { + self.map.insert(Arc::clone(&const_expr.expr), idx); cls.push(const_expr.expr); return; } } } - // If the expression is not in *any* equivalence class, create a new - // one with the expression as the only member: + // Otherwise, create a new class with the expression as the only member: let mut new_class = EquivalenceClass::new(std::iter::once(const_expr.expr)); if new_class.constant.is_none() { new_class.constant = Some(const_expr.across_partitions); } + Self::update_lookup_table(&mut self.map, &new_class, self.classes.len()); self.classes.push(new_class); } @@ -369,7 +383,7 @@ impl EquivalenceGroup { if let Some(AcrossPartitions::Heterogeneous) = cls.constant { if cls.len() == 1 { // If this class becomes trivial, remove it entirely: - self.classes.swap_remove(idx); + self.remove_class_at_idx(idx); continue; } else { cls.constant = None; @@ -388,20 +402,8 @@ impl EquivalenceGroup { left: Arc, right: Arc, ) -> bool { - let mut idx = 0; - let size = self.classes.len(); - let mut first_class = None; - let mut second_class = None; - while (idx < size) && (first_class.is_none() || second_class.is_none()) { - let cls = &self.classes[idx]; - if first_class.is_none() && cls.contains(&left) { - first_class = Some(idx); - } - if second_class.is_none() && cls.contains(&right) { - second_class = Some(idx); - } - idx += 1; - } + let first_class = self.map.get(&left).copied(); + let second_class = self.map.get(&right).copied(); match (first_class, second_class) { (Some(mut first_idx), Some(mut second_idx)) => { // If the given left and right sides belong to different classes, @@ -418,30 +420,70 @@ impl EquivalenceGroup { // Remove the class at `second_idx` and merge its values with // the class at `first_idx`. The convention above makes sure // that `first_idx` is still valid after removing `second_idx`. - let other_class = self.classes.swap_remove(second_idx); + let other_class = self.remove_class_at_idx(second_idx); + // Update the lookup table for the second class: + Self::update_lookup_table(&mut self.map, &other_class, first_idx); self.classes[first_idx].extend(other_class); } (Some(group_idx), None) => { // Right side is new, extend left side's class: + self.map.insert(Arc::clone(&right), group_idx); self.classes[group_idx].push(right); } (None, Some(group_idx)) => { // Left side is new, extend right side's class: + self.map.insert(Arc::clone(&left), group_idx); self.classes[group_idx].push(left); } (None, None) => { // None of the expressions is among existing classes. // Create a new equivalence class and extend the group. - self.classes.push(EquivalenceClass::new([left, right])); + let class = EquivalenceClass::new([left, right]); + Self::update_lookup_table(&mut self.map, &class, self.classes.len()); + self.classes.push(class); } } true } + /// Removes the equivalence class at the given index from this group. + fn remove_class_at_idx(&mut self, idx: usize) -> EquivalenceClass { + // Remove the class at the given index: + let cls = self.classes.swap_remove(idx); + // Remove its entries from the lookup table: + for expr in cls.iter() { + self.map.remove(expr); + } + // Update the lookup table for the moved class: + if idx < self.classes.len() { + Self::update_lookup_table(&mut self.map, &self.classes[idx], idx); + } + cls + } + + /// Updates the entry in lookup table for the given equivalence class with + /// the given index. + fn update_lookup_table( + map: &mut HashMap, usize>, + cls: &EquivalenceClass, + idx: usize, + ) { + for expr in cls.iter() { + map.insert(Arc::clone(expr), idx); + } + } + /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { // First, remove trivial equivalence classes: - self.classes.retain(|cls| !cls.is_trivial()); + let mut idx = 0; + while idx < self.classes.len() { + if self.classes[idx].is_trivial() { + self.remove_class_at_idx(idx); + } else { + idx += 1; + } + } // Then, unify/bridge groups that have common expressions: self.bridge_classes() } @@ -457,7 +499,8 @@ impl EquivalenceGroup { let start_size = self.classes[idx].len(); while next_idx < self.classes.len() { if self.classes[idx].contains_any(&self.classes[next_idx]) { - let extension = self.classes.swap_remove(next_idx); + let extension = self.remove_class_at_idx(next_idx); + Self::update_lookup_table(&mut self.map, &extension, idx); self.classes[idx].extend(extension); } else { next_idx += 1; @@ -472,25 +515,24 @@ impl EquivalenceGroup { /// Extends this equivalence group with the `other` equivalence group. pub fn extend(&mut self, other: Self) { + for (idx, cls) in other.classes.iter().enumerate() { + // Update the lookup table for the new class: + Self::update_lookup_table(&mut self.map, cls, idx); + } self.classes.extend(other.classes); self.remove_redundant_entries(); } /// Normalizes the given physical expression according to this group. The - /// expression is replaced with the first expression in the equivalence - /// class it matches with (if any). + /// expression is replaced with the first (canonical) expression in the + /// equivalence class it matches with (if any). pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.transform(|expr| { - for cls in self.iter() { - // If the equivalence class is non-empty, and it contains this - // expression, use its canonical version: - if let Some(canonical) = cls.canonical_expr() { - if cls.contains(&expr) { - return Ok(Transformed::yes(Arc::clone(canonical))); - } - } - } - Ok(Transformed::no(expr)) + let cls = self.get_equivalence_class(&expr); + let Some(canonical) = cls.and_then(|cls| cls.canonical_expr()) else { + return Ok(Transformed::no(expr)); + }; + Ok(Transformed::yes(Arc::clone(canonical))) }) .data() .unwrap() @@ -576,7 +618,7 @@ impl EquivalenceGroup { // the mapping, then we can project. For example, if we have the // mapping `(a as a1, a + c)` and the equivalence `a == b`, // expression `b` projects to `a1`. - if eq_class.as_ref().is_some_and(|group| group.contains(expr)) { + if eq_class.as_ref().is_some_and(|cls| cls.contains(expr)) { let (target, _) = targets.first(); return Some(Arc::clone(target)); } @@ -711,8 +753,8 @@ impl EquivalenceGroup { if let Some(lit) = expr.as_any().downcast_ref::() { return Some(AcrossPartitions::Uniform(Some(lit.value().clone()))); } - for cls in self.iter() { - if cls.constant.is_some() && cls.contains(expr) { + if let Some(cls) = self.get_equivalence_class(expr) { + if cls.constant.is_some() { return cls.constant.clone(); } } @@ -735,7 +777,7 @@ impl EquivalenceGroup { &self, expr: &Arc, ) -> Option<&EquivalenceClass> { - self.iter().find(|cls| cls.contains(expr)) + self.map.get(expr).map(|idx| &self.classes[*idx]) } /// Combine equivalence groups of the given join children. From d6f73198ca7d0d0ca1a35fc57266051f0a6920d5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 May 2025 17:15:53 +0300 Subject: [PATCH 135/167] Simplify bridge_classes logic --- .../physical-expr/src/equivalence/class.rs | 21 ++++++------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 60bc76b6dd816..7bc1702815111 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -476,12 +476,9 @@ impl EquivalenceGroup { /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { // First, remove trivial equivalence classes: - let mut idx = 0; - while idx < self.classes.len() { + for idx in (0..self.classes.len()).rev() { if self.classes[idx].is_trivial() { self.remove_class_at_idx(idx); - } else { - idx += 1; } } // Then, unify/bridge groups that have common expressions: @@ -494,21 +491,15 @@ impl EquivalenceGroup { /// equal and belong to one class. This utility converts merges such classes. fn bridge_classes(&mut self) { let mut idx = 0; - while idx < self.classes.len() { - let mut next_idx = idx + 1; - let start_size = self.classes[idx].len(); - while next_idx < self.classes.len() { - if self.classes[idx].contains_any(&self.classes[next_idx]) { - let extension = self.remove_class_at_idx(next_idx); + 'scan: while idx < self.classes.len() { + for other_idx in (idx + 1..self.classes.len()).rev() { + if self.classes[idx].contains_any(&self.classes[other_idx]) { + let extension = self.remove_class_at_idx(other_idx); Self::update_lookup_table(&mut self.map, &extension, idx); self.classes[idx].extend(extension); - } else { - next_idx += 1; + continue 'scan; } } - if self.classes[idx].len() > start_size { - continue; - } idx += 1; } } From 8cea6b4c7d9f758317f65252c21ca48a4a303512 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 May 2025 19:43:32 +0300 Subject: [PATCH 136/167] Remove TODOs --- .../physical-expr/src/equivalence/class.rs | 12 ++++-- .../src/equivalence/properties/mod.rs | 41 ++++++++++--------- 2 files changed, 29 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7bc1702815111..7a9cfb812066d 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -395,8 +395,8 @@ impl EquivalenceGroup { /// Adds the equality `left` = `right` to this equivalence group. New /// equality conditions often arise after steps like `Filter(a = b)`, - /// `Alias(a, a as b)` etc. Returns whether the given equality was - /// material (i.e. it was not already known). + /// `Alias(a, a as b)` etc. Returns whether the given equality defines + /// a new equivalence class. pub fn add_equal_conditions( &mut self, left: Arc, @@ -441,9 +441,10 @@ impl EquivalenceGroup { let class = EquivalenceClass::new([left, right]); Self::update_lookup_table(&mut self.map, &class, self.classes.len()); self.classes.push(class); + return true; } } - true + false } /// Removes the equivalence class at the given index from this group. @@ -505,13 +506,16 @@ impl EquivalenceGroup { } /// Extends this equivalence group with the `other` equivalence group. - pub fn extend(&mut self, other: Self) { + /// Returns whether the extension resulted in new equivalence classes. + pub fn extend(&mut self, other: Self) -> bool { + let n_classes = self.classes.len(); for (idx, cls) in other.classes.iter().enumerate() { // Update the lookup table for the new class: Self::update_lookup_table(&mut self.map, cls, idx); } self.classes.extend(other.classes); self.remove_redundant_entries(); + self.classes.len() > n_classes } /// Normalizes the given physical expression according to this group. The diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 5cae22f99e1e5..a38634eb7fdca 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -232,7 +232,7 @@ impl EquivalenceProperties { &mut self, orderings: impl IntoIterator>, ) { - // TODO: Normalize `orderings` before adding: + // Normalize given orderings before adding: self.oeq_class.add_orderings( orderings .into_iter() @@ -248,14 +248,14 @@ impl EquivalenceProperties { /// Incorporates the given equivalence group to into the existing /// equivalence group within. pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { - self.eq_group.extend(other_eq_group); - // TODO: Renormalize the orderings after adding new equivalences: - let oeq_class = mem::take(&mut self.oeq_class); - self.oeq_class = OrderingEquivalenceClass::new( - oeq_class + if self.eq_group.extend(other_eq_group) { + // Renormalize the orderings after adding new equivalence classes: + let oeq_class = mem::take(&mut self.oeq_class); + let orderings = oeq_class .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)), - ); + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_class = OrderingEquivalenceClass::new(orderings); + } } /// Adds a new equality condition into the existing equivalence group. @@ -268,13 +268,12 @@ impl EquivalenceProperties { ) -> Result<()> { // Add equal expressions to the state: if self.eq_group.add_equal_conditions(Arc::clone(&left), right) { - // TODO: Renormalize the orderings after adding the new equality: + // Renormalize the orderings after adding a new equivalence class: let oeq_class = mem::take(&mut self.oeq_class); - self.oeq_class = OrderingEquivalenceClass::new( - oeq_class - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)), - ); + let orderings = oeq_class + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_class = OrderingEquivalenceClass::new(orderings); // Discover any new orderings: self.discover_new_orderings(left)?; } @@ -287,13 +286,15 @@ impl EquivalenceProperties { for constant in constants { self.eq_group.add_constant(constant); } - // TODO: Renormalize the orderings after adding new constants: + // Renormalize the orderings after adding new constants by removing + // the constants from existing orderings: let oeq_class = mem::take(&mut self.oeq_class); - self.oeq_class = OrderingEquivalenceClass::new( - oeq_class - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)), - ); + let orderings = oeq_class.into_iter().map(|ordering| { + ordering.into_iter().filter(|sort_expr| { + self.eq_group.is_expr_constant(&sort_expr.expr).is_none() + }) + }); + self.oeq_class = OrderingEquivalenceClass::new(orderings); // Discover any new orderings based on the constants: for ordering in self.oeq_class.clone() { let leading = Arc::clone(&ordering[0].expr); From bf39ae78ca4cfda2c971934716e43ebbd4ecd7bd Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 May 2025 20:13:27 +0300 Subject: [PATCH 137/167] Simplify generate_dependency_orderings --- .../physical-expr/src/equivalence/properties/dependency.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 4b79d688ad5f6..c29ac0affb111 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -359,7 +359,7 @@ pub fn generate_dependency_orderings( // dependency, meaning that dependent is a leading ordering. dependencies .iter() - .flat_map(|dep| { + .filter_map(|dep| { let prefixes = construct_prefix_orderings(dep, dependency_map); (!prefixes.is_empty()).then_some(prefixes) }) @@ -376,7 +376,6 @@ pub fn generate_dependency_orderings( acc }) }) - .collect::>() }) .collect() } From ce84d2f1ba97021f8ce0397d85c3596f249136c2 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 May 2025 22:06:56 +0300 Subject: [PATCH 138/167] normalized orderings - 2 --- .../src/equivalence/properties/mod.rs | 65 ++++++++----------- 1 file changed, 27 insertions(+), 38 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index a38634eb7fdca..dc829355e81f3 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -890,7 +890,7 @@ impl EquivalenceProperties { let normalized_source = self.eq_group.normalize_expr(Arc::clone(source)); (normalized_source, target.clone()) }) - .collect::() + .collect() } /// Computes projected orderings based on a given projection mapping. @@ -909,12 +909,12 @@ impl EquivalenceProperties { /// /// # Returns /// - /// A vector of `LexOrdering` containing all valid orderings after projection. + /// A vector of all valid (un-normalized) orderings after projection. fn projected_orderings( &self, mapping: &ProjectionMapping, mut oeq_class: OrderingEquivalenceClass, - ) -> OrderingEquivalenceClass { + ) -> Vec { // Normalize source expressions in the mapping: let mapping = self.normalized_mapping(mapping); // Get dependency map for existing orderings: @@ -923,14 +923,13 @@ impl EquivalenceProperties { let orderings = mapping.iter().flat_map(|(source, targets)| { referred_dependencies(&dependency_map, source) .into_iter() - .filter_map(|relevant_deps| { - if let Ok(SortProperties::Ordered(options)) = - get_expr_properties(source, &relevant_deps, &self.schema) - .map(|prop| prop.sort_properties) - { - Some((options, relevant_deps)) + .filter_map(|deps| { + let ep = get_expr_properties(source, &deps, &self.schema); + let sort_properties = ep.map(|prop| prop.sort_properties); + if let Ok(SortProperties::Ordered(options)) = sort_properties { + Some((options, deps)) } else { - // Do not consider unordered cases + // Do not consider unordered cases. None } }) @@ -981,29 +980,24 @@ impl EquivalenceProperties { prefixes }); - // Simplify each ordering by removing redundant sections: - orderings - .chain(projected_orderings) - .map(|lex_ordering| lex_ordering.collapse()) - .collect::>() - .into() + orderings.chain(projected_orderings).collect() } /// Projects constraints according to the given projection mapping. /// - /// This function takes a projection mapping and extracts the column indices of the target columns. - /// It then projects the constraints to only include relationships between - /// columns that exist in the projected output. + /// This function takes a projection mapping and extracts column indices of + /// target columns. It then projects the constraints to only include + /// relationships between columns that exist in the projected output. /// - /// # Arguments + /// # Parameters /// - /// * `mapping` - A reference to `ProjectionMapping` that defines how expressions are mapped - /// in the projection operation + /// * `mapping` - A reference to the `ProjectionMapping` that defines the + /// projection operation. /// /// # Returns /// - /// Returns a new `Constraints` object containing only the constraints - /// that are valid for the projected columns. + /// Returns an optional `Constraints` object containing only the constraints + /// that are valid for the projected columns (if any exists). fn projected_constraints(&self, mapping: &ProjectionMapping) -> Option { let indices = mapping .iter() @@ -1019,22 +1013,17 @@ impl EquivalenceProperties { /// Projects the equivalences within according to `mapping` and /// `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { - let mut proj_eqp = Self { - eq_group: self.eq_group.project(mapping), - oeq_class: self.projected_orderings(mapping, self.oeq_class.clone()), + let eq_group = self.eq_group.project(mapping); + let orderings = self + .projected_orderings(mapping, self.oeq_class.clone()) + .into_iter() + .map(|o| eq_group.normalize_sort_exprs(o)); + Self { + oeq_class: OrderingEquivalenceClass::new(orderings), constraints: self.projected_constraints(mapping).unwrap_or_default(), schema: output_schema, - }; - // TODO: Remove renormalization after fixing `projected_orderings` to - // take the new eq. group as an argument and emit normalized - // orderings. - proj_eqp.oeq_class = OrderingEquivalenceClass::new( - proj_eqp - .oeq_class - .into_iter() - .map(|o| proj_eqp.eq_group.normalize_sort_exprs(o)), - ); - proj_eqp + eq_group, + } } /// Returns the longest (potentially partial) permutation satisfying the From 0c99d3dab52cb87d246e51f8323bac4830a5b7ab Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 10 May 2025 14:53:27 +0300 Subject: [PATCH 139/167] normalized orderings - 3 --- .../physical-expr-common/src/sort_expr.rs | 2 +- .../physical-expr/src/equivalence/class.rs | 9 ++-- .../physical-expr/src/equivalence/ordering.rs | 28 +---------- .../src/equivalence/properties/mod.rs | 50 +++++++++++++++---- 4 files changed, 44 insertions(+), 45 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index d317aea22c088..ce51dea0811e8 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -329,7 +329,7 @@ impl From for PhysicalSortExpr { /// For example, a `vec![a ASC, b DESC]` represents a lexicographical ordering /// that first sorts by column `a` in ascending order, then by column `b` in /// descending order. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct LexOrdering { exprs: Vec, } diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7a9cfb812066d..a80b61da93781 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -506,16 +506,13 @@ impl EquivalenceGroup { } /// Extends this equivalence group with the `other` equivalence group. - /// Returns whether the extension resulted in new equivalence classes. - pub fn extend(&mut self, other: Self) -> bool { - let n_classes = self.classes.len(); + pub fn extend(&mut self, other: Self) { for (idx, cls) in other.classes.iter().enumerate() { // Update the lookup table for the new class: Self::update_lookup_table(&mut self.map, cls, idx); } self.classes.extend(other.classes); self.remove_redundant_entries(); - self.classes.len() > n_classes } /// Normalizes the given physical expression according to this group. The @@ -548,9 +545,9 @@ impl EquivalenceGroup { } /// Normalizes the given sort expressions (i.e. `sort_exprs`) by: - /// - Removing expressions that have a constant value. /// - Replacing sections that belong to some equivalence class in the /// with the first entry in the matching equivalence class. + /// - Removing expressions that have a constant value. /// - Removing duplicate sort expressions. /// /// If columns `a` and `b` are known to be equal, `d` is known to be a @@ -581,9 +578,9 @@ impl EquivalenceGroup { } /// Normalizes the given sort requirements (i.e. `sort_reqs`) by: - /// - Removing expressions that have a constant value. /// - Replacing sections that belong to some equivalence class in the /// with the first entry in the matching equivalence class. + /// - Removing expressions that have a constant value. /// - Removing duplicate sort expressions. /// /// If columns `a` and `b` are known to be equal, `d` is known to be a diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d546bae1b6962..4037e5c63beef 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -16,7 +16,6 @@ // under the License. use std::fmt::Display; -use std::hash::Hash; use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; @@ -44,7 +43,7 @@ use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// /// Here, both `[a ASC, b ASC]` and `[c DESC, d ASC]` describe the table /// ordering. In this case, we say that these orderings are equivalent. -#[derive(Clone, Debug, Default, Eq, PartialEq, Hash)] +#[derive(Clone, Debug, Default, Eq, PartialEq)] pub struct OrderingEquivalenceClass { orderings: Vec, } @@ -210,31 +209,6 @@ impl OrderingEquivalenceClass { /// added as a constant during `ordering_satisfy_requirement()` iterations /// after the corresponding prefix requirement is satisfied. /// - /// ### Example Scenarios - /// - /// In these scenarios, we assume that all expressions share the same sort - /// properties. - /// - /// #### Case 1: Sort Requirement `[a, c]` - /// - /// **Existing Orderings:** `[[a, b, c], [a, d]]`, **Constants:** `[]` - /// 1. `ordering_satisfy_single()` returns `true` because the requirement - /// `a` is satisfied by `[a, b, c].first()`. - /// 2. `a` is added as a constant for the next iteration. - /// 3. The normalized orderings become `[[b, c], [d]]`. - /// 4. `ordering_satisfy_single()` returns `false` for `c`, as neither - /// `[b, c]` nor `[d]` satisfies `c`. - /// - /// #### Case 2: Sort Requirement `[a, d]` - /// - /// **Existing Orderings:** `[[a, b, c], [a, d]]`, **Constants:** `[]` - /// 1. `ordering_satisfy_single()` returns `true` because the requirement - /// `a` is satisfied by `[a, b, c].first()`. - /// 2. `a` is added as a constant for the next iteration. - /// 3. The normalized orderings become `[[b, c], [d]]`. - /// 4. `ordering_satisfy_single()` returns `true` for `d`, as `[d]` satisfies - /// `d`. - /// /// ### Future Improvements /// /// This function may become unnecessary if any of the following improvements diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index dc829355e81f3..5d2fce0d7d932 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -248,13 +248,16 @@ impl EquivalenceProperties { /// Incorporates the given equivalence group to into the existing /// equivalence group within. pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { - if self.eq_group.extend(other_eq_group) { - // Renormalize the orderings after adding new equivalence classes: - let oeq_class = mem::take(&mut self.oeq_class); - let orderings = oeq_class - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)); - self.oeq_class = OrderingEquivalenceClass::new(orderings); + self.eq_group.extend(other_eq_group); + // Renormalize orderings after modifying equivalence classes: + let oeq_class = mem::take(&mut self.oeq_class); + let orderings = oeq_class + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_class = OrderingEquivalenceClass::new(orderings); + for ordering in self.oeq_class.clone() { + let leading = Arc::clone(&ordering[0].expr); + self.discover_new_orderings(leading).unwrap(); } } @@ -424,8 +427,9 @@ impl EquivalenceProperties { LexRequirement::new(self.eq_group.normalize_sort_requirements(sort_reqs)) } - /// Checks whether the given ordering is satisfied by any of the existing - /// orderings. + /// Iteratively checks whether the given ordering is satisfied by any of + /// the existing orderings. See [`Self::ordering_satisfy_requirement`] for + /// more details and examples. pub fn ordering_satisfy( &self, given: impl IntoIterator, @@ -439,8 +443,32 @@ impl EquivalenceProperties { self.common_sort_prefix_length(normalized_ordering) == length } - /// Checks whether the given sort requirements are satisfied by any of the - /// existing orderings. + /// Iteratively checks whether the given sort requirement is satisfied by + /// any of the existing orderings. + /// + /// ### Example Scenarios + /// + /// In these scenarios, assume that all expressions share the same sort + /// properties. + /// + /// #### Case 1: Sort Requirement `[a, c]` + /// + /// **Existing orderings:** `[[a, b, c], [a, d]]`, **constants:** `[]` + /// 1. The function first checks the leading requirement `a`, which is + /// satisfied by `[a, b, c].first()`. + /// 2. `a` is added as a constant for the next iteration. + /// 3. Normalized orderings become `[[b, c], [d]]`. + /// 4. The function fails for `c` in the second iteration, as neither + /// `[b, c]` nor `[d]` satisfies `c`. + /// + /// #### Case 2: Sort Requirement `[a, d]` + /// + /// **Existing orderings:** `[[a, b, c], [a, d]]`, **constants:** `[]` + /// 1. The function first checks the leading requirement `a`, which is + /// satisfied by `[a, b, c].first()`. + /// 2. `a` is added as a constant for the next iteration. + /// 3. Normalized orderings become `[[b, c], [d]]`. + /// 4. The function returns `true` as `[d]` satisfies `d`. pub fn ordering_satisfy_requirement( &self, given: impl IntoIterator, From 1c2b911057e35a3c2822ee060799ca23207bbb2a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 11 May 2025 12:14:21 +0300 Subject: [PATCH 140/167] undo normalized orderings --- .../src/equivalence/properties/dependency.rs | 11 +- .../src/equivalence/properties/joins.rs | 8 +- .../src/equivalence/properties/mod.rs | 192 ++++++++---------- .../src/equivalence/properties/union.rs | 2 +- 4 files changed, 100 insertions(+), 113 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index c29ac0affb111..6d44672d24d6f 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -511,17 +511,16 @@ mod tests { let col_c_expr = col("c", &schema)?; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - eq_properties - .add_equal_conditions(Arc::clone(&col_a_expr), Arc::clone(&col_c_expr))?; + eq_properties.add_equal_conditions(col_a_expr, Arc::clone(&col_c_expr))?; eq_properties.add_orderings([ vec![PhysicalSortExpr::new_default(Arc::clone(&col_b_expr))], - vec![PhysicalSortExpr::new_default(col_c_expr)], + vec![PhysicalSortExpr::new_default(Arc::clone(&col_c_expr))], ]); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_orderings([ vec![PhysicalSortExpr::new_default(col_b_expr)], - vec![PhysicalSortExpr::new_default(col_a_expr)], + vec![PhysicalSortExpr::new_default(col_c_expr)], ]); assert!(eq_properties.oeq_class().eq(expected_eqs.oeq_class())); @@ -1136,11 +1135,11 @@ mod tests { ]); // Add equality condition c = concat(a, b) - eq_properties.add_equal_conditions(Arc::clone(&col_c), a_concat_b)?; + eq_properties.add_equal_conditions(col_c, Arc::clone(&a_concat_b))?; let orderings = eq_properties.oeq_class(); - let expected_ordering1 = [PhysicalSortExpr::new_default(col_c).asc()].into(); + let expected_ordering1 = [PhysicalSortExpr::new_default(a_concat_b).asc()].into(); let expected_ordering2 = [ PhysicalSortExpr::new_default(col_a).asc(), PhysicalSortExpr::new_default(col_b).asc(), diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index f345d480f3509..e2f1171ca316b 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -263,8 +263,8 @@ mod tests { let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); // a=x and d=w - join_eq_properties.add_equal_conditions(Arc::clone(&col_a), col_x)?; - join_eq_properties.add_equal_conditions(Arc::clone(&col_d), col_w)?; + join_eq_properties.add_equal_conditions(col_a, Arc::clone(&col_x))?; + join_eq_properties.add_equal_conditions(col_d, Arc::clone(&col_w))?; updated_right_ordering_equivalence_class( &mut right_oeq_class, @@ -276,8 +276,8 @@ mod tests { // [x ASC, y ASC], [z ASC, w ASC] let orderings = vec![ - vec![(col_a, option_asc), (col_y, option_asc)], - vec![(col_z, option_asc), (col_d, option_asc)], + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], ]; let orderings = convert_to_orderings(&orderings); let expected = OrderingEquivalenceClass::from(orderings); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 5d2fce0d7d932..7ed8ca26945e1 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -204,7 +204,14 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - self.oeq_class.output_ordering() + let mut sort_exprs: Vec<_> = self.oeq_class.output_ordering()?.into(); + // Prune out constant expressions: + sort_exprs.retain(|sort_expr| { + self.eq_group + .get_equivalence_class(&sort_expr.expr) + .is_none_or(|cls| cls.constant.is_none()) + }); + LexOrdering::new(sort_exprs) } /// Extends this `EquivalenceProperties` with the `other` object. @@ -232,12 +239,8 @@ impl EquivalenceProperties { &mut self, orderings: impl IntoIterator>, ) { - // Normalize given orderings before adding: - self.oeq_class.add_orderings( - orderings - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)), - ); + // TODO: Normalization point. + self.oeq_class.add_orderings(orderings); } /// Adds a single ordering to the existing ordering equivalence class. @@ -249,13 +252,9 @@ impl EquivalenceProperties { /// equivalence group within. pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { self.eq_group.extend(other_eq_group); - // Renormalize orderings after modifying equivalence classes: - let oeq_class = mem::take(&mut self.oeq_class); - let orderings = oeq_class - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)); - self.oeq_class = OrderingEquivalenceClass::new(orderings); - for ordering in self.oeq_class.clone() { + // TODO: Normalization point. + // Discover any new orderings based on the new equivalence classes: + for ordering in self.normalized_oeq_class() { let leading = Arc::clone(&ordering[0].expr); self.discover_new_orderings(leading).unwrap(); } @@ -271,12 +270,7 @@ impl EquivalenceProperties { ) -> Result<()> { // Add equal expressions to the state: if self.eq_group.add_equal_conditions(Arc::clone(&left), right) { - // Renormalize the orderings after adding a new equivalence class: - let oeq_class = mem::take(&mut self.oeq_class); - let orderings = oeq_class - .into_iter() - .map(|o| self.eq_group.normalize_sort_exprs(o)); - self.oeq_class = OrderingEquivalenceClass::new(orderings); + // TODO: Normalization point. // Discover any new orderings: self.discover_new_orderings(left)?; } @@ -289,40 +283,44 @@ impl EquivalenceProperties { for constant in constants { self.eq_group.add_constant(constant); } - // Renormalize the orderings after adding new constants by removing - // the constants from existing orderings: - let oeq_class = mem::take(&mut self.oeq_class); - let orderings = oeq_class.into_iter().map(|ordering| { - ordering.into_iter().filter(|sort_expr| { - self.eq_group.is_expr_constant(&sort_expr.expr).is_none() - }) - }); - self.oeq_class = OrderingEquivalenceClass::new(orderings); + // TODO: Normalization point. // Discover any new orderings based on the constants: - for ordering in self.oeq_class.clone() { + for ordering in self.normalized_oeq_class() { let leading = Arc::clone(&ordering[0].expr); self.discover_new_orderings(leading).unwrap(); } } + /// Returns the ordering equivalence class within in normal form. + /// Normalization standardizes expressions according to the equivalence + /// group within, and removes constants/duplicates. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { + self.oeq_class + .iter() + .cloned() + .filter_map(|ordering| self.normalize_sort_exprs(ordering)) + .collect::>() + .into() + } + /// Discover new valid orderings in light of a new equality. Accepts a single /// argument (`expr`) which is used to determine the orderings to update. /// When constants or equivalence classes change, there may be new orderings /// that can be discovered with the new equivalence properties. /// For a discussion, see: fn discover_new_orderings(&mut self, expr: Arc) -> Result<()> { - let normalized_expr = self.eq_group.normalize_expr(expr); + let normal_expr = self.eq_group.normalize_expr(expr); let eq_class = self .eq_group - .get_equivalence_class(&normalized_expr) + .get_equivalence_class(&normal_expr) .map_or_else( - || vec![Arc::clone(&normalized_expr)], + || vec![Arc::clone(&normal_expr)], |class| class.clone().into(), ); let mut new_orderings = vec![]; - for ordering in self.oeq_class.iter() { - if !ordering[0].expr.eq(&normalized_expr) { + for ordering in self.normalized_oeq_class() { + if !ordering[0].expr.eq(&normal_expr) { continue; } @@ -380,15 +378,15 @@ impl EquivalenceProperties { ordering: impl IntoIterator, ) -> Self { // Normalize the given ordering and process: - if let Some(normalized_ordering) = self.normalize_sort_exprs(ordering) { + if let Some(normal_ordering) = self.normalize_sort_exprs(ordering) { // Preserve valid suffixes from existing orderings: let mut orderings: Vec<_> = mem::take(&mut self.oeq_class).into(); orderings.retain(|existing| { // Check if the existing ordering is a prefix of the new ordering: - self.is_prefix_of(&normalized_ordering, existing) + self.is_prefix_of(&normal_ordering, existing) }); if orderings.is_empty() { - orderings.push(normalized_ordering); + orderings.push(normal_ordering); } self.oeq_class = OrderingEquivalenceClass::new(orderings); } @@ -435,12 +433,11 @@ impl EquivalenceProperties { given: impl IntoIterator, ) -> bool { // First, standardize the given ordering: - let Some(normalized_ordering) = self.normalize_sort_exprs(given) else { + let Some(normal_ordering) = self.normalize_sort_exprs(given) else { // If the ordering vanishes after normalization, it is satisfied: return true; }; - let length = normalized_ordering.len(); - self.common_sort_prefix_length(normalized_ordering) == length + normal_ordering.len() == self.common_sort_prefix_length(normal_ordering) } /// Iteratively checks whether the given sort requirement is satisfied by @@ -457,7 +454,7 @@ impl EquivalenceProperties { /// 1. The function first checks the leading requirement `a`, which is /// satisfied by `[a, b, c].first()`. /// 2. `a` is added as a constant for the next iteration. - /// 3. Normalized orderings become `[[b, c], [d]]`. + /// 3. Normal orderings become `[[b, c], [d]]`. /// 4. The function fails for `c` in the second iteration, as neither /// `[b, c]` nor `[d]` satisfies `c`. /// @@ -467,24 +464,24 @@ impl EquivalenceProperties { /// 1. The function first checks the leading requirement `a`, which is /// satisfied by `[a, b, c].first()`. /// 2. `a` is added as a constant for the next iteration. - /// 3. Normalized orderings become `[[b, c], [d]]`. + /// 3. Normal orderings become `[[b, c], [d]]`. /// 4. The function returns `true` as `[d]` satisfies `d`. pub fn ordering_satisfy_requirement( &self, given: impl IntoIterator, ) -> bool { // First, standardize the given requirement: - let Some(normalized_reqs) = self.normalize_sort_requirements(given) else { + let Some(normal_reqs) = self.normalize_sort_requirements(given) else { // If the requirement vanishes after normalization, it is satisfied: return true; }; // Then, check whether given requirement is satisfied by constraints: - if self.satisfied_by_constraints(&normalized_reqs) { + if self.satisfied_by_constraints(&normal_reqs) { return true; } let schema = self.schema(); let mut eq_properties = self.clone(); - for element in normalized_reqs { + for element in normal_reqs { // Check whether given requirement is satisfied: let ExprProperties { sort_properties, .. @@ -522,17 +519,17 @@ impl EquivalenceProperties { /// Returns the number of consecutive sort expressions (starting from the /// left) that are satisfied by the existing ordering. - fn common_sort_prefix_length(&self, normalized_ordering: LexOrdering) -> usize { - let full_length = normalized_ordering.len(); + fn common_sort_prefix_length(&self, normal_ordering: LexOrdering) -> usize { + let full_length = normal_ordering.len(); // Check whether the given ordering is satisfied by constraints: - if self.satisfied_by_constraints_ordering(&normalized_ordering) { + if self.satisfied_by_constraints_ordering(&normal_ordering) { // If constraints satisfy all sort expressions, return the full // length: return full_length; } let schema = self.schema(); let mut eq_properties = self.clone(); - for (idx, element) in normalized_ordering.into_iter().enumerate() { + for (idx, element) in normal_ordering.into_iter().enumerate() { // Check whether given ordering is satisfied: let ExprProperties { sort_properties, .. @@ -572,7 +569,7 @@ impl EquivalenceProperties { full_length } - /// Determines the longest normalized prefix of `ordering` satisfied by the + /// Determines the longest normal prefix of `ordering` satisfied by the /// existing ordering. Returns that prefix as a new `LexOrdering`, and a /// boolean indicating whether all the sort expressions are satisfied. pub fn extract_common_sort_prefix( @@ -580,13 +577,13 @@ impl EquivalenceProperties { ordering: LexOrdering, ) -> (Vec, bool) { // First, standardize the given ordering: - let Some(normalized_ordering) = self.normalize_sort_exprs(ordering) else { + let Some(normal_ordering) = self.normalize_sort_exprs(ordering) else { // If the ordering vanishes after normalization, it is satisfied: return (vec![], true); }; - let prefix_len = self.common_sort_prefix_length(normalized_ordering.clone()); - let flag = prefix_len == normalized_ordering.len(); - let mut sort_exprs: Vec<_> = normalized_ordering.into(); + let prefix_len = self.common_sort_prefix_length(normal_ordering.clone()); + let flag = prefix_len == normal_ordering.len(); + let mut sort_exprs: Vec<_> = normal_ordering.into(); if !flag { sort_exprs.truncate(prefix_len); } @@ -600,12 +597,12 @@ impl EquivalenceProperties { /// unique constraints, also verifies nullable columns. fn satisfied_by_constraints_ordering( &self, - normalized_exprs: &[PhysicalSortExpr], + normal_exprs: &[PhysicalSortExpr], ) -> bool { self.constraints.iter().any(|constraint| match constraint { Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { let check_null = matches!(constraint, Constraint::Unique(_)); - let normalized_size = normalized_exprs.len(); + let normalized_size = normal_exprs.len(); indices.len() <= normalized_size && self.oeq_class.iter().any(|ordering| { let length = ordering.len(); @@ -633,12 +630,9 @@ impl EquivalenceProperties { return false; } // Check if this ordering matches the prefix: - normalized_exprs - .iter() - .zip(ordering) - .all(|(given, existing)| { - existing.satisfy_expr(given, &self.schema) - }) + normal_exprs.iter().zip(ordering).all(|(given, existing)| { + existing.satisfy_expr(given, &self.schema) + }) }) } }) @@ -649,14 +643,11 @@ impl EquivalenceProperties { /// fully satisfies the requirements (i.e. constraint indices form a valid /// prefix of an existing ordering that matches the requirements). For /// unique constraints, also verifies nullable columns. - fn satisfied_by_constraints( - &self, - normalized_reqs: &[PhysicalSortRequirement], - ) -> bool { + fn satisfied_by_constraints(&self, normal_reqs: &[PhysicalSortRequirement]) -> bool { self.constraints.iter().any(|constraint| match constraint { Constraint::PrimaryKey(indices) | Constraint::Unique(indices) => { let check_null = matches!(constraint, Constraint::Unique(_)); - let normalized_size = normalized_reqs.len(); + let normalized_size = normal_reqs.len(); indices.len() <= normalized_size && self.oeq_class.iter().any(|ordering| { let length = ordering.len(); @@ -684,12 +675,9 @@ impl EquivalenceProperties { return false; } // Check if this ordering matches the prefix: - normalized_reqs - .iter() - .zip(ordering) - .all(|(given, existing)| { - existing.satisfy(given, &self.schema) - }) + normal_reqs.iter().zip(ordering).all(|(given, existing)| { + existing.satisfy(given, &self.schema) + }) }) } }) @@ -702,18 +690,17 @@ impl EquivalenceProperties { given: LexRequirement, reference: LexRequirement, ) -> bool { - let Some(normalized_given) = self.normalize_sort_requirements(given) else { + let Some(normal_given) = self.normalize_sort_requirements(given) else { return true; }; - let Some(normalized_reference) = self.normalize_sort_requirements(reference) - else { + let Some(normal_reference) = self.normalize_sort_requirements(reference) else { return true; }; - (normalized_reference.len() <= normalized_given.len()) - && normalized_reference + (normal_reference.len() <= normal_given.len()) + && normal_reference .into_iter() - .zip(normalized_given) + .zip(normal_given) .all(|(reference, given)| given.compatible(&reference)) } @@ -898,25 +885,24 @@ impl EquivalenceProperties { projectable } - /// Returns a new `ProjectionMapping` where source expressions are normalized. - /// - /// This normalization ensures that source expressions are transformed into a - /// consistent representation. This is beneficial for algorithms that rely on - /// exact equalities, as it allows for more precise and reliable comparisons. + /// Returns a new `ProjectionMapping` where source expressions are in normal + /// form. Normalization ensures that source expressions are transformed into + /// a consistent representation, which is beneficial for algorithms that rely + /// on exact equalities, as it allows for more precise and reliable comparisons. /// /// # Parameters /// - /// - `mapping`: A reference to the original `ProjectionMapping` to be normalized. + /// - `mapping`: A reference to the original `ProjectionMapping` to normalize. /// /// # Returns /// - /// A new `ProjectionMapping` with normalized source expressions. - fn normalized_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { + /// A new `ProjectionMapping` with source expressions in normal form. + fn normalize_mapping(&self, mapping: &ProjectionMapping) -> ProjectionMapping { mapping .iter() .map(|(source, target)| { - let normalized_source = self.eq_group.normalize_expr(Arc::clone(source)); - (normalized_source, target.clone()) + let normal_source = self.eq_group.normalize_expr(Arc::clone(source)); + (normal_source, target.clone()) }) .collect() } @@ -937,14 +923,14 @@ impl EquivalenceProperties { /// /// # Returns /// - /// A vector of all valid (un-normalized) orderings after projection. + /// A vector of all valid (but not in normal form) orderings after projection. fn projected_orderings( &self, mapping: &ProjectionMapping, mut oeq_class: OrderingEquivalenceClass, ) -> Vec { // Normalize source expressions in the mapping: - let mapping = self.normalized_mapping(mapping); + let mapping = self.normalize_mapping(mapping); // Get dependency map for existing orderings: oeq_class = Self::substitute_oeq_class(&self.schema, &mapping, oeq_class); let dependency_map = self.construct_dependency_map(oeq_class, &mapping); @@ -1008,7 +994,11 @@ impl EquivalenceProperties { prefixes }); - orderings.chain(projected_orderings).collect() + // Simplify each ordering by removing redundant sections: + orderings + .chain(projected_orderings) + .map(|lex_ordering| lex_ordering.collapse()) + .collect() } /// Projects constraints according to the given projection mapping. @@ -1042,10 +1032,8 @@ impl EquivalenceProperties { /// `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { let eq_group = self.eq_group.project(mapping); - let orderings = self - .projected_orderings(mapping, self.oeq_class.clone()) - .into_iter() - .map(|o| eq_group.normalize_sort_exprs(o)); + let orderings = self.projected_orderings(mapping, self.normalized_oeq_class()); + // TODO: Normalization point. Self { oeq_class: OrderingEquivalenceClass::new(orderings), constraints: self.projected_constraints(mapping).unwrap_or_default(), @@ -1311,15 +1299,15 @@ fn update_properties( Interval::make_unbounded(&node.expr.data_type(eq_properties.schema())?)? } // Now, check what we know about orderings: - let normalized_expr = eq_properties + let normal_expr = eq_properties .eq_group .normalize_expr(Arc::clone(&node.expr)); - let oeq_class = &eq_properties.oeq_class; - if eq_properties.is_expr_constant(&normalized_expr).is_some() - || oeq_class.is_expr_partial_const(&normalized_expr) + let oeq_class = eq_properties.normalized_oeq_class(); + if eq_properties.is_expr_constant(&normal_expr).is_some() + || oeq_class.is_expr_partial_const(&normal_expr) { node.data.sort_properties = SortProperties::Singleton; - } else if let Some(options) = oeq_class.get_options(&normalized_expr) { + } else if let Some(options) = oeq_class.get_options(&normal_expr) { node.data.sort_properties = SortProperties::Ordered(options); } Ok(Transformed::yes(node)) diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 73077c76492e4..7d69d669a3992 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -148,7 +148,7 @@ impl UnionEquivalentOrderingBuilder { ) { let constants = source.constants(); let properties_constants = properties.constants(); - for mut ordering in source.oeq_class.clone() { + for mut ordering in source.normalized_oeq_class() { // Progressively shorten the ordering to search for a satisfied prefix: loop { ordering = match self.try_add_ordering( From e8018bd86154915a1ac0893ea4ac8d39c3d49c7b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 11 May 2025 17:09:08 +0300 Subject: [PATCH 141/167] Fix logical conflicts --- .../enforce_distribution.rs | 15 ++++-------- datafusion/datasource/src/memory.rs | 24 +++++++------------ 2 files changed, 14 insertions(+), 25 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 6b0976e8415b3..cfd339fdf2a9a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -20,13 +20,11 @@ use std::ops::Deref; use std::sync::Arc; use crate::physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, parquet_exec_with_sort, repartition_exec, - schema, sort_exec, sort_exec_with_preserve_partitioning, sort_merge_join_exec, + check_integrity, coalesce_partitions_exec, parquet_exec_with_sort, + parquet_exec_with_stats, repartition_exec, schema, sort_exec, + sort_exec_with_preserve_partitioning, sort_merge_join_exec, sort_preserving_merge_exec, union_exec, }; -use crate::physical_optimizer::test_utils::{ - parquet_exec_with_sort, parquet_exec_with_stats, -}; use arrow::array::{RecordBatch, UInt64Array, UInt8Array}; use arrow::compute::SortOptions; @@ -3507,11 +3505,8 @@ async fn test_distribute_sort_parquet() -> Result<()> { ); let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let physical_plan = sort_exec(sort_key, parquet_exec_with_stats(10000 * 8192), false); + let sort_key = [PhysicalSortExpr::new_default(col("c", &schema)?)].into(); + let physical_plan = sort_exec(sort_key, parquet_exec_with_stats(10000 * 8192)); // prior to optimization, this is the starting plan let starting = &[ diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 23432f1e4c973..2258689cf6e13 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -771,7 +771,7 @@ impl MemorySourceConfig { target_partitions: usize, output_ordering: LexOrdering, ) -> Result>>> { - if !self.eq_properties().ordering_satisfy(&output_ordering) { + if !self.eq_properties().ordering_satisfy(output_ordering) { Ok(None) } else { let total_num_batches = @@ -1126,19 +1126,17 @@ mod memory_source_tests { #[cfg(test)] mod tests { + use super::*; use crate::test_util::col; use crate::tests::{aggr_test_schema, make_partition}; - use super::*; - use arrow::array::{ArrayRef, Int32Array, Int64Array, StringArray}; - use arrow::compute::SortOptions; - use datafusion_physical_expr::PhysicalSortExpr; - use datafusion_physical_plan::expressions::lit; - use arrow::datatypes::{DataType, Field}; use datafusion_common::assert_batches_eq; use datafusion_common::stats::{ColumnStatistics, Precision}; + use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_plan::expressions::lit; + use futures::StreamExt; #[tokio::test] @@ -1624,10 +1622,8 @@ mod tests { #[test] fn test_repartition_with_sort_information() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); + let sort_key: LexOrdering = + [PhysicalSortExpr::new_default(col("c", &schema)?)].into(); let has_sort = vec![sort_key.clone()]; let output_ordering = Some(sort_key); @@ -1674,10 +1670,8 @@ mod tests { #[test] fn test_repartition_with_batch_ordering_not_matching_sizing() -> Result<()> { let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); + let sort_key: LexOrdering = + [PhysicalSortExpr::new_default(col("c", &schema)?)].into(); let has_sort = vec![sort_key.clone()]; let output_ordering = Some(sort_key); From 5e65c355ab4e4cdc85ee91dc85f44ac864dde98a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 11 May 2025 17:12:44 +0300 Subject: [PATCH 142/167] Fix imports --- datafusion/datasource/src/memory.rs | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 2258689cf6e13..b6ac369a53f0e 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -26,16 +26,6 @@ use std::sync::Arc; use crate::sink::DataSink; use crate::source::{DataSource, DataSourceExec}; -use async_trait::async_trait; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::memory::MemoryStream; -use datafusion_physical_plan::projection::{ - all_alias_free_columns, new_projections_for_columns, ProjectionExec, -}; -use datafusion_physical_plan::{ - common, ColumnarValue, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - PhysicalExpr, PlanProperties, SendableRecordBatchStream, Statistics, -}; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{Schema, SchemaRef}; @@ -49,6 +39,17 @@ use datafusion_physical_expr::equivalence::{ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::memory::MemoryStream; +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, +}; +use datafusion_physical_plan::{ + common, ColumnarValue, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + PhysicalExpr, PlanProperties, SendableRecordBatchStream, Statistics, +}; + +use async_trait::async_trait; use futures::StreamExt; use itertools::Itertools; use tokio::sync::RwLock; @@ -1073,16 +1074,16 @@ mod memory_source_tests { use crate::memory::MemorySourceConfig; use crate::source::DataSourceExec; - use datafusion_physical_plan::ExecutionPlan; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::Result; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::PhysicalSortExpr; - use datafusion_physical_expr_common::sort_expr::LexOrdering; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_physical_plan::ExecutionPlan; #[test] - fn test_memory_order_eq() -> datafusion_common::Result<()> { + fn test_memory_order_eq() -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, false), Field::new("b", DataType::Int64, false), From 04e96f20674a6b217ecfd63cfae35aaaca43bb36 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 11 May 2025 17:33:52 +0300 Subject: [PATCH 143/167] Remove noop code --- .../src/enforce_sorting/sort_pushdown.rs | 1 - datafusion/physical-plan/src/joins/utils.rs | 37 ------------------- 2 files changed, 38 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 9ddc1ad54c6bf..0fb78993b22e6 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -444,7 +444,6 @@ fn try_pushdown_requirements_to_join( new_left_ordering, new_right_ordering, join_type, - smj.on(), smj.left().schema().fields.len(), &smj.maintains_input_order(), Some(probe_side), diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index a135cf4979ef2..391afe33245e1 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -45,7 +45,6 @@ use arrow::datatypes::{ }; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; @@ -125,28 +124,6 @@ pub fn adjust_right_output_partitioning( } } -/// Replaces the right column (first index in the `on_column` tuple) with -/// the left column (zeroth index in the tuple) inside `right_ordering`. -fn replace_on_columns_of_right_ordering( - on_columns: &[(PhysicalExprRef, PhysicalExprRef)], - right_ordering: &mut LexOrdering, -) { - for (left_col, right_col) in on_columns { - for item in right_ordering.iter_mut() { - item.expr = Arc::clone(&item.expr) - .transform(|e| { - if e.eq(right_col) { - Ok(Transformed::yes(Arc::clone(left_col))) - } else { - Ok(Transformed::no(e)) - } - }) - .data() - .expect("closure is infallible"); - } - } -} - fn offset_ordering( ordering: &LexOrdering, join_type: &JoinType, @@ -171,7 +148,6 @@ pub fn calculate_join_output_ordering( left_ordering: Option<&LexOrdering>, right_ordering: Option<&LexOrdering>, join_type: JoinType, - on_columns: &[(PhysicalExprRef, PhysicalExprRef)], left_columns_len: usize, maintains_input_order: &[bool], probe_side: Option, @@ -181,10 +157,6 @@ pub fn calculate_join_output_ordering( // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { if let Some(right_ordering) = right_ordering { - replace_on_columns_of_right_ordering( - on_columns, - &mut right_ordering.clone(), - ); let right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); return if let Some(left_ordering) = left_ordering { @@ -202,10 +174,6 @@ pub fn calculate_join_output_ordering( // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { return if let Some(right_ordering) = right_ordering { - replace_on_columns_of_right_ordering( - on_columns, - &mut right_ordering.clone(), - ); let mut right_offset = offset_ordering(right_ordering, &join_type, left_columns_len); if let Some(left_ordering) = left_ordering { @@ -2329,10 +2297,6 @@ mod tests { PhysicalSortExpr::new_default(Arc::new(Column::new("y", 1))), ]); let join_type = JoinType::Inner; - let on_columns = [( - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("x", 0)) as _, - )]; let left_columns_len = 5; let maintains_input_orders = [[true, false], [false, true]]; let probe_sides = [Some(JoinSide::Left), Some(JoinSide::Right)]; @@ -2362,7 +2326,6 @@ mod tests { left_ordering.as_ref(), right_ordering.as_ref(), join_type, - &on_columns, left_columns_len, maintains_input_order, probe_side, From 3a7d51819a76284b2a8e4f4ac28ac566a317e733 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 11 May 2025 19:57:24 +0300 Subject: [PATCH 144/167] Move add_offset_to_expr --- .../physical-expr/src/equivalence/class.rs | 3 ++- .../physical-expr/src/equivalence/mod.rs | 23 +--------------- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../src/equivalence/properties/joins.rs | 3 ++- datafusion/physical-expr/src/lib.rs | 8 +++--- datafusion/physical-expr/src/physical_expr.rs | 27 +++++++++++++++++-- datafusion/physical-plan/src/joins/utils.rs | 5 ++-- 7 files changed, 38 insertions(+), 33 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index a80b61da93781..5f8d98def4955 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -21,8 +21,9 @@ use std::sync::Arc; use std::vec::IntoIter; use super::projection::ProjectionTargets; -use super::{add_offset_to_expr, ProjectionMapping}; +use super::ProjectionMapping; use crate::expressions::{Column, Literal}; +use crate::physical_expr::add_offset_to_expr; use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 3e30e76acc34d..7b92169a9da5f 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -18,11 +18,9 @@ use std::borrow::Borrow; use std::sync::Arc; -use crate::expressions::Column; use crate::PhysicalExpr; use arrow::compute::SortOptions; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; mod class; @@ -37,25 +35,6 @@ pub use properties::{ calculate_union, join_equivalence_properties, EquivalenceProperties, }; -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -pub fn add_offset_to_expr( - expr: Arc, - offset: usize, -) -> Arc { - expr.transform_down(|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), - None => Ok(Transformed::no(e)), - }) - .data() - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. -} - // Convert each tuple to a `PhysicalSortExpr` and construct a vector. pub fn convert_to_sort_exprs>>( args: &[(T, SortOptions)], @@ -77,7 +56,7 @@ pub fn convert_to_orderings>>( #[cfg(test)] mod tests { use super::*; - use crate::expressions::col; + use crate::expressions::{col, Column}; use crate::{LexRequirement, PhysicalSortExpr}; use arrow::compute::SortOptions; diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 4037e5c63beef..d32879920adfc 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -20,7 +20,7 @@ use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; -use crate::equivalence::add_offset_to_expr; +use crate::physical_expr::add_offset_to_expr; use crate::{LexOrdering, PhysicalExpr}; use arrow::compute::SortOptions; diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index e2f1171ca316b..9ffd0f399a3a6 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -128,9 +128,10 @@ mod tests { use std::sync::Arc; use super::*; + use crate::equivalence::convert_to_orderings; use crate::equivalence::tests::create_test_schema; - use crate::equivalence::{add_offset_to_expr, convert_to_orderings}; use crate::expressions::col; + use crate::physical_expr::add_offset_to_expr; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Fields, Schema}; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index a7dcefabe869a..9dcf0bea63756 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,12 +54,12 @@ pub use equivalence::{ }; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ - create_ordering, create_physical_sort_expr, create_physical_sort_exprs, - physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, - PhysicalExprRef, + add_offset_to_expr, create_ordering, create_physical_sort_expr, + create_physical_sort_exprs, physical_exprs_bag_equal, physical_exprs_contains, + physical_exprs_equal, }; -pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +pub use datafusion_physical_expr_common::physical_expr::{PhysicalExpr, PhysicalExprRef}; pub use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 3162db0bbe3c2..48d2efa30a695 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -18,12 +18,35 @@ use std::sync::Arc; use crate::create_physical_expr; +use crate::expressions::Column; + +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{DFSchema, HashMap}; use datafusion_expr::execution_props::ExecutionProps; -pub(crate) use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -pub use datafusion_physical_expr_common::physical_expr::PhysicalExprRef; + use itertools::izip; +pub(crate) use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub fn add_offset_to_expr( + expr: Arc, + offset: usize, +) -> Arc { + expr.transform_down(|e| match e.as_any().downcast_ref::() { + Some(col) => Ok(Transformed::yes(Arc::new(Column::new( + col.name(), + offset + col.index(), + )))), + None => Ok(Transformed::no(e)), + }) + .data() + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. +} + /// This function is similar to the `contains` method of `Vec`. It finds /// whether `expr` is among `physical_exprs`. pub fn physical_exprs_contains( diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 391afe33245e1..6d9907a396b6f 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -49,10 +49,11 @@ use datafusion_common::{ plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; use datafusion_expr::interval_arithmetic::Interval; -use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{LexOrdering, PhysicalExpr, PhysicalExprRef}; +use datafusion_physical_expr::{ + add_offset_to_expr, LexOrdering, PhysicalExpr, PhysicalExprRef, +}; use crate::joins::SharedBitmapBuilder; use crate::projection::ProjectionExec; From 3e4de1eba4d5a9df4ede231193a51674b24407eb Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 12 May 2025 20:03:26 +0300 Subject: [PATCH 145/167] Remove mutation from LexOrdering --- datafusion/datasource/src/statistics.rs | 75 +++++++++--------- .../physical-expr-common/src/sort_expr.rs | 6 -- .../physical-expr/src/equivalence/class.rs | 36 +++------ .../physical-expr/src/equivalence/ordering.rs | 18 ++--- .../src/equivalence/properties/joins.rs | 8 +- datafusion/physical-expr/src/lib.rs | 6 +- datafusion/physical-expr/src/physical_expr.rs | 78 ++++++++++--------- .../src/enforce_sorting/sort_pushdown.rs | 45 +++++------ datafusion/physical-plan/src/joins/utils.rs | 67 +++++++--------- datafusion/physical-plan/src/projection.rs | 25 ++++-- datafusion/physical-plan/src/sorts/sort.rs | 14 ++-- .../src/sorts/sort_preserving_merge.rs | 14 ++-- datafusion/physical-plan/src/streaming.rs | 15 ++-- 13 files changed, 193 insertions(+), 214 deletions(-) diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 39b2d30d66ec8..db9af0ff76754 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -20,24 +20,25 @@ //! Currently, this module houses code to sort file groups if they are non-overlapping with //! respect to the required sort order. See [`MinMaxStatistics`] -use futures::{Stream, StreamExt}; use std::sync::Arc; use crate::file_groups::FileGroup; use crate::PartitionedFile; use arrow::array::RecordBatch; +use arrow::compute::SortColumn; use arrow::datatypes::SchemaRef; -use arrow::{ - compute::SortColumn, - row::{Row, Rows}, -}; +use arrow::row::{Row, Rows}; use datafusion_common::stats::Precision; -use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError, Result}; +use datafusion_common::{ + plan_datafusion_err, plan_err, DataFusionError, Result, ScalarValue, +}; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::{ColumnStatistics, Statistics}; +use futures::{Stream, StreamExt}; + /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. /// The min/max values are ordered by [`Self::sort_order`]. /// Furthermore, any columns that are reversed in the sort order have their min/max values swapped. @@ -71,9 +72,7 @@ impl MinMaxStatistics { projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) files: impl IntoIterator, ) -> Result { - use datafusion_common::ScalarValue; - - let statistics_and_partition_values = files + let Some(statistics_and_partition_values) = files .into_iter() .map(|file| { file.statistics @@ -81,9 +80,9 @@ impl MinMaxStatistics { .zip(Some(file.partition_values.as_slice())) }) .collect::>>() - .ok_or_else(|| { - DataFusionError::Plan("Parquet file missing statistics".to_string()) - })?; + else { + return plan_err!("Parquet file missing statistics"); + }; // Helper function to get min/max statistics for a given column of projected_schema let get_min_max = |i: usize| -> Result<(Vec, Vec)> { @@ -96,9 +95,7 @@ impl MinMaxStatistics { .get_value() .cloned() .zip(s.column_statistics[i].max_value.get_value().cloned()) - .ok_or_else(|| { - DataFusionError::Plan("statistics not found".to_string()) - }) + .ok_or_else(|| plan_datafusion_err!("statistics not found")) } else { let partition_value = &pv[i - s.column_statistics.len()]; Ok((partition_value.clone(), partition_value.clone())) @@ -109,8 +106,11 @@ impl MinMaxStatistics { .unzip()) }; - let sort_columns = sort_columns_from_physical_sort_exprs(projected_sort_order) - .ok_or(plan_datafusion_err!("sort expression must be on column"))?; + let Some(sort_columns) = + sort_columns_from_physical_sort_exprs(projected_sort_order) + else { + return plan_err!("sort expression must be on column"); + }; // Project the schema & sort order down to just the relevant columns let min_max_schema = Arc::new( @@ -118,12 +118,16 @@ impl MinMaxStatistics { .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, ); - let mut min_max_sort_order = projected_sort_order.clone(); - for (idx, (sort_expr, col)) in - min_max_sort_order.iter_mut().zip(&sort_columns).enumerate() - { - sort_expr.expr = Arc::new(Column::new(col.name(), idx)); - } + let min_max_sort_order = projected_sort_order + .iter() + .zip(sort_columns.iter()) + .enumerate() + .map(|(idx, (sort_expr, col))| { + let expr = Arc::new(Column::new(col.name(), idx)); + PhysicalSortExpr::new(expr, sort_expr.options) + }); + // Safe to `unwrap` as we know that sort columns are non-empty: + let min_max_sort_order = LexOrdering::new(min_max_sort_order).unwrap(); let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns .iter() @@ -183,24 +187,23 @@ impl MinMaxStatistics { .map_err(|e| e.context("create sort fields"))?; let converter = RowConverter::new(sort_fields)?; - let sort_columns = sort_columns_from_physical_sort_exprs(sort_order) - .ok_or(plan_datafusion_err!("sort expression must be on column"))?; + let Some(sort_columns) = sort_columns_from_physical_sort_exprs(sort_order) else { + return plan_err!("sort expression must be on column"); + }; // swap min/max if they're reversed in the ordering let (new_min_cols, new_max_cols): (Vec<_>, Vec<_>) = sort_order .iter() .zip(sort_columns.iter().copied()) .map(|(sort_expr, column)| { - if sort_expr.options.descending { - max_values - .column_by_name(column.name()) - .zip(min_values.column_by_name(column.name())) + let maxes = max_values.column_by_name(column.name()); + let mins = min_values.column_by_name(column.name()); + let opt_value = if sort_expr.options.descending { + maxes.zip(mins) } else { - min_values - .column_by_name(column.name()) - .zip(max_values.column_by_name(column.name())) - } - .ok_or_else(|| { + mins.zip(maxes) + }; + opt_value.ok_or_else(|| { plan_datafusion_err!( "missing column in MinMaxStatistics::new: '{}'", column.name() @@ -278,7 +281,7 @@ fn sort_columns_from_physical_sort_exprs( sort_order .iter() .map(|expr| expr.expr.as_any().downcast_ref::()) - .collect::>>() + .collect() } /// Get all files as well as the file level summary statistics (no statistic for partition columns). diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index ce51dea0811e8..f089a35ee2ffe 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -415,12 +415,6 @@ impl Deref for LexOrdering { } } -impl DerefMut for LexOrdering { - fn deref_mut(&mut self) -> &mut Self::Target { - self.exprs.as_mut_slice() - } -} - impl Display for LexOrdering { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut first = true; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 5f8d98def4955..be7b7c8ba0d3c 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -231,27 +231,12 @@ impl EquivalenceClass { } } - /// Returns true if this equivalence class contains the given expression. - pub fn contains(&self, expr: &Arc) -> bool { - self.exprs.contains(expr) - } - - /// Returns true if this equivalence class has any entries in common with + /// Returns whether this equivalence class has any entries in common with /// `other`. pub fn contains_any(&self, other: &Self) -> bool { self.exprs.intersection(&other.exprs).next().is_some() } - /// Returns the number of items in this equivalence class. - pub fn len(&self) -> usize { - self.exprs.len() - } - - /// Returns whether this equivalence class is empty. - pub fn is_empty(&self) -> bool { - self.exprs.is_empty() - } - /// Returns whether this equivalence class is trivial, meaning that it is /// either empty, or contains a single expression that is not a constant. /// Such classes are not useful, and can be removed from equivalence groups. @@ -259,23 +244,26 @@ impl EquivalenceClass { self.exprs.is_empty() || (self.exprs.len() == 1 && self.constant.is_none()) } - /// Iterate over all elements in this class (in some arbitrary order). - pub fn iter(&self) -> impl Iterator> { - self.exprs.iter() - } - /// Return a new equivalence class that have the specified offset added to /// each expression (used when schemas are appended such as in joins) - pub fn with_offset(&self, offset: usize) -> Self { + pub fn with_offset(&self, offset: isize) -> Self { let new_exprs = self .exprs .iter() .cloned() - .map(|e| add_offset_to_expr(e, offset)); + .map(|e| add_offset_to_expr(e, offset).unwrap()); Self::new(new_exprs) } } +impl Deref for EquivalenceClass { + type Target = IndexSet>; + + fn deref(&self) -> &Self::Target { + &self.exprs + } +} + impl IntoIterator for EquivalenceClass { type Item = Arc; type IntoIter = as IntoIterator>::IntoIter; @@ -787,7 +775,7 @@ impl EquivalenceGroup { self.iter().cloned().chain( right_equivalences .iter() - .map(|cls| cls.with_offset(left_size)), + .map(|cls| cls.with_offset(left_size as _)), ), ); // In we have an inner join, expressions in the "on" condition diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d32879920adfc..baf26904005fc 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -20,11 +20,10 @@ use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; -use crate::physical_expr::add_offset_to_expr; -use crate::{LexOrdering, PhysicalExpr}; +use crate::{add_offset_to_ordering, LexOrdering, PhysicalExpr}; use arrow::compute::SortOptions; -use datafusion_common::HashSet; +use datafusion_common::{HashSet, Result}; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// An `OrderingEquivalenceClass` keeps track of distinct alternative orderings @@ -169,12 +168,13 @@ impl OrderingEquivalenceClass { /// Adds `offset` value to the index of each expression inside this /// ordering equivalence class. - pub fn add_offset(&mut self, offset: usize) { - for ordering in self.orderings.iter_mut() { - for sort_expr in ordering.iter_mut() { - sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); - } - } + pub fn add_offset(&mut self, offset: isize) { + let orderings = std::mem::take(&mut self.orderings); + self.orderings = orderings + .into_iter() + .map(|ordering| add_offset_to_ordering(ordering, offset)) + .collect::>() + .unwrap(); } /// Gets sort options associated with this expression if it is a leading diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 9ffd0f399a3a6..eb5978e387b74 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -119,7 +119,7 @@ pub fn updated_right_ordering_equivalence_class( join_type, JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right ) { - right_oeq_class.add_offset(left_size); + right_oeq_class.add_offset(left_size as _); } } @@ -143,9 +143,9 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let col_c = &col("c", &schema)?; - let offset = schema.fields.len(); - let col_a2 = &add_offset_to_expr(Arc::clone(col_a), offset); - let col_b2 = &add_offset_to_expr(Arc::clone(col_b), offset); + let offset = schema.fields.len() as _; + let col_a2 = &add_offset_to_expr(Arc::clone(col_a), offset)?; + let col_b2 = &add_offset_to_expr(Arc::clone(col_b), offset)?; let option_asc = SortOptions { descending: false, nulls_first: false, diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 9dcf0bea63756..e3cb01bd1166d 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,9 +54,9 @@ pub use equivalence::{ }; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ - add_offset_to_expr, create_ordering, create_physical_sort_expr, - create_physical_sort_exprs, physical_exprs_bag_equal, physical_exprs_contains, - physical_exprs_equal, + add_offset_to_expr, add_offset_to_ordering, create_ordering, + create_physical_sort_expr, create_physical_sort_exprs, physical_exprs_bag_equal, + physical_exprs_contains, physical_exprs_equal, }; pub use datafusion_physical_expr_common::physical_expr::{PhysicalExpr, PhysicalExprRef}; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 48d2efa30a695..b93b05cf979df 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -17,34 +17,38 @@ use std::sync::Arc; -use crate::create_physical_expr; -use crate::expressions::Column; +use crate::expressions::{self, Column}; +use crate::{create_physical_expr, LexOrdering, PhysicalSortExpr}; +use arrow::compute::SortOptions; +use arrow::datatypes::Schema; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::{plan_err, Result}; use datafusion_common::{DFSchema, HashMap}; use datafusion_expr::execution_props::ExecutionProps; +use datafusion_expr::{Expr, SortExpr}; use itertools::izip; +// Exports: pub(crate) use datafusion_physical_expr_common::physical_expr::PhysicalExpr; /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. pub fn add_offset_to_expr( expr: Arc, - offset: usize, -) -> Arc { + offset: isize, +) -> Result> { expr.transform_down(|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), + Some(col) => { + let Some(idx) = col.index().checked_add_signed(offset) else { + return plan_err!("Column index overflow"); + }; + Ok(Transformed::yes(Arc::new(Column::new(col.name(), idx)))) + } None => Ok(Transformed::no(e)), }) .data() - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. } /// This function is similar to the `contains` method of `Vec`. It finds @@ -83,26 +87,21 @@ pub fn physical_exprs_bag_equal( multi_set_lhs == multi_set_rhs } -use crate::{expressions, LexOrdering, PhysicalSortExpr}; -use arrow::compute::SortOptions; -use arrow::datatypes::Schema; -use datafusion_common::plan_err; -use datafusion_common::Result; -use datafusion_expr::{Expr, SortExpr}; - -/// Converts logical sort expressions to physical sort expressions +/// Converts logical sort expressions to physical sort expressions. /// -/// This function transforms a collection of logical sort expressions into their physical -/// representation that can be used during query execution. +/// This function transforms a collection of logical sort expressions into their +/// physical representation that can be used during query execution. /// /// # Arguments /// -/// * `schema` - The schema containing column definitions -/// * `sort_order` - A collection of logical sort expressions grouped into lexicographic orderings +/// * `schema` - The schema containing column definitions. +/// * `sort_order` - A collection of logical sort expressions grouped into +/// lexicographic orderings. /// /// # Returns /// -/// A vector of lexicographic orderings for physical execution, or an error if the transformation fails +/// A vector of lexicographic orderings for physical execution, or an error if +/// the transformation fails. /// /// # Examples /// @@ -142,13 +141,8 @@ pub fn create_ordering( match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { Ok(expr) => { - sort_exprs.push(PhysicalSortExpr { - expr, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); + let opts = SortOptions::new(!sort.asc, sort.nulls_first); + sort_exprs.push(PhysicalSortExpr::new(expr, opts)); } // Cannot find expression in the projected_schema, stop iterating // since rest of the orderings are violated @@ -169,6 +163,21 @@ pub fn create_ordering( Ok(all_sort_orders) } +pub fn add_offset_to_ordering( + ordering: LexOrdering, + offset: isize, +) -> Result { + let updated_exprs = ordering + .into_iter() + .map(|mut sort_expr| { + sort_expr.expr = add_offset_to_expr(sort_expr.expr, offset)?; + Ok(sort_expr) + }) + .collect::>>()?; + // Can safely unwrap since adding an offset is a one-to-one operation: + Ok(LexOrdering::new(updated_exprs).unwrap()) +} + /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( e: &SortExpr, @@ -176,11 +185,8 @@ pub fn create_physical_sort_expr( execution_props: &ExecutionProps, ) -> Result { create_physical_expr(&e.expr, input_dfschema, execution_props).map(|expr| { - let options = SortOptions { - descending: !e.asc, - nulls_first: e.nulls_first, - }; - PhysicalSortExpr { expr, options } + let options = SortOptions::new(!e.asc, e.nulls_first); + PhysicalSortExpr::new(expr, options) }) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 0fb78993b22e6..67b0d3e1d87a1 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -24,13 +24,14 @@ use crate::utils::{ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, Result}; +use datafusion_common::{internal_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::{add_offset_to_ordering, EquivalenceProperties}; use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortRequirement, + LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, + PhysicalSortRequirement, }; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{ @@ -281,19 +282,27 @@ fn pushdown_requirement_to_children( Ok(Some(vec![Some(parent_required); plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { let left_columns_len = smj.left().schema().fields().len(); - let parent_ordering = LexOrdering::from(parent_required.first().clone()); + let parent_ordering: Vec = parent_required + .first() + .iter() + .cloned() + .map(Into::into) + .collect(); let eqp = smj.properties().equivalence_properties(); match expr_source_side(eqp, parent_ordering, smj.join_type(), left_columns_len) { Some((JoinSide::Left, ordering)) => try_pushdown_requirements_to_join( smj, parent_required.into_single(), - ordering, + LexOrdering::new(ordering).unwrap(), JoinSide::Left, ), - Some((JoinSide::Right, mut ordering)) => { + Some((JoinSide::Right, ordering)) => { let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); - ordering = shift_right_ordering(ordering, right_offset)?; + let ordering = add_offset_to_ordering( + LexOrdering::new(ordering).unwrap(), + -(right_offset as isize), + )?; try_pushdown_requirements_to_join( smj, parent_required.into_single(), @@ -447,7 +456,7 @@ fn try_pushdown_requirements_to_join( smj.left().schema().fields.len(), &smj.maintains_input_order(), Some(probe_side), - ); + )?; let mut smj_eqs = smj.properties().equivalence_properties().clone(); if let Some(new_output_ordering) = new_output_ordering { // smj will have this ordering when its input changes. @@ -472,10 +481,10 @@ fn try_pushdown_requirements_to_join( fn expr_source_side( eqp: &EquivalenceProperties, - mut ordering: LexOrdering, + mut ordering: Vec, join_type: JoinType, left_columns_len: usize, -) -> Option<(JoinSide, LexOrdering)> { +) -> Option<(JoinSide, Vec)> { // TODO: Handle the case where a prefix of the ordering comes from the left // and a suffix from the right. match join_type { @@ -545,22 +554,6 @@ fn expr_source_side( } } -fn shift_right_ordering( - mut parent_ordering: LexOrdering, - left_columns_len: usize, -) -> Result { - for req in parent_ordering.iter_mut() { - let Some(col) = req.expr.as_any().downcast_ref::() else { - return plan_err!( - "Expect to shift all the parent required column indexes for SortMergeJoin" - ); - }; - let offset = col.index() - left_columns_len; - req.expr = Arc::new(Column::new(col.name(), offset)); - } - Ok(parent_ordering) -} - /// Handles the custom pushdown of parent-required sorting requirements down to /// the child execution plans, considering whether the input order is maintained. /// diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 6d9907a396b6f..bab2e124ea1b4 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -25,7 +25,9 @@ use std::ops::Range; use std::sync::Arc; use std::task::{Context, Poll}; +use crate::joins::SharedBitmapBuilder; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; +use crate::projection::ProjectionExec; use crate::{ ColumnStatistics, ExecutionPlan, ExecutionPlanProperties, Partitioning, Statistics, }; @@ -52,11 +54,10 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - add_offset_to_expr, LexOrdering, PhysicalExpr, PhysicalExprRef, + add_offset_to_expr, add_offset_to_ordering, LexOrdering, PhysicalExpr, + PhysicalExprRef, }; -use crate::joins::SharedBitmapBuilder; -use crate::projection::ProjectionExec; use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; use parking_lot::Mutex; @@ -117,33 +118,15 @@ pub fn adjust_right_output_partitioning( Partitioning::Hash(exprs, size) => { let new_exprs = exprs .iter() - .map(|expr| add_offset_to_expr(Arc::clone(expr), left_columns_len)) - .collect(); + .map(|expr| add_offset_to_expr(Arc::clone(expr), left_columns_len as _)) + .collect::>() + .unwrap(); Partitioning::Hash(new_exprs, *size) } result => result.clone(), } } -fn offset_ordering( - ordering: &LexOrdering, - join_type: &JoinType, - offset: usize, -) -> LexOrdering { - match join_type { - // In the case below, right ordering should be offsetted with the left - // side length, since we append the right table to the left table. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let mut ordering = ordering.clone(); - for sort_expr in ordering.iter_mut() { - sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); - } - ordering - } - _ => ordering.clone(), - } -} - /// Calculate the output ordering of a given join operation. pub fn calculate_join_output_ordering( left_ordering: Option<&LexOrdering>, @@ -152,43 +135,53 @@ pub fn calculate_join_output_ordering( left_columns_len: usize, maintains_input_order: &[bool], probe_side: Option, -) -> Option { +) -> Result> { match maintains_input_order { [true, false] => { // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { - if let Some(right_ordering) = right_ordering { + if let Some(right_ordering) = right_ordering.cloned() { let right_offset = - offset_ordering(right_ordering, &join_type, left_columns_len); + add_offset_to_ordering(right_ordering, left_columns_len as _)?; return if let Some(left_ordering) = left_ordering { let mut result = left_ordering.clone(); result.extend(right_offset); - Some(result) + Ok(Some(result)) } else { - Some(right_offset) + Ok(Some(right_offset)) }; } } - left_ordering.cloned() + Ok(left_ordering.cloned()) } [false, true] => { // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { - return if let Some(right_ordering) = right_ordering { + return if let Some(right_ordering) = right_ordering.cloned() { let mut right_offset = - offset_ordering(right_ordering, &join_type, left_columns_len); + add_offset_to_ordering(right_ordering, left_columns_len as _)?; if let Some(left_ordering) = left_ordering { right_offset.extend(left_ordering.clone()); } - Some(right_offset) + Ok(Some(right_offset)) } else { - left_ordering.cloned() + Ok(left_ordering.cloned()) }; } - right_ordering.map(|o| offset_ordering(o, &join_type, left_columns_len)) + right_ordering + .map(|o| match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Full + | JoinType::Right => { + add_offset_to_ordering(o.clone(), left_columns_len as _) + } + _ => Ok(o.clone()), + }) + .transpose() } // Doesn't maintain ordering, output ordering is None. - [false, false] => None, + [false, false] => Ok(None), [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), } @@ -2330,7 +2323,7 @@ mod tests { left_columns_len, maintains_input_order, probe_side, - ), + )?, expected[i] ); } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 95d69a744f131..2fe9ae10cfc73 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -46,11 +46,10 @@ use datafusion_common::{internal_err, JoinSide, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; -use itertools::Itertools; use log::trace; /// Execution plan for a projection @@ -664,6 +663,22 @@ pub fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } +pub fn update_ordering( + ordering: LexOrdering, + projected_exprs: &[(Arc, String)], +) -> Result> { + let mut updated_exprs = vec![]; + for mut sort_expr in ordering.into_iter() { + let Some(updated_expr) = update_expr(&sort_expr.expr, projected_exprs, false)? + else { + return Ok(None); + }; + sort_expr.expr = updated_expr; + updated_exprs.push(sort_expr); + } + Ok(LexOrdering::new(updated_exprs)) +} + /// Downcasts all the expressions in `exprs` to `Column`s. If any of the given /// expressions is not a `Column`, returns `None`. pub fn physical_to_column_exprs( @@ -698,7 +713,7 @@ pub fn new_join_children( alias.clone(), ) }) - .collect_vec(), + .collect(), Arc::clone(left_child), )?; let left_size = left_child.schema().fields().len() as i32; @@ -716,7 +731,7 @@ pub fn new_join_children( alias.clone(), ) }) - .collect_vec(), + .collect(), Arc::clone(right_child), )?; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 8d34341ec1075..5d6ebf8e46561 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -31,7 +31,7 @@ use crate::limit::LimitStream; use crate::metrics::{ BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, SpillMetrics, }; -use crate::projection::{make_with_child, update_expr, ProjectionExec}; +use crate::projection::{make_with_child, update_ordering, ProjectionExec}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::get_record_batch_memory_size; use crate::spill::in_progress_spill_file::InProgressSpillFile; @@ -1344,14 +1344,10 @@ impl ExecutionPlan for SortExec { return Ok(None); } - let mut updated_exprs = self.expr().clone(); - for sort in updated_exprs.iter_mut() { - let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? - else { - return Ok(None); - }; - sort.expr = updated_expr; - } + let Some(updated_exprs) = update_ordering(self.expr.clone(), projection.expr())? + else { + return Ok(None); + }; Ok(Some(Arc::new( SortExec::new(updated_exprs, make_with_child(projection, self.input())?) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 700b0bea0fa52..1528a5419d3de 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use crate::projection::{make_with_child, update_expr, ProjectionExec}; +use crate::projection::{make_with_child, update_ordering, ProjectionExec}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, @@ -365,14 +365,10 @@ impl ExecutionPlan for SortPreservingMergeExec { return Ok(None); } - let mut updated_exprs = self.expr().clone(); - for sort in updated_exprs.iter_mut() { - let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? - else { - return Ok(None); - }; - sort.expr = updated_expr; - } + let Some(updated_exprs) = update_ordering(self.expr.clone(), projection.expr())? + else { + return Ok(None); + }; Ok(Some(Arc::new( SortPreservingMergeExec::new( diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 94ff2cecc0b09..3f2e99b1d0038 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -27,7 +27,7 @@ use crate::execution_plan::{Boundedness, EmissionType}; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{ - all_alias_free_columns, new_projections_for_columns, update_expr, ProjectionExec, + all_alias_free_columns, new_projections_for_columns, update_ordering, ProjectionExec, }; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; @@ -306,15 +306,10 @@ impl ExecutionPlan for StreamingTableExec { ); let mut lex_orderings = vec![]; - for mut ordering in self.projected_output_ordering().into_iter() { - for sort_expr in ordering.iter_mut() { - let Some(new_sort_expr) = - update_expr(&sort_expr.expr, projection.expr(), false)? - else { - return Ok(None); - }; - sort_expr.expr = new_sort_expr; - } + for ordering in self.projected_output_ordering().into_iter() { + let Some(ordering) = update_ordering(ordering, projection.expr())? else { + return Ok(None); + }; lex_orderings.push(ordering); } From 2dbb8ceaa6b9be16855f68bce56e58d13705ebe5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 12 May 2025 21:01:32 +0300 Subject: [PATCH 146/167] Remove unwraps --- .../physical-expr/src/equivalence/class.rs | 84 +++++++++---------- .../src/equivalence/properties/joins.rs | 14 ++-- .../src/equivalence/properties/mod.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 4 +- .../physical-plan/src/joins/hash_join.rs | 2 +- .../src/joins/nested_loop_join.rs | 2 +- .../src/joins/sort_merge_join.rs | 10 +-- .../src/joins/symmetric_hash_join.rs | 11 ++- 8 files changed, 63 insertions(+), 66 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index be7b7c8ba0d3c..8886aa8fadb3b 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -22,12 +22,12 @@ use std::vec::IntoIter; use super::projection::ProjectionTargets; use super::ProjectionMapping; -use crate::expressions::{Column, Literal}; +use crate::expressions::Literal; use crate::physical_expr::add_offset_to_expr; use crate::{PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{HashMap, JoinType, ScalarValue}; +use datafusion_common::{HashMap, JoinType, Result, ScalarValue}; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; @@ -244,15 +244,19 @@ impl EquivalenceClass { self.exprs.is_empty() || (self.exprs.len() == 1 && self.constant.is_none()) } - /// Return a new equivalence class that have the specified offset added to - /// each expression (used when schemas are appended such as in joins) - pub fn with_offset(&self, offset: isize) -> Self { - let new_exprs = self + /// Adds the given offset to all columns in the expressions inside this + /// class. This is used when schemas are appended, e.g. in joins. + pub fn with_offset(&self, offset: isize) -> Result { + let mut cls = Self::default(); + for expr_result in self .exprs .iter() .cloned() - .map(|e| add_offset_to_expr(e, offset).unwrap()); - Self::new(new_exprs) + .map(|e| add_offset_to_expr(e, offset)) + { + cls.push(expr_result?); + } + Ok(cls) } } @@ -308,19 +312,7 @@ pub struct EquivalenceGroup { impl EquivalenceGroup { /// Creates an equivalence group from the given equivalence classes. pub fn new(classes: impl IntoIterator) -> Self { - let classes = classes.into_iter().collect::>(); - let mut result = Self { - map: classes - .iter() - .enumerate() - .flat_map(|(idx, cls)| { - cls.iter().map(move |expr| (Arc::clone(expr), idx)) - }) - .collect(), - classes, - }; - result.remove_redundant_entries(); - result + classes.into_iter().collect::>().into() } /// Adds `expr` as a constant expression to this equivalence group. @@ -768,14 +760,15 @@ impl EquivalenceGroup { join_type: &JoinType, left_size: usize, on: &[(PhysicalExprRef, PhysicalExprRef)], - ) -> Self { - match join_type { + ) -> Result { + let group = match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { let mut result = Self::new( self.iter().cloned().chain( right_equivalences .iter() - .map(|cls| cls.with_offset(left_size as _)), + .map(|cls| cls.with_offset(left_size as _)) + .collect::>>()?, ), ); // In we have an inner join, expressions in the "on" condition @@ -784,22 +777,8 @@ impl EquivalenceGroup { for (lhs, rhs) in on.iter() { let new_lhs = Arc::clone(lhs); // Rewrite rhs to point to the right side of the join: - let new_rhs = Arc::clone(rhs) - .transform(|expr| { - if let Some(column) = - expr.as_any().downcast_ref::() - { - let new_column = Arc::new(Column::new( - column.name(), - column.index() + left_size, - )); - return Ok(Transformed::yes(new_column as _)); - } - - Ok(Transformed::no(expr)) - }) - .data() - .unwrap(); + let new_rhs = + add_offset_to_expr(Arc::clone(rhs), left_size as _)?; result.add_equal_conditions(new_lhs, new_rhs); } } @@ -807,7 +786,8 @@ impl EquivalenceGroup { } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => self.clone(), JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), - } + }; + Ok(group) } /// Checks if two expressions are equal directly or through equivalence @@ -896,11 +876,28 @@ impl Display for EquivalenceGroup { } } +impl From> for EquivalenceGroup { + fn from(classes: Vec) -> Self { + let mut result = Self { + map: classes + .iter() + .enumerate() + .flat_map(|(idx, cls)| { + cls.iter().map(move |expr| (Arc::clone(expr), idx)) + }) + .collect(), + classes, + }; + result.remove_redundant_entries(); + result + } +} + #[cfg(test)] mod tests { use super::*; use crate::equivalence::tests::create_test_params; - use crate::expressions::{binary, col, lit, BinaryExpr, Literal}; + use crate::expressions::{binary, col, lit, BinaryExpr, Column, Literal}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{Result, ScalarValue}; @@ -944,8 +941,7 @@ mod tests { }) .map(EquivalenceClass::new) .collect::>(); - let mut eq_groups = EquivalenceGroup::new(entries.clone()); - eq_groups.bridge_classes(); + let eq_groups: EquivalenceGroup = entries.clone().into(); let eq_groups = eq_groups.classes; let err_msg = format!( "error in test entries: {:?}, expected: {:?}, actual:{:?}", diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index eb5978e387b74..57ef156236436 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +use super::EquivalenceProperties; use crate::{equivalence::OrderingEquivalenceClass, PhysicalExprRef}; -use arrow::datatypes::SchemaRef; -use datafusion_common::{JoinSide, JoinType}; -use super::EquivalenceProperties; +use arrow::datatypes::SchemaRef; +use datafusion_common::{JoinSide, JoinType, Result}; /// Calculate ordering equivalence properties for the given join operation. pub fn join_equivalence_properties( @@ -30,7 +30,7 @@ pub fn join_equivalence_properties( maintains_input_order: &[bool], probe_side: Option, on: &[(PhysicalExprRef, PhysicalExprRef)], -) -> EquivalenceProperties { +) -> Result { let left_size = left.schema.fields.len(); let mut result = EquivalenceProperties::new(join_schema); result.add_equivalence_group(left.eq_group().join( @@ -38,7 +38,7 @@ pub fn join_equivalence_properties( join_type, left_size, on, - )); + )?); let EquivalenceProperties { oeq_class: left_oeq_class, @@ -100,7 +100,7 @@ pub fn join_equivalence_properties( [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), } - result + Ok(result) } /// In the context of a join, update the right side `OrderingEquivalenceClass` @@ -204,7 +204,7 @@ mod tests { &[true, false], Some(JoinSide::Left), &[], - ); + )?; let err_msg = format!("expected: {:?}, actual:{:?}", expected, &join_eq.oeq_class); assert_eq!(join_eq.oeq_class.len(), expected.len(), "{}", err_msg); diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 7ed8ca26945e1..690d08c73f08c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1225,7 +1225,7 @@ impl EquivalenceProperties { // Update the schema, the equivalence group and the ordering equivalence // class: self.schema = schema; - self.eq_group = EquivalenceGroup::new(eq_classes); + self.eq_group = eq_classes.into(); self.oeq_class = new_orderings.into(); Ok(self) } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index cbc38f1149725..93b9123ae6139 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -154,7 +154,9 @@ impl CrossJoinExec { &[false, false], None, &[], - ); + ) + // This `unwrap` is safe as we only add a positive offset to columns. + .unwrap(); // Get output partitioning: // TODO: Optimize the cross join implementation to generate M * N diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3a2c193101764..7614b12e54769 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -532,7 +532,7 @@ impl HashJoinExec { &Self::maintains_input_order(join_type), Some(Self::probe_side()), on, - ); + )?; let mut output_partitioning = match mode { PartitionMode::CollectLeft => { diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 566f06e9e7c9f..214873f49f907 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -259,7 +259,7 @@ impl NestedLoopJoinExec { None, // No on columns in nested loop join &[], - ); + )?; let mut output_partitioning = asymmetric_join_output_partitioning(left, right, &join_type); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index fd75f7b180592..8a1a54a6e086c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -207,7 +207,7 @@ impl SortMergeJoinExec { let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); let cache = - Self::compute_properties(&left, &right, Arc::clone(&schema), join_type, &on); + Self::compute_properties(&left, &right, Arc::clone(&schema), join_type, &on)?; Ok(Self { left, right, @@ -299,7 +299,7 @@ impl SortMergeJoinExec { schema: SchemaRef, join_type: JoinType, join_on: JoinOnRef, - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -309,17 +309,17 @@ impl SortMergeJoinExec { &Self::maintains_input_order(join_type), Some(Self::probe_side(&join_type)), join_on, - ); + )?; let output_partitioning = symmetric_join_output_partitioning(left, right, &join_type); - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, EmissionType::Incremental, boundedness_from_children([left, right]), - ) + )) } pub fn swap_inputs(&self) -> Result> { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index bfcf6c9237e6e..d1efe367c981b 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -236,8 +236,7 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); let schema = Arc::new(schema); - let cache = - Self::compute_properties(&left, &right, Arc::clone(&schema), *join_type, &on); + let cache = Self::compute_properties(&left, &right, schema, *join_type, &on)?; Ok(SymmetricHashJoinExec { left, right, @@ -262,7 +261,7 @@ impl SymmetricHashJoinExec { schema: SchemaRef, join_type: JoinType, join_on: JoinOnRef, - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -273,17 +272,17 @@ impl SymmetricHashJoinExec { // Has alternating probe side None, join_on, - ); + )?; let output_partitioning = symmetric_join_output_partitioning(left, right, &join_type); - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, emission_type_from_children([left, right]), boundedness_from_children([left, right]), - ) + )) } /// left stream From 32efc6251cced5edba6c23b635d6363931b642ee Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 12 May 2025 21:44:46 +0300 Subject: [PATCH 147/167] Remove unwraps - 2 --- datafusion/physical-expr/src/equivalence/ordering.rs | 6 +++--- .../physical-expr/src/equivalence/properties/joins.rs | 11 ++++++----- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index baf26904005fc..63d36f2b451f7 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -168,13 +168,13 @@ impl OrderingEquivalenceClass { /// Adds `offset` value to the index of each expression inside this /// ordering equivalence class. - pub fn add_offset(&mut self, offset: isize) { + pub fn add_offset(&mut self, offset: isize) -> Result<()> { let orderings = std::mem::take(&mut self.orderings); self.orderings = orderings .into_iter() .map(|ordering| add_offset_to_ordering(ordering, offset)) - .collect::>() - .unwrap(); + .collect::>()?; + Ok(()) } /// Gets sort options associated with this expression if it is a leading diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index 57ef156236436..b2dabe8ac77c0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -57,7 +57,7 @@ pub fn join_equivalence_properties( &mut right_oeq_class, join_type, left_size, - ); + )?; // Right side ordering equivalence properties should be prepended // with those of the left side while constructing output ordering @@ -78,7 +78,7 @@ pub fn join_equivalence_properties( &mut right_oeq_class, join_type, left_size, - ); + )?; // In this special case, left side ordering can be prefixed with // the right side ordering. if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { @@ -114,13 +114,14 @@ pub fn updated_right_ordering_equivalence_class( right_oeq_class: &mut OrderingEquivalenceClass, join_type: &JoinType, left_size: usize, -) { +) -> Result<()> { if matches!( join_type, JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right ) { - right_oeq_class.add_offset(left_size as _); + right_oeq_class.add_offset(left_size as _)?; } + Ok(()) } #[cfg(test)] @@ -271,7 +272,7 @@ mod tests { &mut right_oeq_class, &join_type, left_columns_len, - ); + )?; join_eq_properties.add_orderings(right_oeq_class); let result = join_eq_properties.oeq_class().clone(); From 66205067ccc4d6125daa157908e5e48a55a6b9f6 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 12 May 2025 22:49:10 +0300 Subject: [PATCH 148/167] Remove unwraps - 3 --- .../physical-plan/src/joins/cross_join.rs | 3 +- .../physical-plan/src/joins/hash_join.rs | 4 +-- .../src/joins/nested_loop_join.rs | 2 +- .../src/joins/sort_merge_join.rs | 2 +- .../src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/joins/utils.rs | 28 ++++++++++--------- 6 files changed, 22 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 93b9123ae6139..fde9b0f3419a9 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -164,7 +164,8 @@ impl CrossJoinExec { let output_partitioning = adjust_right_output_partitioning( right.output_partitioning(), left.schema().fields.len(), - ); + ) + .unwrap(); PlanProperties::new( eq_properties, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7614b12e54769..b96dbf2d6644b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -536,13 +536,13 @@ impl HashJoinExec { let mut output_partitioning = match mode { PartitionMode::CollectLeft => { - asymmetric_join_output_partitioning(left, right, &join_type) + asymmetric_join_output_partitioning(left, right, &join_type)? } PartitionMode::Auto => Partitioning::UnknownPartitioning( right.output_partitioning().partition_count(), ), PartitionMode::Partitioned => { - symmetric_join_output_partitioning(left, right, &join_type) + symmetric_join_output_partitioning(left, right, &join_type)? } }; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 214873f49f907..118a6e8bc438a 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -262,7 +262,7 @@ impl NestedLoopJoinExec { )?; let mut output_partitioning = - asymmetric_join_output_partitioning(left, right, &join_type); + asymmetric_join_output_partitioning(left, right, &join_type)?; let emission_type = if left.boundedness().is_unbounded() { EmissionType::Final diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 8a1a54a6e086c..dff05f3c907e7 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -312,7 +312,7 @@ impl SortMergeJoinExec { )?; let output_partitioning = - symmetric_join_output_partitioning(left, right, &join_type); + symmetric_join_output_partitioning(left, right, &join_type)?; Ok(PlanProperties::new( eq_properties, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index d1efe367c981b..240932ae8fa09 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -275,7 +275,7 @@ impl SymmetricHashJoinExec { )?; let output_partitioning = - symmetric_join_output_partitioning(left, right, &join_type); + symmetric_join_output_partitioning(left, right, &join_type)?; Ok(PlanProperties::new( eq_properties, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index bab2e124ea1b4..957240ca4d7d3 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -113,18 +113,18 @@ fn check_join_set_is_valid( pub fn adjust_right_output_partitioning( right_partitioning: &Partitioning, left_columns_len: usize, -) -> Partitioning { - match right_partitioning { +) -> Result { + let result = match right_partitioning { Partitioning::Hash(exprs, size) => { let new_exprs = exprs .iter() .map(|expr| add_offset_to_expr(Arc::clone(expr), left_columns_len as _)) - .collect::>() - .unwrap(); + .collect::>()?; Partitioning::Hash(new_exprs, *size) } result => result.clone(), - } + }; + Ok(result) } /// Calculate the output ordering of a given join operation. @@ -1257,35 +1257,36 @@ pub(crate) fn symmetric_join_output_partitioning( left: &Arc, right: &Arc, join_type: &JoinType, -) -> Partitioning { +) -> Result { let left_columns_len = left.schema().fields.len(); let left_partitioning = left.output_partitioning(); let right_partitioning = right.output_partitioning(); - match join_type { + let result = match join_type { JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { left_partitioning.clone() } JoinType::RightSemi | JoinType::RightAnti => right_partitioning.clone(), JoinType::Inner | JoinType::Right => { - adjust_right_output_partitioning(right_partitioning, left_columns_len) + adjust_right_output_partitioning(right_partitioning, left_columns_len)? } JoinType::Full => { // We could also use left partition count as they are necessarily equal. Partitioning::UnknownPartitioning(right_partitioning.partition_count()) } - } + }; + Ok(result) } pub(crate) fn asymmetric_join_output_partitioning( left: &Arc, right: &Arc, join_type: &JoinType, -) -> Partitioning { - match join_type { +) -> Result { + let result = match join_type { JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( right.output_partitioning(), left.schema().fields().len(), - ), + )?, JoinType::RightSemi | JoinType::RightAnti => right.output_partitioning().clone(), JoinType::Left | JoinType::LeftSemi @@ -1294,7 +1295,8 @@ pub(crate) fn asymmetric_join_output_partitioning( | JoinType::LeftMark => Partitioning::UnknownPartitioning( right.output_partitioning().partition_count(), ), - } + }; + Ok(result) } /// Trait for incrementally generating Join output. From 0b61beaf431a0aaaf062629e9869c483b6221005 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 00:47:57 +0300 Subject: [PATCH 149/167] Remove unwraps - 4 --- .../physical-expr/src/equivalence/ordering.rs | 10 +++-- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/physical_expr.rs | 28 ++++++------- .../src/enforce_sorting/sort_pushdown.rs | 28 +++++++------ datafusion/physical-plan/src/joins/utils.rs | 42 +++++++++++-------- 5 files changed, 59 insertions(+), 51 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 63d36f2b451f7..4974f46c796ff 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -20,7 +20,7 @@ use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; -use crate::{add_offset_to_ordering, LexOrdering, PhysicalExpr}; +use crate::{add_offset_to_physical_sort_exprs, LexOrdering, PhysicalExpr}; use arrow::compute::SortOptions; use datafusion_common::{HashSet, Result}; @@ -170,10 +170,12 @@ impl OrderingEquivalenceClass { /// ordering equivalence class. pub fn add_offset(&mut self, offset: isize) -> Result<()> { let orderings = std::mem::take(&mut self.orderings); - self.orderings = orderings + for ordering_result in orderings .into_iter() - .map(|ordering| add_offset_to_ordering(ordering, offset)) - .collect::>()?; + .map(|o| add_offset_to_physical_sort_exprs(o, offset)) + { + self.orderings.extend(LexOrdering::new(ordering_result?)); + } Ok(()) } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e3cb01bd1166d..6741f94c95458 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,7 +54,7 @@ pub use equivalence::{ }; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ - add_offset_to_expr, add_offset_to_ordering, create_ordering, + add_offset_to_expr, add_offset_to_physical_sort_exprs, create_ordering, create_physical_sort_expr, create_physical_sort_exprs, physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, }; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index b93b05cf979df..80dd8ce069b79 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -163,21 +163,6 @@ pub fn create_ordering( Ok(all_sort_orders) } -pub fn add_offset_to_ordering( - ordering: LexOrdering, - offset: isize, -) -> Result { - let updated_exprs = ordering - .into_iter() - .map(|mut sort_expr| { - sort_expr.expr = add_offset_to_expr(sort_expr.expr, offset)?; - Ok(sort_expr) - }) - .collect::>>()?; - // Can safely unwrap since adding an offset is a one-to-one operation: - Ok(LexOrdering::new(updated_exprs).unwrap()) -} - /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( e: &SortExpr, @@ -202,6 +187,19 @@ pub fn create_physical_sort_exprs( .collect() } +pub fn add_offset_to_physical_sort_exprs( + sort_exprs: impl IntoIterator, + offset: isize, +) -> Result> { + sort_exprs + .into_iter() + .map(|mut sort_expr| { + sort_expr.expr = add_offset_to_expr(sort_expr.expr, offset)?; + Ok(sort_expr) + }) + .collect() +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 67b0d3e1d87a1..8f1084f6f6d05 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -28,7 +28,9 @@ use datafusion_common::{internal_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{add_offset_to_ordering, EquivalenceProperties}; +use datafusion_physical_expr::{ + add_offset_to_physical_sort_exprs, EquivalenceProperties, +}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement, @@ -293,14 +295,14 @@ fn pushdown_requirement_to_children( Some((JoinSide::Left, ordering)) => try_pushdown_requirements_to_join( smj, parent_required.into_single(), - LexOrdering::new(ordering).unwrap(), + ordering, JoinSide::Left, ), Some((JoinSide::Right, ordering)) => { let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); - let ordering = add_offset_to_ordering( - LexOrdering::new(ordering).unwrap(), + let ordering = add_offset_to_physical_sort_exprs( + ordering, -(right_offset as isize), )?; try_pushdown_requirements_to_join( @@ -404,18 +406,19 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirement, - ordering: LexOrdering, + sort_exprs: Vec, push_side: JoinSide, ) -> Result>>> { let mut smj_required_orderings = smj.required_input_ordering(); + let ordering = LexOrdering::new(sort_exprs.clone()); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { let left_eq_properties = smj .left() .equivalence_properties() .clone() - .with_reorder(ordering.clone()); + .with_reorder(sort_exprs); let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; @@ -424,16 +427,15 @@ fn try_pushdown_requirements_to_join( { return Ok(None); } - // After re-ordering requirement is still satisfied - (Some(&ordering), smj.right().output_ordering()) + // After re-ordering, requirement is still satisfied: + (ordering.as_ref(), smj.right().output_ordering()) } JoinSide::Right => { let right_eq_properties = smj .right() .equivalence_properties() .clone() - .with_reorder(ordering.clone()); - + .with_reorder(sort_exprs); let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); }; @@ -442,8 +444,8 @@ fn try_pushdown_requirements_to_join( { return Ok(None); } - // After re-ordering requirement is still satisfied - (smj.left().output_ordering(), Some(&ordering)) + // After re-ordering, requirement is still satisfied: + (smj.left().output_ordering(), ordering.as_ref()) } JoinSide::None => return Ok(None), }; @@ -465,7 +467,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(OrderingRequirements::from(ordering)); + let new_req = ordering.map(Into::into); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 957240ca4d7d3..2ab664891afc5 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -54,7 +54,7 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - add_offset_to_expr, add_offset_to_ordering, LexOrdering, PhysicalExpr, + add_offset_to_expr, add_offset_to_physical_sort_exprs, LexOrdering, PhysicalExpr, PhysicalExprRef, }; @@ -141,14 +141,16 @@ pub fn calculate_join_output_ordering( // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { if let Some(right_ordering) = right_ordering.cloned() { - let right_offset = - add_offset_to_ordering(right_ordering, left_columns_len as _)?; + let right_offset = add_offset_to_physical_sort_exprs( + right_ordering, + left_columns_len as _, + )?; return if let Some(left_ordering) = left_ordering { let mut result = left_ordering.clone(); result.extend(right_offset); Ok(Some(result)) } else { - Ok(Some(right_offset)) + Ok(LexOrdering::new(right_offset)) }; } } @@ -158,27 +160,31 @@ pub fn calculate_join_output_ordering( // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { return if let Some(right_ordering) = right_ordering.cloned() { - let mut right_offset = - add_offset_to_ordering(right_ordering, left_columns_len as _)?; + let mut right_offset = add_offset_to_physical_sort_exprs( + right_ordering, + left_columns_len as _, + )?; if let Some(left_ordering) = left_ordering { right_offset.extend(left_ordering.clone()); } - Ok(Some(right_offset)) + Ok(LexOrdering::new(right_offset)) } else { Ok(left_ordering.cloned()) }; } - right_ordering - .map(|o| match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Full - | JoinType::Right => { - add_offset_to_ordering(o.clone(), left_columns_len as _) - } - _ => Ok(o.clone()), - }) - .transpose() + let Some(right_ordering) = right_ordering else { + return Ok(None); + }; + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + add_offset_to_physical_sort_exprs( + right_ordering.clone(), + left_columns_len as _, + ) + .map(LexOrdering::new) + } + _ => Ok(Some(right_ordering.clone())), + } } // Doesn't maintain ordering, output ordering is None. [false, false] => Ok(None), From 93722a6e286365b359cd93fdf83bdd9bd8828f52 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 01:00:47 +0300 Subject: [PATCH 150/167] Remove collapse from LexOrdering --- .../physical-expr-common/src/sort_expr.rs | 21 +++++++------------ .../physical-expr/src/equivalence/ordering.rs | 12 ++++------- .../src/equivalence/properties/mod.rs | 5 +---- 3 files changed, 12 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index f089a35ee2ffe..dd172ccf0cab9 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -328,7 +328,9 @@ impl From for PhysicalSortExpr { /// /// For example, a `vec![a ASC, b DESC]` represents a lexicographical ordering /// that first sorts by column `a` in ascending order, then by column `b` in -/// descending order. +/// descending order. The ordering is non-degenerate, meaning it contains at +/// least one element, and it is duplicate-free, meaning it does not contain +/// multiple entries for the same column. #[derive(Debug, Clone, PartialEq, Eq)] pub struct LexOrdering { exprs: Vec, @@ -338,7 +340,10 @@ impl LexOrdering { /// Creates a new [`LexOrdering`] from the given vector of sort expressions. /// If the vector is empty, returns `None`. pub fn new(exprs: impl IntoIterator) -> Option { - let exprs = exprs.into_iter().collect::>(); + let exprs = exprs + .into_iter() + .unique_by(|s| Arc::clone(&s.expr)) + .collect::>(); (!exprs.is_empty()).then(|| Self { exprs }) } @@ -358,18 +363,6 @@ impl LexOrdering { self.exprs.capacity() } - /// Constructs a duplicate-free `LexOrdering` by filtering out entries with - /// the same physical expression inside. For example, `[a ASC, a DESC]` - /// collapses to `[a ASC]`. - pub fn collapse(mut self) -> Self { - self.exprs = self - .exprs - .into_iter() - .unique_by(|s| Arc::clone(&s.expr)) - .collect(); - self - } - /// Truncates the `LexOrdering`, keeping only the first `len` elements. /// Returns `true` if truncation made a change, `false` otherwise. Negative /// cases happen in two scenarios: (1) When `len` is greater than or equal diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 4974f46c796ff..f0a0e61c94f4a 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -138,14 +138,10 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - self.orderings - .iter() - .cloned() - .reduce(|mut cat, o| { - cat.extend(o); - cat - }) - .map(|o| o.collapse()) + self.orderings.iter().cloned().reduce(|mut cat, o| { + cat.extend(o); + cat + }) } // Append orderings in `other` to all existing orderings in this ordering diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 690d08c73f08c..047c9fb678509 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -995,10 +995,7 @@ impl EquivalenceProperties { }); // Simplify each ordering by removing redundant sections: - orderings - .chain(projected_orderings) - .map(|lex_ordering| lex_ordering.collapse()) - .collect() + orderings.chain(projected_orderings).collect() } /// Projects constraints according to the given projection mapping. From e56ea339ada7cf8500ae11b082612720e5f83278 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 10:03:17 +0300 Subject: [PATCH 151/167] Remove unwraps - 5 --- datafusion/physical-plan/src/joins/cross_join.rs | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index fde9b0f3419a9..7510d34a0389b 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -115,7 +115,7 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns).with_metadata(metadata)); - let cache = Self::compute_properties(&left, &right, Arc::clone(&schema)); + let cache = Self::compute_properties(&left, &right, Arc::clone(&schema)).unwrap(); CrossJoinExec { left, @@ -142,7 +142,7 @@ impl CrossJoinExec { left: &Arc, right: &Arc, schema: SchemaRef, - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties // TODO: Check equivalence properties of cross join, it may preserve // ordering in some cases. @@ -154,9 +154,7 @@ impl CrossJoinExec { &[false, false], None, &[], - ) - // This `unwrap` is safe as we only add a positive offset to columns. - .unwrap(); + )?; // Get output partitioning: // TODO: Optimize the cross join implementation to generate M * N @@ -164,15 +162,14 @@ impl CrossJoinExec { let output_partitioning = adjust_right_output_partitioning( right.output_partitioning(), left.schema().fields.len(), - ) - .unwrap(); + )?; - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, EmissionType::Final, boundedness_from_children([left, right]), - ) + )) } /// Returns a new `ExecutionPlan` that computes the same join as this one, From 22b216ea7eadb7b65ec087073c15dc6d30663b47 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 10:29:40 +0300 Subject: [PATCH 152/167] Remove unwraps - 6 --- .../physical-expr/src/equivalence/class.rs | 4 +-- .../src/equivalence/properties/union.rs | 27 +++++++------------ 2 files changed, 11 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 8886aa8fadb3b..dc4966584736a 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -246,7 +246,7 @@ impl EquivalenceClass { /// Adds the given offset to all columns in the expressions inside this /// class. This is used when schemas are appended, e.g. in joins. - pub fn with_offset(&self, offset: isize) -> Result { + pub fn try_with_offset(&self, offset: isize) -> Result { let mut cls = Self::default(); for expr_result in self .exprs @@ -767,7 +767,7 @@ impl EquivalenceGroup { self.iter().cloned().chain( right_equivalences .iter() - .map(|cls| cls.with_offset(left_size as _)) + .map(|cls| cls.try_with_offset(left_size as _)) .collect::>>()?, ), ); diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 7d69d669a3992..5620ce9cea862 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -276,31 +276,22 @@ impl UnionEquivalentOrderingBuilder { } } -/// Advances two iterators in parallel -/// -/// If the next expressions are equal, the iterators are advanced and returns -/// the matched expression . -/// -/// Otherwise, the iterators are left unchanged and return `None` +/// Advances two iterators in parallel if the next expressions are equal. +/// Otherwise, the iterators are left unchanged and returns `None`. fn advance_if_match( iter1: &mut Peekable>, iter2: &mut Peekable>, ) -> Option { - if matches!((iter1.peek(), iter2.peek()), (Some(expr1), Some(expr2)) if expr1.eq(expr2)) - { - iter1.next().unwrap(); + let (expr1, expr2) = (iter1.peek()?, iter2.peek()?); + expr1.eq(expr2).then(|| { + iter1.next(); iter2.next().cloned() - } else { - None - } + }) + .flatten() } -/// Advances the iterator with a constant -/// -/// If the next expression matches one of the constants, advances the iterator -/// returning the matched expression -/// -/// Otherwise, the iterator is left unchanged and returns `None` +/// Advances the iterator with a constant if the next expression matches one of +/// the constants. Otherwise, the iterator is left unchanged and returns `None`. fn advance_if_matches_constant( iter: &mut Peekable>, constants: &[ConstExpr], From 6e825a3b8dedcbdf67117b8b9e6342baf39844c4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 11:14:28 +0300 Subject: [PATCH 153/167] Remove unwraps - 7 --- datafusion/physical-expr/src/equivalence/ordering.rs | 2 +- .../physical-expr/src/equivalence/properties/joins.rs | 2 +- .../physical-expr/src/equivalence/properties/mod.rs | 11 ++++++----- .../src/windows/bounded_window_agg_exec.rs | 10 +++++----- datafusion/physical-plan/src/windows/mod.rs | 8 ++++---- .../physical-plan/src/windows/window_agg_exec.rs | 10 +++++----- 6 files changed, 22 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index f0a0e61c94f4a..19171ceecd298 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -609,7 +609,7 @@ mod tests { .into_iter() .map(|eq_class| EquivalenceClass::new(eq_class.into_iter().cloned())); let eq_group = EquivalenceGroup::new(classes); - eq_properties.add_equivalence_group(eq_group); + eq_properties.add_equivalence_group(eq_group)?; let constants = constants.into_iter().map(|expr| { ConstExpr::new(Arc::clone(expr), AcrossPartitions::Uniform(None)) diff --git a/datafusion/physical-expr/src/equivalence/properties/joins.rs b/datafusion/physical-expr/src/equivalence/properties/joins.rs index b2dabe8ac77c0..53afe8124952c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/joins.rs +++ b/datafusion/physical-expr/src/equivalence/properties/joins.rs @@ -38,7 +38,7 @@ pub fn join_equivalence_properties( join_type, left_size, on, - )?); + )?)?; let EquivalenceProperties { oeq_class: left_oeq_class, diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 047c9fb678509..fa1e885d8f7a3 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -215,11 +215,11 @@ impl EquivalenceProperties { } /// Extends this `EquivalenceProperties` with the `other` object. - pub fn extend(mut self, other: Self) -> Self { + pub fn extend(mut self, other: Self) -> Result { self.constraints.extend(other.constraints); - self.add_equivalence_group(other.eq_group); + self.add_equivalence_group(other.eq_group)?; self.add_orderings(other.oeq_class); - self + Ok(self) } /// Clears (empties) the ordering equivalence class within this object. @@ -250,14 +250,15 @@ impl EquivalenceProperties { /// Incorporates the given equivalence group to into the existing /// equivalence group within. - pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { + pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) -> Result<()> { self.eq_group.extend(other_eq_group); // TODO: Normalization point. // Discover any new orderings based on the new equivalence classes: for ordering in self.normalized_oeq_class() { let leading = Arc::clone(&ordering[0].expr); - self.discover_new_orderings(leading).unwrap(); + self.discover_new_orderings(leading)?; } + Ok(()) } /// Adds a new equality condition into the existing equivalence group. diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 79e33c699451d..50c7d9a02b939 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -126,7 +126,7 @@ impl BoundedWindowAggExec { vec![] } }; - let cache = Self::compute_properties(&input, &schema, &window_expr); + let cache = Self::compute_properties(&input, &schema, &window_expr)?; Ok(Self { input, window_expr, @@ -194,9 +194,9 @@ impl BoundedWindowAggExec { input: &Arc, schema: &SchemaRef, window_exprs: &[Arc], - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties(schema, input, window_exprs); + let eq_properties = window_equivalence_properties(schema, input, window_exprs)?; // As we can have repartitioning using the partition keys, this can // be either one or more than one, depending on the presence of @@ -204,13 +204,13 @@ impl BoundedWindowAggExec { let output_partitioning = input.output_partitioning().clone(); // Construct properties cache - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, // TODO: Emission type and boundedness information can be enhanced here input.pipeline_behavior(), input.boundedness(), - ) + )) } pub fn partition_keys(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ad9fecd4b4544..e95fc48340f9d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -348,11 +348,11 @@ pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, window_exprs: &[Arc], -) -> EquivalenceProperties { +) -> Result { // We need to update the schema, so we can't directly use input's equivalence // properties. let mut window_eq_properties = EquivalenceProperties::new(Arc::clone(schema)) - .extend(input.equivalence_properties().clone()); + .extend(input.equivalence_properties().clone())?; let window_schema_len = schema.fields.len(); let input_schema_len = window_schema_len - window_exprs.len(); @@ -375,7 +375,7 @@ pub(crate) fn window_equivalence_properties( // the input plan's orderings, then we cannot further introduce any // new orderings for the window plan. if !no_partitioning && all_satisfied_lexs.is_empty() { - return window_eq_properties; + return Ok(window_eq_properties); } else if let Some(std_expr) = expr.as_any().downcast_ref::() { std_expr.add_equal_orderings(&mut window_eq_properties); @@ -486,7 +486,7 @@ pub(crate) fn window_equivalence_properties( } } } - window_eq_properties + Ok(window_eq_properties) } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 348f7248affe1..bfefd16164562 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -82,7 +82,7 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr); + let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr)?; Ok(Self { input, window_expr, @@ -123,9 +123,9 @@ impl WindowAggExec { schema: SchemaRef, input: &Arc, window_exprs: &[Arc], - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties: - let eq_properties = window_equivalence_properties(&schema, input, window_exprs); + let eq_properties = window_equivalence_properties(&schema, input, window_exprs)?; // Get output partitioning: // Because we can have repartitioning using the partition keys this @@ -133,13 +133,13 @@ impl WindowAggExec { let output_partitioning = input.output_partitioning().clone(); // Construct properties cache: - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, // TODO: Emission type and boundedness information can be enhanced here EmissionType::Final, input.boundedness(), - ) + )) } pub fn partition_keys(&self) -> Vec> { From f9f2545b4b9cf2eafaeca554cf8aad4fa44c2b98 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 11:40:53 +0300 Subject: [PATCH 154/167] Remove unwraps - 8 --- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../src/equivalence/properties/dependency.rs | 10 +- .../src/equivalence/properties/mod.rs | 25 ++- .../src/equivalence/properties/union.rs | 184 +++++++++--------- .../physical-plan/src/aggregates/mod.rs | 12 +- datafusion/physical-plan/src/filter.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 16 +- 9 files changed, 136 insertions(+), 121 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 8f6b55ab1cf50..032e6c1e082c7 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -97,7 +97,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti // Define a and f are aliases eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_f))?; // Column e has constant value. - eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))])?; // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 1ceddb6f83ab0..a8b750baf7ce9 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -81,7 +81,7 @@ mod sp_repartition_fuzz_tests { // Define a and f are aliases eq_properties.add_equal_conditions(Arc::clone(col_a), Arc::clone(col_f))?; // Column e has constant value. - eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(col_e))])?; // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 19171ceecd298..a3b65c7d76e14 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -614,7 +614,7 @@ mod tests { let constants = constants.into_iter().map(|expr| { ConstExpr::new(Arc::clone(expr), AcrossPartitions::Uniform(None)) }); - eq_properties.add_constants(constants); + eq_properties.add_constants(constants)?; let reqs = convert_to_sort_exprs(&reqs); assert_eq!( diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 6d44672d24d6f..deb66881877ec 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -785,7 +785,7 @@ mod tests { let col_h = &col("h", &test_schema)?; // Add column h as constant - eq_properties.add_constants(vec![ConstExpr::from(Arc::clone(col_h))]); + eq_properties.add_constants(vec![ConstExpr::from(Arc::clone(col_h))])?; let test_cases = vec![ // TEST CASE 1 @@ -983,7 +983,7 @@ mod tests { } properties.add_constants( case.constants.iter().cloned().map(ConstExpr::from), - ); + )?; properties }, // Constants before equal conditions @@ -991,7 +991,7 @@ mod tests { let mut properties = base_properties.clone(); properties.add_constants( case.constants.iter().cloned().map(ConstExpr::from), - ); + )?; for [left, right] in case.equal_conditions { properties.add_equal_conditions(left, right)? } @@ -1191,7 +1191,7 @@ mod tests { // Setup constant columns let col_a = col("a", &schema)?; let col_b = col("b", &schema)?; - eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_a))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_a))])?; let sort_exprs = vec![ PhysicalSortExpr::new_default(col_a), @@ -1333,7 +1333,7 @@ mod tests { let col_e = col("e", &schema)?; // Constants: c is constant - eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_c))]); + eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_c))])?; // Equality: b = d eq_properties.add_equal_conditions(Arc::clone(&col_b), Arc::clone(&col_d))?; diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index fa1e885d8f7a3..796edf6694389 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -250,7 +250,10 @@ impl EquivalenceProperties { /// Incorporates the given equivalence group to into the existing /// equivalence group within. - pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) -> Result<()> { + pub fn add_equivalence_group( + &mut self, + other_eq_group: EquivalenceGroup, + ) -> Result<()> { self.eq_group.extend(other_eq_group); // TODO: Normalization point. // Discover any new orderings based on the new equivalence classes: @@ -279,7 +282,10 @@ impl EquivalenceProperties { } /// Track/register physical expressions with constant values. - pub fn add_constants(&mut self, constants: impl IntoIterator) { + pub fn add_constants( + &mut self, + constants: impl IntoIterator, + ) -> Result<()> { // Add the new constant to the equivalence group: for constant in constants { self.eq_group.add_constant(constant); @@ -288,8 +294,9 @@ impl EquivalenceProperties { // Discover any new orderings based on the constants: for ordering in self.normalized_oeq_class() { let leading = Arc::clone(&ordering[0].expr); - self.discover_new_orderings(leading).unwrap(); + self.discover_new_orderings(leading)?; } + Ok(()) } /// Returns the ordering equivalence class within in normal form. @@ -513,7 +520,9 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties.add_constants(std::iter::once(const_expr)); + eq_properties + .add_constants(std::iter::once(const_expr)) + .unwrap(); } true } @@ -563,7 +572,9 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties.add_constants(std::iter::once(const_expr)); + eq_properties + .add_constants(std::iter::once(const_expr)) + .unwrap(); } // All sort expressions are satisfied, return full length: @@ -1101,7 +1112,9 @@ impl EquivalenceProperties { // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { let const_expr = ConstExpr::from(Arc::clone(expr)); - eq_properties.add_constants(std::iter::once(const_expr)); + eq_properties + .add_constants(std::iter::once(const_expr)) + .unwrap(); search_indices.shift_remove(idx); } // Add new ordered section to the state. diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 5620ce9cea862..f2d1da88b1bc5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -76,7 +76,7 @@ fn calculate_union_binary( let orderings = orderings.build(); let mut eq_properties = EquivalenceProperties::new(lhs.schema); - eq_properties.add_constants(constants); + eq_properties.add_constants(constants)?; eq_properties.add_orderings(orderings); Ok(eq_properties) } @@ -283,11 +283,12 @@ fn advance_if_match( iter2: &mut Peekable>, ) -> Option { let (expr1, expr2) = (iter1.peek()?, iter2.peek()?); - expr1.eq(expr2).then(|| { + if expr1.eq(expr2) { iter1.next(); iter2.next().cloned() - }) - .flatten() + } else { + None + } } /// Advances the iterator with a constant if the next expression matches one of @@ -326,84 +327,84 @@ mod tests { } #[test] - fn test_union_equivalence_properties_multi_children_1() { + fn test_union_equivalence_properties_multi_children_1() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); let schema3 = append_fields(&schema, "2"); UnionEquivalenceTest::new(&schema) // Children 1 - .with_child_sort(vec![vec!["a", "b", "c"]], &schema) + .with_child_sort(vec![vec!["a", "b", "c"]], &schema)? // Children 2 - .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2) + .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2)? // Children 3 - .with_child_sort(vec![vec!["a2", "b2"]], &schema3) - .with_expected_sort(vec![vec!["a", "b"]]) + .with_child_sort(vec![vec!["a2", "b2"]], &schema3)? + .with_expected_sort(vec![vec!["a", "b"]])? .run() } #[test] - fn test_union_equivalence_properties_multi_children_2() { + fn test_union_equivalence_properties_multi_children_2() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); let schema3 = append_fields(&schema, "2"); UnionEquivalenceTest::new(&schema) // Children 1 - .with_child_sort(vec![vec!["a", "b", "c"]], &schema) + .with_child_sort(vec![vec!["a", "b", "c"]], &schema)? // Children 2 - .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2) + .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2)? // Children 3 - .with_child_sort(vec![vec!["a2", "b2", "c2"]], &schema3) - .with_expected_sort(vec![vec!["a", "b", "c"]]) + .with_child_sort(vec![vec!["a2", "b2", "c2"]], &schema3)? + .with_expected_sort(vec![vec!["a", "b", "c"]])? .run() } #[test] - fn test_union_equivalence_properties_multi_children_3() { + fn test_union_equivalence_properties_multi_children_3() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); let schema3 = append_fields(&schema, "2"); UnionEquivalenceTest::new(&schema) // Children 1 - .with_child_sort(vec![vec!["a", "b"]], &schema) + .with_child_sort(vec![vec!["a", "b"]], &schema)? // Children 2 - .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2) + .with_child_sort(vec![vec!["a1", "b1", "c1"]], &schema2)? // Children 3 - .with_child_sort(vec![vec!["a2", "b2", "c2"]], &schema3) - .with_expected_sort(vec![vec!["a", "b"]]) + .with_child_sort(vec![vec!["a2", "b2", "c2"]], &schema3)? + .with_expected_sort(vec![vec!["a", "b"]])? .run() } #[test] - fn test_union_equivalence_properties_multi_children_4() { + fn test_union_equivalence_properties_multi_children_4() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); let schema3 = append_fields(&schema, "2"); UnionEquivalenceTest::new(&schema) // Children 1 - .with_child_sort(vec![vec!["a", "b"]], &schema) + .with_child_sort(vec![vec!["a", "b"]], &schema)? // Children 2 - .with_child_sort(vec![vec!["a1", "b1"]], &schema2) + .with_child_sort(vec![vec!["a1", "b1"]], &schema2)? // Children 3 - .with_child_sort(vec![vec!["b2", "c2"]], &schema3) - .with_expected_sort(vec![]) + .with_child_sort(vec![vec!["b2", "c2"]], &schema3)? + .with_expected_sort(vec![])? .run() } #[test] - fn test_union_equivalence_properties_multi_children_5() { + fn test_union_equivalence_properties_multi_children_5() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); UnionEquivalenceTest::new(&schema) // Children 1 - .with_child_sort(vec![vec!["a", "b"], vec!["c"]], &schema) + .with_child_sort(vec![vec!["a", "b"], vec!["c"]], &schema)? // Children 2 - .with_child_sort(vec![vec!["a1", "b1"], vec!["c1"]], &schema2) - .with_expected_sort(vec![vec!["a", "b"], vec!["c"]]) + .with_child_sort(vec![vec!["a1", "b1"], vec!["c1"]], &schema2)? + .with_expected_sort(vec![vec!["a", "b"], vec!["c"]])? .run() } #[test] - fn test_union_equivalence_properties_constants_common_constants() { + fn test_union_equivalence_properties_constants_common_constants() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -411,23 +412,23 @@ mod tests { vec![vec!["a"]], vec!["b", "c"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child: [b ASC], const [a, c] vec![vec!["b"]], vec!["a", "c"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union expected orderings: [[a ASC], [b ASC]], const [c] vec![vec!["a"], vec!["b"]], vec!["c"], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_prefix() { + fn test_union_equivalence_properties_constants_prefix() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -435,23 +436,23 @@ mod tests { vec![vec!["a"]], vec![], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child: [a ASC, b ASC], const [] vec![vec!["a", "b"]], vec![], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [a ASC], const [] vec![vec!["a"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_asc_desc_mismatch() { + fn test_union_equivalence_properties_constants_asc_desc_mismatch() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -459,23 +460,23 @@ mod tests { vec![vec!["a"]], vec![], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [a DESC], const [] vec![vec!["a DESC"]], vec![], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union doesn't have any ordering or constant vec![], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_different_schemas() { + fn test_union_equivalence_properties_constants_different_schemas() -> Result<()> { let schema = create_test_schema().unwrap(); let schema2 = append_fields(&schema, "1"); UnionEquivalenceTest::new(&schema) @@ -484,13 +485,13 @@ mod tests { vec![vec!["a"]], vec![], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [a1 ASC, b1 ASC], const [] vec![vec!["a1", "b1"]], vec![], &schema2, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [a ASC] // @@ -498,12 +499,12 @@ mod tests { // corresponding schemas. vec![vec!["a"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_fill_gaps() { + fn test_union_equivalence_properties_constants_fill_gaps() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -511,13 +512,13 @@ mod tests { vec![vec!["a", "c"]], vec!["b"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [b ASC, c ASC], const [a] vec![vec!["b", "c"]], vec!["a"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [ // [a ASC, b ASC, c ASC], @@ -525,12 +526,12 @@ mod tests { // ], const [] vec![vec!["a", "b", "c"], vec!["b", "a", "c"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_no_fill_gaps() { + fn test_union_equivalence_properties_constants_no_fill_gaps() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -538,23 +539,23 @@ mod tests { vec![vec!["a", "c"]], vec!["d"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [b ASC, c ASC], const [a] vec![vec!["b", "c"]], vec!["a"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [[a]] (only a is constant) vec![vec!["a"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_fill_some_gaps() { + fn test_union_equivalence_properties_constants_fill_some_gaps() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -562,23 +563,24 @@ mod tests { vec![vec!["c"]], vec!["a", "b"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [a DESC, b], const [] vec![vec!["a DESC", "b"]], vec![], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [[a, b]] (can fill in the a/b with constants) vec![vec!["a DESC", "b"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_fill_gaps_non_symmetric() { + fn test_union_equivalence_properties_constants_fill_gaps_non_symmetric() -> Result<()> + { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -586,13 +588,13 @@ mod tests { vec![vec!["a", "c"]], vec!["b"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child orderings: [b ASC, c ASC], const [a] vec![vec!["b DESC", "c"]], vec!["a"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: [ // [a ASC, b ASC, c ASC], @@ -600,12 +602,12 @@ mod tests { // ], const [] vec![vec!["a", "b DESC", "c"], vec!["b DESC", "a", "c"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_gap_fill_symmetric() { + fn test_union_equivalence_properties_constants_gap_fill_symmetric() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -613,25 +615,25 @@ mod tests { vec![vec!["a", "b", "d"]], vec!["c"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child: [a ASC, c ASC, d ASC], const [b] vec![vec!["a", "c", "d"]], vec!["b"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: // [a, b, c, d] // [a, c, b, d] vec![vec!["a", "c", "b", "d"], vec!["a", "b", "c", "d"]], vec![], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_gap_fill_and_common() { + fn test_union_equivalence_properties_constants_gap_fill_and_common() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -639,24 +641,24 @@ mod tests { vec![vec!["a DESC", "d"]], vec!["b", "c"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child: [a DESC, c ASC, d ASC], const [b] vec![vec!["a DESC", "c", "d"]], vec!["b"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: // [a DESC, c, d] [b] vec![vec!["a DESC", "c", "d"]], vec!["b"], - ) + )? .run() } #[test] - fn test_union_equivalence_properties_constants_middle_desc() { + fn test_union_equivalence_properties_constants_middle_desc() -> Result<()> { let schema = create_test_schema().unwrap(); UnionEquivalenceTest::new(&schema) .with_child_sort_and_const_exprs( @@ -666,20 +668,20 @@ mod tests { vec![vec!["a", "b DESC", "d"]], vec!["c"], &schema, - ) + )? .with_child_sort_and_const_exprs( // Second child: [a ASC, c ASC, d ASC], const [b] vec![vec!["a", "c", "d"]], vec!["b"], &schema, - ) + )? .with_expected_sort_and_const_exprs( // Union orderings: // [a, b, d] (c constant) // [a, c, d] (b constant) vec![vec!["a", "c", "b DESC", "d"], vec!["a", "b DESC", "c", "d"]], vec![], - ) + )? .run() } @@ -712,10 +714,10 @@ mod tests { mut self, orderings: Vec>, schema: &SchemaRef, - ) -> Self { - let properties = self.make_props(orderings, vec![], schema); + ) -> Result { + let properties = self.make_props(orderings, vec![], schema)?; self.child_properties.push(properties); - self + Ok(self) } /// Add a union input with the specified orderings and constant @@ -728,19 +730,19 @@ mod tests { orderings: Vec>, constants: Vec<&str>, schema: &SchemaRef, - ) -> Self { - let properties = self.make_props(orderings, constants, schema); + ) -> Result { + let properties = self.make_props(orderings, constants, schema)?; self.child_properties.push(properties); - self + Ok(self) } /// Set the expected output sort order for the union of the children /// /// See [`Self::make_props`] for the format of the strings in `orderings` - fn with_expected_sort(mut self, orderings: Vec>) -> Self { - let properties = self.make_props(orderings, vec![], &self.output_schema); + fn with_expected_sort(mut self, orderings: Vec>) -> Result { + let properties = self.make_props(orderings, vec![], &self.output_schema)?; self.expected_properties = Some(properties); - self + Ok(self) } /// Set the expected output sort order and constant expressions for the @@ -752,15 +754,16 @@ mod tests { mut self, orderings: Vec>, constants: Vec<&str>, - ) -> Self { - let properties = self.make_props(orderings, constants, &self.output_schema); + ) -> Result { + let properties = + self.make_props(orderings, constants, &self.output_schema)?; self.expected_properties = Some(properties); - self + Ok(self) } /// compute the union's output equivalence properties from the child /// properties, and compare them to the expected properties - fn run(self) { + fn run(self) -> Result<()> { let Self { output_schema, child_properties, @@ -792,6 +795,7 @@ mod tests { ), ); } + Ok(()) } fn assert_eq_properties_same( @@ -839,7 +843,7 @@ mod tests { orderings: Vec>, constants: Vec<&str>, schema: &SchemaRef, - ) -> EquivalenceProperties { + ) -> Result { let orderings = orderings .iter() .map(|ordering| { @@ -857,8 +861,8 @@ mod tests { let mut props = EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings); - props.add_constants(constants); - props + props.add_constants(constants)?; + Ok(props) } } @@ -878,7 +882,7 @@ mod tests { AcrossPartitions::Uniform(Some(literal_10.clone())), ); let mut input1 = EquivalenceProperties::new(Arc::clone(&schema)); - input1.add_constants(vec![const_expr1]); + input1.add_constants(vec![const_expr1])?; // Create second input with a=10 let const_expr2 = ConstExpr::new( @@ -886,7 +890,7 @@ mod tests { AcrossPartitions::Uniform(Some(literal_10.clone())), ); let mut input2 = EquivalenceProperties::new(Arc::clone(&schema)); - input2.add_constants(vec![const_expr2]); + input2.add_constants(vec![const_expr2])?; // Calculate union properties let union_props = calculate_union(vec![input1, input2], schema)?; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 8807f2a9370db..832696e08bba4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -528,7 +528,7 @@ impl AggregateExec { &mode, &input_order_mode, aggr_expr.as_slice(), - ); + )?; Ok(AggregateExec { mode, @@ -665,7 +665,7 @@ impl AggregateExec { mode: &AggregateMode, input_order_mode: &InputOrderMode, aggr_exprs: &[Arc], - ) -> PlanProperties { + ) -> Result { // Construct equivalence properties: let mut eq_properties = input .equivalence_properties() @@ -674,13 +674,11 @@ impl AggregateExec { // If the group by is empty, then we ensure that the operator will produce // only one row, and mark the generated result as a constant value. if group_expr_mapping.is_empty() { - let mut constants = eq_properties.constants().to_vec(); let new_constants = aggr_exprs.iter().enumerate().map(|(idx, func)| { let column = Arc::new(Column::new(func.name(), idx)); ConstExpr::from(column as Arc) }); - constants.extend(new_constants); - eq_properties.add_constants(constants); + eq_properties.add_constants(new_constants)?; } // Group by expression will be a distinct value after the aggregation. @@ -719,12 +717,12 @@ impl AggregateExec { input.pipeline_behavior() }; - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, emission_type, input.boundedness(), - ) + )) } pub fn input_order_mode(&self) -> &InputOrderMode { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 50de5fb87ba5e..77392ac745a2e 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -284,10 +284,10 @@ impl FilterExec { ConstExpr::new(expr, AcrossPartitions::Uniform(value.cloned())) }); // This is for statistics - eq_properties.add_constants(constants); + eq_properties.add_constants(constants)?; // This is for logical constant (for example: a = '1', then a could be marked as a constant) // to do: how to deal with multiple situation to represent = (for example c1 between 0 and 0) - eq_properties.add_constants(Self::extend_constants(input, predicate)); + eq_properties.add_constants(Self::extend_constants(input, predicate))?; let mut output_partitioning = input.output_partitioning().clone(); // If contains projection, update the PlanProperties. diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index e95fc48340f9d..fcd3ce4e15923 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -391,7 +391,7 @@ pub(crate) fn window_equivalence_properties( if no_partitioning { // Window function has a constant result across the table: window_eq_properties - .add_constants(std::iter::once(ConstExpr::from(window_col))) + .add_constants(std::iter::once(ConstExpr::from(window_col)))? } else { // Window function results in a partial constant value in // some ordering. Adjust the ordering equivalences accordingly: @@ -513,7 +513,7 @@ pub fn get_best_fitting_window( let orderby_keys = window_exprs[0].order_by(); let (should_reverse, input_order_mode) = if let Some((should_reverse, input_order_mode)) = - get_window_mode(partitionby_exprs, orderby_keys, input) + get_window_mode(partitionby_exprs, orderby_keys, input)? { (should_reverse, input_order_mode) } else { @@ -581,7 +581,7 @@ pub fn get_window_mode( partitionby_exprs: &[Arc], orderby_keys: &[PhysicalSortExpr], input: &Arc, -) -> Option<(bool, InputOrderMode)> { +) -> Result> { let mut input_eqs = input.equivalence_properties().clone(); let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); let partition_by_reqs = indices @@ -593,7 +593,7 @@ pub fn get_window_mode( .collect::>(); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().cloned().map(ConstExpr::from); - input_eqs.add_constants(const_exprs); + input_eqs.add_constants(const_exprs)?; let reverse_orderby_keys = orderby_keys.iter().map(|e| e.reverse()).collect::>(); for (should_swap, orderbys) in @@ -610,10 +610,10 @@ pub fn get_window_mode( } else { InputOrderMode::PartiallySorted(indices) }; - return Some((should_swap, mode)); + return Ok(Some((should_swap, mode))); } } - None + Ok(None) } fn sort_options_resolving_constant(expr: Arc) -> Vec { @@ -1000,7 +1000,7 @@ mod tests { order_by_exprs.push(PhysicalSortExpr { expr, options }); } let res = - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded); + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; // Since reversibility is not important in this test. Convert Option<(bool, InputOrderMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -1165,7 +1165,7 @@ mod tests { } assert_eq!( - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded), + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); From 5df5233cdfa7c05ed06f64ea85076d7fb495b92b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 12:46:29 +0300 Subject: [PATCH 155/167] Remove unwraps - 9 --- .../src/equivalence/properties/mod.rs | 14 ++++++------- .../src/enforce_distribution.rs | 4 ++-- .../src/enforce_sorting/mod.rs | 4 ++-- .../replace_with_order_preserving_variants.rs | 20 ++++++++++--------- .../src/enforce_sorting/sort_pushdown.rs | 10 +++++----- .../physical-optimizer/src/sanity_checker.rs | 2 +- .../src/update_aggr_exprs.rs | 4 ++-- datafusion/physical-optimizer/src/utils.rs | 9 +++++---- datafusion/physical-plan/src/windows/mod.rs | 2 +- 9 files changed, 35 insertions(+), 34 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 796edf6694389..38045e9bfd685 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -477,15 +477,15 @@ impl EquivalenceProperties { pub fn ordering_satisfy_requirement( &self, given: impl IntoIterator, - ) -> bool { + ) -> Result { // First, standardize the given requirement: let Some(normal_reqs) = self.normalize_sort_requirements(given) else { // If the requirement vanishes after normalization, it is satisfied: - return true; + return Ok(true); }; // Then, check whether given requirement is satisfied by constraints: if self.satisfied_by_constraints(&normal_reqs) { - return true; + return Ok(true); } let schema = self.schema(); let mut eq_properties = self.clone(); @@ -505,7 +505,7 @@ impl EquivalenceProperties { SortProperties::Unordered => false, }; if !satisfy { - return false; + return Ok(false); } // Treat satisfied keys as constants in subsequent iterations. We // can do this because the "next" key only matters in a lexicographical @@ -520,11 +520,9 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties - .add_constants(std::iter::once(const_expr)) - .unwrap(); + eq_properties.add_constants(std::iter::once(const_expr))?; } - true + Ok(true) } /// Returns the number of consecutive sort expressions (starting from the diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 84e7967b48e4f..478ce39eecb9e 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1291,7 +1291,7 @@ pub fn ensure_distribution( let ordering_satisfied = child .plan .equivalence_properties() - .ordering_satisfy_requirement(sort_req.clone()); + .ordering_satisfy_requirement(sort_req.clone())?; if (!ordering_satisfied || !order_preserving_variants_desirable) && child.data @@ -1308,7 +1308,7 @@ pub fn ensure_distribution( .downcast_ref::() .map(|output| output.fetch()) .unwrap_or(None), - ); + )?; } } // Stop tracking distribution changing operators diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 60475e9cce672..0db14d7d6d5df 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -417,7 +417,7 @@ pub fn parallelize_sorts( // deals with the children and their children and so on. requirements = requirements.children.swap_remove(0); - requirements = add_sort_above_with_check(requirements, sort_reqs, fetch); + requirements = add_sort_above_with_check(requirements, sort_reqs, fetch)?; let spm = SortPreservingMergeExec::new(sort_exprs, Arc::clone(&requirements.plan)); @@ -501,7 +501,7 @@ pub fn ensure_sorting( if let Some(required) = required_ordering { let eq_properties = child.plan.equivalence_properties(); let req = required.into_single(); - if !eq_properties.ordering_satisfy_requirement(req.clone()) { + if !eq_properties.ordering_satisfy_requirement(req.clone())? { // Make sure we preserve the ordering requirements: if physical_ordering.is_some() { child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 42c51d0fd4c18..c5f04c5f7ec90 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -178,17 +178,19 @@ pub fn plan_with_order_breaking_variants( .map(|(node, maintains, required_ordering)| { // Replace with non-order preserving variants as long as ordering is // not required by intermediate operators: - if maintains - && (is_sort_preserving_merge(plan) - || !required_ordering.is_some_and(|required_ordering| { - node.plan - .equivalence_properties() - .ordering_satisfy_requirement(required_ordering.into_single()) - })) - { + if !maintains { + Ok(node) + } else if is_sort_preserving_merge(plan) { plan_with_order_breaking_variants(node) + } else if let Some(required_ordering) = required_ordering { + let eqp = node.plan.equivalence_properties(); + if !eqp.ordering_satisfy_requirement(required_ordering.into_single())? { + plan_with_order_breaking_variants(node) + } else { + Ok(node) + } } else { - Ok(node) + plan_with_order_breaking_variants(node) } }) .collect::>()?; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 8f1084f6f6d05..ac132d81e2688 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -124,7 +124,7 @@ fn pushdown_sorts_helper( let eqp = plan.equivalence_properties(); let satisfy_parent = - eqp.ordering_satisfy_requirement(parent_requirement.first().clone()); + eqp.ordering_satisfy_requirement(parent_requirement.first().clone())?; if is_sort(&plan) { let Some(sort_ordering) = plan.output_ordering().cloned() else { @@ -234,7 +234,7 @@ fn pushdown_requirement_to_children( // that's the case, we block the pushdown of sort operation. if !plan .equivalence_properties() - .ordering_satisfy_requirement(parent_required.into_single()) + .ordering_satisfy_requirement(parent_required.into_single())? { return Ok(None); } @@ -423,7 +423,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); }; if !left_eq_properties - .ordering_satisfy_requirement(left_requirement.into_single()) + .ordering_satisfy_requirement(left_requirement.into_single())? { return Ok(None); } @@ -440,7 +440,7 @@ fn try_pushdown_requirements_to_join( return Ok(None); }; if !right_eq_properties - .ordering_satisfy_requirement(right_requirement.into_single()) + .ordering_satisfy_requirement(right_requirement.into_single())? { return Ok(None); } @@ -464,7 +464,7 @@ fn try_pushdown_requirements_to_join( // smj will have this ordering when its input changes. smj_eqs = smj_eqs.with_reorder(new_output_ordering); } - let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); + let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required)?; Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); let new_req = ordering.map(Into::into); diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index 10b023d80e171..acc70d39f057b 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -138,7 +138,7 @@ pub fn check_plan_sanity( let child_eq_props = child.equivalence_properties(); if let Some(sort_req) = sort_req { let sort_req = sort_req.into_single(); - if !child_eq_props.ordering_satisfy_requirement(sort_req.clone()) { + if !child_eq_props.ordering_satisfy_requirement(sort_req.clone())? { let plan_str = get_plan_string(&plan); return plan_err!( "Plan: {:?} does not satisfy order requirements: {}. Child-{} order: {}", diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index 03ec87773febe..f8ab40b58f985 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -170,7 +170,7 @@ fn try_convert_aggregate_if_better( .iter() .map(|e| e.clone().into()) .collect::>(), - )) { + ))? { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) } else if eq_properties.ordering_satisfy_requirement(concat_slices( @@ -179,7 +179,7 @@ fn try_convert_aggregate_if_better( .iter() .map(|e| e.reverse().into()) .collect::>(), - )) { + ))? { // Converting to reverse enables more efficient execution // given the existing ordering (if possible): aggr_expr diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 7552d202b7a97..3655e555a7440 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -17,6 +17,7 @@ use std::sync::Arc; +use datafusion_common::Result; use datafusion_physical_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; @@ -64,15 +65,15 @@ pub fn add_sort_above_with_check( node: PlanContext, sort_requirements: LexRequirement, fetch: Option, -) -> PlanContext { +) -> Result> { if !node .plan .equivalence_properties() - .ordering_satisfy_requirement(sort_requirements.clone()) + .ordering_satisfy_requirement(sort_requirements.clone())? { - add_sort_above(node, sort_requirements, fetch) + Ok(add_sort_above(node, sort_requirements, fetch)) } else { - node + Ok(node) } } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index fcd3ce4e15923..67e2b97cea6f7 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -601,7 +601,7 @@ pub fn get_window_mode( { let mut req = partition_by_reqs.clone(); req.extend(orderbys.iter().cloned().map(Into::into)); - if req.is_empty() || input_eqs.ordering_satisfy_requirement(req) { + if req.is_empty() || input_eqs.ordering_satisfy_requirement(req)? { // Window can be run with existing ordering let mode = if indices.len() == partitionby_exprs.len() { InputOrderMode::Sorted From 8c70b7bb33d0fe31812f6d04c9e7f654c22a0d9f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 20:54:01 +0300 Subject: [PATCH 156/167] Remove unwraps - 10 --- .../tests/fuzz_cases/equivalence/ordering.rs | 6 +-- .../fuzz_cases/equivalence/projection.rs | 2 +- datafusion/datasource/src/memory.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 8 ++-- .../src/equivalence/properties/dependency.rs | 8 ++-- .../src/equivalence/properties/mod.rs | 27 ++++++------- .../src/equivalence/properties/union.rs | 19 ++++----- .../src/enforce_sorting/mod.rs | 35 ++++++---------- .../replace_with_order_preserving_variants.rs | 40 ++++++++----------- .../src/enforce_sorting/sort_pushdown.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 4 +- datafusion/physical-plan/src/sorts/sort.rs | 13 +++--- datafusion/physical-plan/src/windows/mod.rs | 24 +++++++---- 13 files changed, 91 insertions(+), 99 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 0e650bf921990..1020c740a34a6 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -77,7 +77,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(ordering), + eq_properties.ordering_satisfy(ordering)?, expected, "{}", err_msg @@ -149,7 +149,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(ordering), + eq_properties.ordering_satisfy(ordering)?, (expected | false), "{}", err_msg @@ -317,7 +317,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expected ); assert_eq!( - eq_properties.ordering_satisfy(ordering), + eq_properties.ordering_satisfy(ordering)?, expected, "{err_msg}" ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index 01c493072d6b1..ae9c81d7e5dbf 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -181,7 +181,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - projected_eq.ordering_satisfy(ordering), + projected_eq.ordering_satisfy(ordering)?, expected, "{}", err_msg diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index b6ac369a53f0e..873daa8dc2797 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -772,7 +772,7 @@ impl MemorySourceConfig { target_partitions: usize, output_ordering: LexOrdering, ) -> Result>>> { - if !self.eq_properties().ordering_satisfy(output_ordering) { + if !self.eq_properties().ordering_satisfy(output_ordering)? { Ok(None) } else { let total_num_batches = diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index a3b65c7d76e14..d2b8920ecdd07 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -331,12 +331,12 @@ mod tests { Arc::clone(&input_schema), [finer.clone()], ); - assert!(eq_properties_finer.ordering_satisfy(crude.clone())); + assert!(eq_properties_finer.ordering_satisfy(crude.clone())?); // Crude ordering doesn't satisfy finer ordering. should return false let eq_properties_crude = EquivalenceProperties::new_with_orderings(Arc::clone(&input_schema), [crude]); - assert!(!eq_properties_crude.ordering_satisfy(finer)); + assert!(!eq_properties_crude.ordering_satisfy(finer)?); Ok(()) } @@ -618,7 +618,7 @@ mod tests { let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(reqs), + eq_properties.ordering_satisfy(reqs)?, expected, "{}", err_msg @@ -678,7 +678,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(reqs), + eq_properties.ordering_satisfy(reqs)?, expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index deb66881877ec..23395c8b3e62a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -1005,7 +1005,7 @@ mod tests { .collect::>>()?; assert_eq!( - properties.ordering_satisfy(sort), + properties.ordering_satisfy(sort)?, case.should_satisfy_ordering, "failed test '{}'", case.name @@ -1501,7 +1501,7 @@ mod tests { "{}: ordering {:?} should not be satisfied before adding constraints", name, ordering ); - assert!(!eq_properties.ordering_satisfy(ordering), "{err_msg}"); + assert!(!eq_properties.ordering_satisfy(ordering)?, "{err_msg}"); } // Add base ordering @@ -1521,7 +1521,7 @@ mod tests { "{}: ordering {:?} should be satisfied after adding constraints", name, ordering ); - assert!(eq_properties.ordering_satisfy(ordering), "{err_msg}"); + assert!(eq_properties.ordering_satisfy(ordering)?, "{err_msg}"); } // Test that unsatisfied orderings remain unsatisfied @@ -1530,7 +1530,7 @@ mod tests { "{}: ordering {:?} should not be satisfied after adding constraints", name, ordering ); - assert!(!eq_properties.ordering_satisfy(ordering), "{err_msg}"); + assert!(!eq_properties.ordering_satisfy(ordering)?, "{err_msg}"); } } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 38045e9bfd685..50adc0cda7b91 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -439,13 +439,13 @@ impl EquivalenceProperties { pub fn ordering_satisfy( &self, given: impl IntoIterator, - ) -> bool { + ) -> Result { // First, standardize the given ordering: let Some(normal_ordering) = self.normalize_sort_exprs(given) else { // If the ordering vanishes after normalization, it is satisfied: - return true; + return Ok(true); }; - normal_ordering.len() == self.common_sort_prefix_length(normal_ordering) + Ok(normal_ordering.len() == self.common_sort_prefix_length(normal_ordering)?) } /// Iteratively checks whether the given sort requirement is satisfied by @@ -527,13 +527,13 @@ impl EquivalenceProperties { /// Returns the number of consecutive sort expressions (starting from the /// left) that are satisfied by the existing ordering. - fn common_sort_prefix_length(&self, normal_ordering: LexOrdering) -> usize { + fn common_sort_prefix_length(&self, normal_ordering: LexOrdering) -> Result { let full_length = normal_ordering.len(); // Check whether the given ordering is satisfied by constraints: if self.satisfied_by_constraints_ordering(&normal_ordering) { // If constraints satisfy all sort expressions, return the full // length: - return full_length; + return Ok(full_length); } let schema = self.schema(); let mut eq_properties = self.clone(); @@ -555,7 +555,7 @@ impl EquivalenceProperties { if !satisfy { // As soon as one sort expression is unsatisfied, return how // many we've satisfied so far: - return idx; + return Ok(idx); } // Treat satisfied keys as constants in subsequent iterations. We // can do this because the "next" key only matters in a lexicographical @@ -570,13 +570,10 @@ impl EquivalenceProperties { // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. let const_expr = ConstExpr::from(element.expr); - eq_properties - .add_constants(std::iter::once(const_expr)) - .unwrap(); + eq_properties.add_constants(std::iter::once(const_expr))? } - // All sort expressions are satisfied, return full length: - full_length + Ok(full_length) } /// Determines the longest normal prefix of `ordering` satisfied by the @@ -585,19 +582,19 @@ impl EquivalenceProperties { pub fn extract_common_sort_prefix( &self, ordering: LexOrdering, - ) -> (Vec, bool) { + ) -> Result<(Vec, bool)> { // First, standardize the given ordering: let Some(normal_ordering) = self.normalize_sort_exprs(ordering) else { // If the ordering vanishes after normalization, it is satisfied: - return (vec![], true); + return Ok((vec![], true)); }; - let prefix_len = self.common_sort_prefix_length(normal_ordering.clone()); + let prefix_len = self.common_sort_prefix_length(normal_ordering.clone())?; let flag = prefix_len == normal_ordering.len(); let mut sort_exprs: Vec<_> = normal_ordering.into(); if !flag { sort_exprs.truncate(prefix_len); } - (sort_exprs, flag) + Ok((sort_exprs, flag)) } /// Checks if the sort expressions are satisfied by any of the table diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index f2d1da88b1bc5..9399efec7a479 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -71,8 +71,8 @@ fn calculate_union_binary( // Next, calculate valid orderings for the union by searching for prefixes // in both sides. let mut orderings = UnionEquivalentOrderingBuilder::new(); - orderings.add_satisfied_orderings(&lhs, &rhs); - orderings.add_satisfied_orderings(&rhs, &lhs); + orderings.add_satisfied_orderings(&lhs, &rhs)?; + orderings.add_satisfied_orderings(&rhs, &lhs)?; let orderings = orderings.build(); let mut eq_properties = EquivalenceProperties::new(lhs.schema); @@ -145,7 +145,7 @@ impl UnionEquivalentOrderingBuilder { &mut self, source: &EquivalenceProperties, properties: &EquivalenceProperties, - ) { + ) -> Result<()> { let constants = source.constants(); let properties_constants = properties.constants(); for mut ordering in source.normalized_oeq_class() { @@ -156,7 +156,7 @@ impl UnionEquivalentOrderingBuilder { &constants, properties, &properties_constants, - ) { + )? { AddedOrdering::Yes => break, AddedOrdering::No(ordering) => { let mut sort_exprs: Vec<_> = ordering.into(); @@ -170,6 +170,7 @@ impl UnionEquivalentOrderingBuilder { } } } + Ok(()) } /// Adds `ordering`, potentially augmented with `constants`, if it satisfies @@ -186,12 +187,12 @@ impl UnionEquivalentOrderingBuilder { constants: &[ConstExpr], properties: &EquivalenceProperties, properties_constants: &[ConstExpr], - ) -> AddedOrdering { - if properties.ordering_satisfy(ordering.clone()) { + ) -> Result { + if properties.ordering_satisfy(ordering.clone())? { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); - AddedOrdering::Yes + Ok(AddedOrdering::Yes) } else if self.try_find_augmented_ordering( &ordering, constants, @@ -199,9 +200,9 @@ impl UnionEquivalentOrderingBuilder { properties_constants, ) { // Augmented with constants to match the properties. - AddedOrdering::Yes + Ok(AddedOrdering::Yes) } else { - AddedOrdering::No(ordering) + Ok(AddedOrdering::No(ordering)) } } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 0db14d7d6d5df..8a71b28486a2a 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -283,7 +283,7 @@ fn replace_with_partial_sort( let mut common_prefix_length = 0; while child_eq_properties - .ordering_satisfy(sort_exprs[0..common_prefix_length + 1].to_vec()) + .ordering_satisfy(sort_exprs[0..common_prefix_length + 1].to_vec())? { common_prefix_length += 1; } @@ -481,7 +481,7 @@ pub fn ensure_sorting( if requirements.children.is_empty() { return Ok(Transformed::no(requirements)); } - let maybe_requirements = analyze_immediate_sort_removal(requirements); + let maybe_requirements = analyze_immediate_sort_removal(requirements)?; requirements = if !maybe_requirements.transformed { maybe_requirements.data } else { @@ -556,23 +556,19 @@ pub fn ensure_sorting( /// Otherwise, returns the original plan fn analyze_immediate_sort_removal( mut node: PlanWithCorrespondingSort, -) -> Transformed { +) -> Result> { let Some(sort_exec) = node.plan.as_any().downcast_ref::() else { - return Transformed::no(node); + return Ok(Transformed::no(node)); }; let sort_input = sort_exec.input(); // Check if the sort is unnecessary: - if !sort_exec - .properties() - .output_ordering() - .is_none_or(|ordering| { - sort_input - .equivalence_properties() - .ordering_satisfy(ordering.clone()) - }) - { - return Transformed::no(node); - }; + let properties = sort_exec.properties(); + if let Some(ordering) = properties.output_ordering().cloned() { + let eqp = sort_input.equivalence_properties(); + if !eqp.ordering_satisfy(ordering)? { + return Ok(Transformed::no(node)); + } + } node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 { @@ -589,12 +585,7 @@ fn analyze_immediate_sort_removal( node.children = node.children.swap_remove(0).children; if let Some(fetch) = sort_exec.fetch() { // If the sort has a fetch, we need to add a limit: - if sort_exec - .properties() - .output_partitioning() - .partition_count() - == 1 - { + if properties.output_partitioning().partition_count() == 1 { Arc::new(GlobalLimitExec::new(Arc::clone(sort_input), 0, Some(fetch))) } else { Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) @@ -607,7 +598,7 @@ fn analyze_immediate_sort_removal( child.data = false; } node.data = false; - Transformed::yes(node) + Ok(Transformed::yes(node)) } /// Adjusts a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index c5f04c5f7ec90..b536e7960208e 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -179,19 +179,16 @@ pub fn plan_with_order_breaking_variants( // Replace with non-order preserving variants as long as ordering is // not required by intermediate operators: if !maintains { - Ok(node) + return Ok(node); } else if is_sort_preserving_merge(plan) { - plan_with_order_breaking_variants(node) + return plan_with_order_breaking_variants(node); } else if let Some(required_ordering) = required_ordering { let eqp = node.plan.equivalence_properties(); - if !eqp.ordering_satisfy_requirement(required_ordering.into_single())? { - plan_with_order_breaking_variants(node) - } else { - Ok(node) + if eqp.ordering_satisfy_requirement(required_ordering.into_single())? { + return Ok(node); } - } else { - plan_with_order_breaking_variants(node) } + plan_with_order_breaking_variants(node) }) .collect::>()?; sort_input.data = false; @@ -280,21 +277,18 @@ pub fn replace_with_order_preserving_variants( )?; // If the alternate plan makes this sort unnecessary, accept the alternate: - if requirements.plan.output_ordering().is_none_or(|ordering| { - alternate_plan - .plan - .equivalence_properties() - .ordering_satisfy(ordering.clone()) - }) { - for child in alternate_plan.children.iter_mut() { - child.data = false; + if let Some(ordering) = requirements.plan.output_ordering() { + let eqp = alternate_plan.plan.equivalence_properties(); + if !eqp.ordering_satisfy(ordering.clone())? { + // The alternate plan does not help, use faster order-breaking variants: + alternate_plan = plan_with_order_breaking_variants(alternate_plan)?; + alternate_plan.data = false; + requirements.children = vec![alternate_plan]; + return Ok(Transformed::yes(requirements)); } - Ok(Transformed::yes(alternate_plan)) - } else { - // The alternate plan does not help, use faster order-breaking variants: - alternate_plan = plan_with_order_breaking_variants(alternate_plan)?; - alternate_plan.data = false; - requirements.children = vec![alternate_plan]; - Ok(Transformed::yes(requirements)) } + for child in alternate_plan.children.iter_mut() { + child.data = false; + } + Ok(Transformed::yes(alternate_plan)) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index ac132d81e2688..6bfca84183ae4 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -335,7 +335,7 @@ fn pushdown_requirement_to_children( let old_ordering = spm_eqs.output_ordering().unwrap(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); - if spm_eqs.ordering_satisfy(old_ordering) { + if spm_eqs.ordering_satisfy(old_ordering)? { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. Ok(Some(vec![Some(parent_required)])) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 832696e08bba4..b639f6c75d0b0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1143,7 +1143,7 @@ pub fn get_finer_aggregate_exprs_requirement( if let Some(finer) = forward_finer { if !finer { continue; - } else if eq_properties.ordering_satisfy(aggr_req.clone()) { + } else if eq_properties.ordering_satisfy(aggr_req.clone())? { requirement = Some(aggr_req); continue; } @@ -1166,7 +1166,7 @@ pub fn get_finer_aggregate_exprs_requirement( if let Some(finer) = determine_finer(&requirement, &rev_aggr_req) { if !finer { *aggr_expr = Arc::new(reverse_aggr_expr); - } else if eq_properties.ordering_satisfy(rev_aggr_req.clone()) { + } else if eq_properties.ordering_satisfy(rev_aggr_req.clone())? { *aggr_expr = Arc::new(reverse_aggr_expr); requirement = Some(rev_aggr_req); } else { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 5d6ebf8e46561..cf9080c0fe44c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -977,7 +977,8 @@ impl SortExec { pub fn new(expr: LexOrdering, input: Arc) -> Self { let preserve_partitioning = false; let (cache, sort_prefix) = - Self::compute_properties(&input, expr.clone(), preserve_partitioning); + Self::compute_properties(&input, expr.clone(), preserve_partitioning) + .unwrap(); Self { expr, input, @@ -1075,10 +1076,10 @@ impl SortExec { input: &Arc, sort_exprs: LexOrdering, preserve_partitioning: bool, - ) -> (PlanProperties, Vec) { + ) -> Result<(PlanProperties, Vec)> { let (sort_prefix, sort_satisfied) = input .equivalence_properties() - .extract_common_sort_prefix(sort_exprs.clone()); + .extract_common_sort_prefix(sort_exprs.clone())?; // The emission type depends on whether the input is already sorted: // - If already fully sorted, we can emit results in the same way as the input @@ -1116,7 +1117,7 @@ impl SortExec { let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); - ( + Ok(( PlanProperties::new( eq_properties, output_partitioning, @@ -1124,7 +1125,7 @@ impl SortExec { boundedness, ), sort_prefix, - ) + )) } } @@ -1229,7 +1230,7 @@ impl ExecutionPlan for SortExec { let sort_satisfied = self .input .equivalence_properties() - .ordering_satisfy(self.expr.clone()); + .ordering_satisfy(self.expr.clone())?; match (sort_satisfied, self.fetch.as_ref()) { (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 67e2b97cea6f7..6a3e671ebd889 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -364,13 +364,17 @@ pub(crate) fn window_equivalence_properties( // Collect columns defining partitioning, and construct all `SortOptions` // variations for them. Then, we will check each one whether it satisfies // the existing ordering provided by the input plan. - let all_satisfied_lexs = partitioning_exprs + let mut all_satisfied_lexs = vec![]; + for lex in partitioning_exprs .iter() .map(|pb_order| sort_options_resolving_constant(Arc::clone(pb_order))) .multi_cartesian_product() .filter_map(LexOrdering::new) - .filter(|lex| window_eq_properties.ordering_satisfy(lex.clone())) - .collect::>(); + { + if window_eq_properties.ordering_satisfy(lex.clone())? { + all_satisfied_lexs.push(lex); + } + } // If there is a partitioning, and no possible ordering cannot satisfy // the input plan's orderings, then we cannot further introduce any // new orderings for the window plan. @@ -453,20 +457,24 @@ pub(crate) fn window_equivalence_properties( // utilize set-monotonicity since the set shrinks as the frame // boundary starts "touching" the end of the table. else if frame.is_causal() { - let mut args_all_lexs = sliding_expr + let args_all_lexs = sliding_expr .get_aggregate_expr() .expressions() .into_iter() .map(sort_options_resolving_constant) .multi_cartesian_product(); - let mut asc = false; - if args_all_lexs.any(|order| { + let (mut asc, mut satisfied) = (false, false); + for order in args_all_lexs { if let Some(f) = order.first() { asc = !f.options.descending; } - window_eq_properties.ordering_satisfy(order) - }) { + if window_eq_properties.ordering_satisfy(order)? { + satisfied = true; + break; + } + } + if satisfied { let increasing = set_monotonicity.eq(&SetMonotonicity::Increasing); let window_col = Column::new(expr.name(), i + input_schema_len); From 73294b6766336f262cb172712c9f903525eeb5b6 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 13 May 2025 23:51:48 +0300 Subject: [PATCH 157/167] Remove collapse from LexRequirement --- .../physical-expr-common/src/sort_expr.rs | 34 +++++++++---------- .../src/output_requirements.rs | 21 ++++++------ .../physical-plan/src/aggregates/mod.rs | 4 +-- datafusion/physical-plan/src/projection.rs | 22 +++++++++++- 4 files changed, 50 insertions(+), 31 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index dd172ccf0cab9..016040f359ec8 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -449,6 +449,13 @@ impl From for Vec { /// This object represents a lexicographical ordering requirement and contains /// a vector of `PhysicalSortRequirement` objects. +/// +/// For example, a `vec![a Some(ASC), b None]` represents a lexicographical +/// requirement that firsts imposes an ordering by column `a` in ascending +/// order, then by column `b` in *any* (ascending or descending) order. The +/// ordering is non-degenerate, meaning it contains at least one element, and +/// it is duplicate-free, meaning it does not contain multiple entries for the +/// same column. #[derive(Debug, Clone, PartialEq)] pub struct LexRequirement { reqs: Vec, @@ -477,18 +484,6 @@ impl LexRequirement { pub fn capacity(&self) -> usize { self.reqs.capacity() } - - /// Constructs a duplicate-free `LexRequirement` by filtering out entries - /// with the same physical expression inside. For example, the requirement - /// `[a Some(ASC), a None]` collapses to `[a Some(ASC)]`. - pub fn collapse(mut self) -> Self { - self.reqs = self - .reqs - .into_iter() - .unique_by(|r| Arc::clone(&r.expr)) - .collect(); - self - } } impl From<[PhysicalSortRequirement; N]> for LexRequirement { @@ -510,12 +505,6 @@ impl Deref for LexRequirement { } } -impl DerefMut for LexRequirement { - fn deref_mut(&mut self) -> &mut Self::Target { - self.reqs.as_mut_slice() - } -} - impl IntoIterator for LexRequirement { type Item = PhysicalSortRequirement; type IntoIter = IntoIter; @@ -619,6 +608,15 @@ impl OrderingRequirements { Self::Hard(alts) | Self::Soft(alts) => &alts[0], } } + + /// Returns all alternatives as a vector of `LexRequirement` objects and a + /// boolean value indicating softness/hardness of the requirements. + pub fn get_alternatives(self) -> (Vec, bool) { + match self { + Self::Hard(alts) => (alts, false), + Self::Soft(alts) => (alts, true), + } + } } impl From for OrderingRequirements { diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 5da0c5bb0ac1b..2a5dc7dbc9ca0 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::Distribution; use datafusion_physical_expr_common::sort_expr::OrderingRequirements; use datafusion_physical_plan::projection::{ - make_with_child, update_expr, ProjectionExec, + make_with_child, update_expr, update_ordering_requirement, ProjectionExec, }; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -219,16 +219,17 @@ impl ExecutionPlan for OutputRequirementExec { } let mut requirements = self.required_input_ordering().swap_remove(0); - if let Some(reqs) = requirements.as_mut() { - for lex in reqs.iter_mut() { - for item in lex.iter_mut() { - let Some(new_expr) = update_expr(&item.expr, proj_exprs, false)? - else { - return Ok(None); - }; - item.expr = new_expr; - } + if let Some(reqs) = requirements { + let mut updated_reqs = vec![]; + let (lexes, soft) = reqs.get_alternatives(); + for lex in lexes.into_iter() { + let Some(updated_lex) = update_ordering_requirement(lex, proj_exprs)? + else { + return Ok(None); + }; + updated_reqs.push(updated_lex); } + requirements = OrderingRequirements::new_alternatives(updated_reqs, soft); } let dist_req = match &self.required_input_distribution()[0] { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b639f6c75d0b0..3064c9c62d6c6 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -493,8 +493,8 @@ impl AggregateExec { )?; new_requirements.extend(req); - let required_input_ordering = LexRequirement::new(new_requirements) - .map(|r| OrderingRequirements::new_soft(r.collapse())); + let required_input_ordering = + LexRequirement::new(new_requirements).map(OrderingRequirements::new_soft); // If our aggregation has grouping sets then our base grouping exprs will // be expanded based on the flags in `group_by.groups` where for each diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 2fe9ae10cfc73..a29f4aeb4090b 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -47,7 +47,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{fmt_sql, PhysicalExprRef}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -663,6 +663,8 @@ pub fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } +/// Updates the given lexicographic ordering according to given projected +/// expressions using the [`update_expr`] function. pub fn update_ordering( ordering: LexOrdering, projected_exprs: &[(Arc, String)], @@ -679,6 +681,24 @@ pub fn update_ordering( Ok(LexOrdering::new(updated_exprs)) } +/// Updates the given lexicographic requirement according to given projected +/// expressions using the [`update_expr`] function. +pub fn update_ordering_requirement( + reqs: LexRequirement, + projected_exprs: &[(Arc, String)], +) -> Result> { + let mut updated_exprs = vec![]; + for mut sort_expr in reqs.into_iter() { + let Some(updated_expr) = update_expr(&sort_expr.expr, projected_exprs, false)? + else { + return Ok(None); + }; + sort_expr.expr = updated_expr; + updated_exprs.push(sort_expr); + } + Ok(LexRequirement::new(updated_exprs)) +} + /// Downcasts all the expressions in `exprs` to `Column`s. If any of the given /// expressions is not a `Column`, returns `None`. pub fn physical_to_column_exprs( From c2197f3812970d5f39be44b904bf82d5ae7637fd Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 14 May 2025 10:43:09 +0300 Subject: [PATCH 158/167] Simplify ordering_satisfy --- .../physical-expr-common/src/sort_expr.rs | 39 ++++++++++++------- .../src/equivalence/properties/mod.rs | 20 +++++----- 2 files changed, 36 insertions(+), 23 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 7ee3e49887a30..28288b569d4c8 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -162,25 +162,24 @@ impl PhysicalSortExpr { requirement: &PhysicalSortRequirement, schema: &Schema, ) -> bool { - let opts = &requirement.options; self.expr.eq(&requirement.expr) - && if self.expr.nullable(schema).unwrap_or(true) { - opts.is_none_or(|opts| self.options == opts) - } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - opts.is_none_or(|opts| self.options.descending == opts.descending) - } + && requirement.options.is_none_or(|opts| { + options_compatible( + &self.options, + &opts, + self.expr.nullable(schema).unwrap_or(true), + ) + }) } /// Checks whether this sort expression satisfies the given `sort_expr`. pub fn satisfy_expr(&self, sort_expr: &Self, schema: &Schema) -> bool { self.expr.eq(&sort_expr.expr) - && if self.expr.nullable(schema).unwrap_or(true) { - self.options == sort_expr.options - } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - self.options.descending == sort_expr.options.descending - } + && options_compatible( + &self.options, + &sort_expr.options, + self.expr.nullable(schema).unwrap_or(true), + ) } } @@ -203,6 +202,20 @@ impl Display for PhysicalSortExpr { } } +/// Returns whether the given two [`SortOptions`] are compatible. +pub fn options_compatible( + options_lhs: &SortOptions, + options_rhs: &SortOptions, + nullable: bool, +) -> bool { + if nullable { + options_lhs == options_rhs + } else { + // If the column is not nullable, NULLS FIRST/LAST is not important. + options_lhs.descending == options_rhs.descending + } +} + /// Represents sort requirement associated with a plan /// /// If the requirement includes [`SortOptions`] then both the diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 50adc0cda7b91..9c02dba5c9ff7 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -43,6 +43,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, Constraint, Constraints, HashMap, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; +use datafusion_physical_expr_common::sort_expr::options_compatible; use datafusion_physical_expr_common::utils::ExprPropertiesNode; use indexmap::IndexSet; @@ -495,11 +496,10 @@ impl EquivalenceProperties { sort_properties, .. } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); let satisfy = match sort_properties { - SortProperties::Ordered(options) => { - let sort_expr = - PhysicalSortExpr::new(Arc::clone(&element.expr), options); - sort_expr.satisfy(&element, schema) - } + SortProperties::Ordered(options) => element.options.is_none_or(|opts| { + let nullable = element.expr.nullable(schema).unwrap_or(true); + options_compatible(&options, &opts, nullable) + }), // Singleton expressions satisfies any requirement. SortProperties::Singleton => true, SortProperties::Unordered => false, @@ -543,11 +543,11 @@ impl EquivalenceProperties { sort_properties, .. } = eq_properties.get_expr_properties(Arc::clone(&element.expr)); let satisfy = match sort_properties { - SortProperties::Ordered(options) => { - let sort_expr = - PhysicalSortExpr::new(Arc::clone(&element.expr), options); - sort_expr.satisfy_expr(&element, schema) - } + SortProperties::Ordered(options) => options_compatible( + &options, + &element.options, + element.expr.nullable(schema).unwrap_or(true), + ), // Singleton expressions satisfies any ordering. SortProperties::Singleton => true, SortProperties::Unordered => false, From 1685834061799e03aa9335b8b18c85506981fbcf Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 16 May 2025 13:04:25 +0300 Subject: [PATCH 159/167] Enforce uniqueness in LexOrdering --- .../physical-expr-common/src/sort_expr.rs | 138 +++++++++++------- .../physical-expr/src/equivalence/class.rs | 5 - .../physical-expr/src/equivalence/ordering.rs | 17 +-- .../src/equivalence/properties/dependency.rs | 6 +- .../src/equivalence/properties/mod.rs | 2 +- .../src/equivalence/properties/union.rs | 11 +- datafusion/physical-plan/src/display.rs | 31 ++-- 7 files changed, 117 insertions(+), 93 deletions(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 28288b569d4c8..9fdda4ba8bb5d 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -29,11 +29,9 @@ use crate::physical_expr::{fmt_sql, PhysicalExpr}; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::Result; +use datafusion_common::{HashSet, Result}; use datafusion_expr_common::columnar_value::ColumnarValue; -use itertools::Itertools; - /// Represents Sort operation for a column in a RecordBatch /// /// Example: @@ -304,7 +302,7 @@ impl PhysicalSortRequirement { } } -/// Returns the SQL string representation of the given [SortOptions] object. +/// Returns the SQL string representation of the given [`SortOptions`] object. #[inline] fn to_str(options: &SortOptions) -> &str { match (options.descending, options.nulls_first) { @@ -328,10 +326,9 @@ impl From for PhysicalSortExpr { /// /// Reference: fn from(value: PhysicalSortRequirement) -> Self { - let options = value.options.unwrap_or(SortOptions { - descending: false, - nulls_first: false, - }); + let options = value + .options + .unwrap_or_else(|| SortOptions::new(false, false)); Self::new(value.expr, options) } } @@ -347,27 +344,39 @@ impl From for PhysicalSortExpr { #[derive(Debug, Clone, PartialEq, Eq)] pub struct LexOrdering { exprs: Vec, + set: HashSet>, } impl LexOrdering { /// Creates a new [`LexOrdering`] from the given vector of sort expressions. /// If the vector is empty, returns `None`. pub fn new(exprs: impl IntoIterator) -> Option { - let exprs = exprs - .into_iter() - .unique_by(|s| Arc::clone(&s.expr)) - .collect::>(); - (!exprs.is_empty()).then(|| Self { exprs }) + let (non_empty, ordering) = Self::construct(exprs); + non_empty.then_some(ordering) } /// Appends an element to the back of the `LexOrdering`. - pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { - self.exprs.push(physical_sort_expr) + pub fn push(&mut self, sort_expr: PhysicalSortExpr) { + if self.set.insert(Arc::clone(&sort_expr.expr)) { + self.exprs.push(sort_expr); + } } /// Add all elements from `iter` to the `LexOrdering`. - pub fn extend(&mut self, iter: impl IntoIterator) { - self.exprs.extend(iter) + pub fn extend(&mut self, sort_exprs: impl IntoIterator) { + for sort_expr in sort_exprs { + if self.set.insert(Arc::clone(&sort_expr.expr)) { + self.exprs.push(sort_expr); + } + } + } + + /// Returns the leading `PhysicalSortExpr` of the `LexOrdering`. Note that + /// this function does not return an `Option`, as a `LexOrdering` is always + /// non-degenerate (i.e. it contains at least one element). + pub fn first(&self) -> &PhysicalSortExpr { + // Can safely `unwrap` because `LexOrdering` is non-degenerate: + self.exprs.first().unwrap() } /// Returns the number of elements that can be stored in the `LexOrdering` @@ -385,9 +394,24 @@ impl LexOrdering { if len == 0 || len >= self.exprs.len() { return false; } + for PhysicalSortExpr { expr, .. } in self.exprs[len..].iter() { + self.set.remove(expr); + } self.exprs.truncate(len); true } + + /// Constructs a new `LexOrdering` from the given sort requirements w/o + /// enforcing non-degeneracy. This function is used internally and is not + /// meant (or safe) for external use. + fn construct(exprs: impl IntoIterator) -> (bool, Self) { + let mut set = HashSet::new(); + let exprs = exprs + .into_iter() + .filter_map(|s| set.insert(Arc::clone(&s.expr)).then_some(s)) + .collect(); + (!set.is_empty(), Self { exprs, set }) + } } impl PartialOrd for LexOrdering { @@ -404,12 +428,12 @@ impl PartialOrd for LexOrdering { impl From<[PhysicalSortExpr; N]> for LexOrdering { fn from(value: [PhysicalSortExpr; N]) -> Self { - // TODO: Replace this with a condition on the generic parameter when - // Rust supports it. + // TODO: Replace this assertion with a condition on the generic parameter + // when Rust supports it. assert!(N > 0); - Self { - exprs: value.to_vec(), - } + let (non_empty, ordering) = Self::construct(value); + debug_assert!(non_empty); + ordering } } @@ -469,6 +493,11 @@ impl From for Vec { /// ordering is non-degenerate, meaning it contains at least one element, and /// it is duplicate-free, meaning it does not contain multiple entries for the /// same column. +/// +/// Note that a `LexRequirement` need not enforce the uniqueness of its sort +/// expressions after construction like a `LexOrdering` does, because it provides +/// no mutation methods. If such methods become necessary, we will need to +/// enforce uniqueness like the latter object. #[derive(Debug, Clone, PartialEq)] pub struct LexRequirement { reqs: Vec, @@ -478,35 +507,41 @@ impl LexRequirement { /// Creates a new [`LexRequirement`] from the given vector of sort expressions. /// If the vector is empty, returns `None`. pub fn new(reqs: impl IntoIterator) -> Option { - let reqs = reqs.into_iter().collect::>(); - (!reqs.is_empty()).then(|| Self { reqs }) - } - - /// Appends an element to the back of the `LexRequirement`. - pub fn push(&mut self, requirement: PhysicalSortRequirement) { - self.reqs.push(requirement) - } - - /// Add all elements from `iter` to the `LexRequirement`. - pub fn extend(&mut self, iter: impl IntoIterator) { - self.reqs.extend(iter) - } - - /// Returns the number of elements that can be stored in the `LexRequirement` - /// without reallocating. - pub fn capacity(&self) -> usize { - self.reqs.capacity() + let (non_empty, requirements) = Self::construct(reqs); + non_empty.then_some(requirements) + } + + /// Returns the leading `PhysicalSortRequirement` of the `LexRequirement`. + /// Note that this function does not return an `Option`, as a `LexRequirement` + /// is always non-degenerate (i.e. it contains at least one element). + pub fn first(&self) -> &PhysicalSortRequirement { + // Can safely `unwrap` because `LexRequirement` is non-degenerate: + self.reqs.first().unwrap() + } + + /// Constructs a new `LexRequirement` from the given sort requirements w/o + /// enforcing non-degeneracy. This function is used internally and is not + /// meant (or safe) for external use. + fn construct( + reqs: impl IntoIterator, + ) -> (bool, Self) { + let mut set = HashSet::new(); + let reqs = reqs + .into_iter() + .filter_map(|r| set.insert(Arc::clone(&r.expr)).then_some(r)) + .collect(); + (!set.is_empty(), Self { reqs }) } } impl From<[PhysicalSortRequirement; N]> for LexRequirement { fn from(value: [PhysicalSortRequirement; N]) -> Self { - // TODO: Replace this with a condition on the generic parameter when - // Rust supports it. + // TODO: Replace this assertion with a condition on the generic parameter + // when Rust supports it. assert!(N > 0); - Self { - reqs: value.to_vec(), - } + let (non_empty, requirement) = Self::construct(value); + debug_assert!(non_empty); + requirement } } @@ -546,18 +581,19 @@ impl From for Vec { impl From for LexRequirement { fn from(value: LexOrdering) -> Self { // Can construct directly as `value` is non-degenerate: - Self { - reqs: value.into_iter().map(Into::into).collect(), - } + let (non_empty, requirements) = + Self::construct(value.into_iter().map(Into::into)); + debug_assert!(non_empty); + requirements } } impl From for LexOrdering { fn from(value: LexRequirement) -> Self { // Can construct directly as `value` is non-degenerate: - Self { - exprs: value.into_iter().map(Into::into).collect(), - } + let (non_empty, ordering) = Self::construct(value.into_iter().map(Into::into)); + debug_assert!(non_empty); + ordering } } diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 4a6dd5f83377c..8739d8c19dea0 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -31,7 +31,6 @@ use datafusion_common::{HashMap, JoinType, Result, ScalarValue}; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; use indexmap::{IndexMap, IndexSet}; -use itertools::Itertools; /// Represents whether a constant expression's value is uniform or varies across /// partitions. Has two variants: @@ -529,7 +528,6 @@ impl EquivalenceGroup { /// - Replacing sections that belong to some equivalence class in the /// with the first entry in the matching equivalence class. /// - Removing expressions that have a constant value. - /// - Removing duplicate sort expressions. /// /// If columns `a` and `b` are known to be equal, `d` is known to be a /// constant, and `sort_exprs` is `[b ASC, d DESC, c ASC, a ASC]`, this @@ -542,7 +540,6 @@ impl EquivalenceGroup { .into_iter() .map(|sort_expr| self.normalize_sort_expr(sort_expr)) .filter(|sort_expr| self.is_expr_constant(&sort_expr.expr).is_none()) - .unique_by(|sort_expr| Arc::clone(&sort_expr.expr)) } /// Normalizes the given sort requirement according to this group. The @@ -562,7 +559,6 @@ impl EquivalenceGroup { /// - Replacing sections that belong to some equivalence class in the /// with the first entry in the matching equivalence class. /// - Removing expressions that have a constant value. - /// - Removing duplicate sort expressions. /// /// If columns `a` and `b` are known to be equal, `d` is known to be a /// constant, and `sort_reqs` is `[b ASC, d DESC, c ASC, a ASC]`, this @@ -575,7 +571,6 @@ impl EquivalenceGroup { .into_iter() .map(|req| self.normalize_sort_requirement(req)) .filter(|req| self.is_expr_constant(&req.expr).is_none()) - .unique_by(|req| Arc::clone(&req.expr)) } /// Perform an indirect projection of `expr` by consulting the equivalence diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 54701f32b9717..651ca57dd83de 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -221,15 +221,14 @@ impl OrderingEquivalenceClass { ]; for ordering in self.iter() { - if let Some(leading_ordering) = ordering.first() { - if leading_ordering.expr.eq(expr) { - let opt = ( - leading_ordering.options.descending, - leading_ordering.options.nulls_first, - ); - constantness_defining_pairs[0].remove(&opt); - constantness_defining_pairs[1].remove(&opt); - } + let leading_ordering = ordering.first(); + if leading_ordering.expr.eq(expr) { + let opt = ( + leading_ordering.options.descending, + leading_ordering.options.nulls_first, + ); + constantness_defining_pairs[0].remove(&opt); + constantness_defining_pairs[1].remove(&opt); } } diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 1efc829e9e0e0..adbef9ff9d4a2 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -147,7 +147,7 @@ impl<'a> DependencyEnumerator<'a> { }; for ordering in orderings.iter_mut() { - ordering.push(target.clone()) + ordering.push(target.clone()); } orderings }) @@ -652,7 +652,7 @@ mod tests { let leading_orderings = eq_properties .oeq_class() .iter() - .flat_map(|ordering| ordering.first().cloned()) + .map(|ordering| ordering.first().clone()) .collect::>(); let expr_props = eq_properties.get_expr_properties(Arc::clone(&expr)); let err_msg = format!( @@ -1347,7 +1347,7 @@ mod tests { vec![PhysicalSortExpr::new_default(Arc::clone(&col_e))], ]); - // Initial ordering: [b ASC, c ASC] + // New ordering: [b ASC, c ASC] let new_order = vec![ PhysicalSortExpr::new_default(Arc::clone(&col_b)), PhysicalSortExpr::new_default(Arc::clone(&col_c)), diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 9c02dba5c9ff7..449063dc92a6c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -994,7 +994,7 @@ impl EquivalenceProperties { // Append current ordering on top its dependencies: for ordering in prefixes.iter_mut() { if let Some(target) = &node.target { - ordering.push(target.clone()) + ordering.push(target.clone()); } } } diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 9399efec7a479..df6df1074cfeb 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -16,7 +16,6 @@ // under the License. use std::iter::Peekable; -use std::slice::Iter; use std::sync::Arc; use super::EquivalenceProperties; @@ -279,9 +278,9 @@ impl UnionEquivalentOrderingBuilder { /// Advances two iterators in parallel if the next expressions are equal. /// Otherwise, the iterators are left unchanged and returns `None`. -fn advance_if_match( - iter1: &mut Peekable>, - iter2: &mut Peekable>, +fn advance_if_match<'a>( + iter1: &mut Peekable>, + iter2: &mut Peekable>, ) -> Option { let (expr1, expr2) = (iter1.peek()?, iter2.peek()?); if expr1.eq(expr2) { @@ -294,8 +293,8 @@ fn advance_if_match( /// Advances the iterator with a constant if the next expression matches one of /// the constants. Otherwise, the iterator is left unchanged and returns `None`. -fn advance_if_matches_constant( - iter: &mut Peekable>, +fn advance_if_matches_constant<'a>( + iter: &mut Peekable>, constants: &[ConstExpr], ) -> Option { let expr = iter.peek()?; diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index f555755dd20a5..56335f13d01bf 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -1034,27 +1034,22 @@ impl fmt::Display for ProjectSchemaDisplay<'_> { } pub fn display_orderings(f: &mut Formatter, orderings: &[LexOrdering]) -> fmt::Result { - if let Some(ordering) = orderings.first() { - if !ordering.is_empty() { - let start = if orderings.len() == 1 { - ", output_ordering=" - } else { - ", output_orderings=[" - }; - write!(f, "{start}")?; - for (idx, ordering) in - orderings.iter().enumerate().filter(|(_, o)| !o.is_empty()) - { - match idx { - 0 => write!(f, "[{ordering}]")?, - _ => write!(f, ", [{ordering}]")?, - } + if !orderings.is_empty() { + let start = if orderings.len() == 1 { + ", output_ordering=" + } else { + ", output_orderings=[" + }; + write!(f, "{start}")?; + for (idx, ordering) in orderings.iter().enumerate() { + match idx { + 0 => write!(f, "[{ordering}]")?, + _ => write!(f, ", [{ordering}]")?, } - let end = if orderings.len() == 1 { "" } else { "]" }; - write!(f, "{end}")?; } + let end = if orderings.len() == 1 { "" } else { "]" }; + write!(f, "{end}")?; } - Ok(()) } From a1ae1cc18b2247f5ad91a988a6171446e85fbdc5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 19 May 2025 10:34:24 +0300 Subject: [PATCH 160/167] Fix with_reorder --- .../src/equivalence/properties/dependency.rs | 40 +++++----------- .../src/equivalence/properties/mod.rs | 48 ++++++------------- .../src/enforce_sorting/sort_pushdown.rs | 8 ++-- .../physical-plan/src/sorts/partial_sort.rs | 11 +++-- datafusion/physical-plan/src/sorts/sort.rs | 2 +- 5 files changed, 39 insertions(+), 70 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index adbef9ff9d4a2..7c421877c6bc5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -925,7 +925,7 @@ mod tests { let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) .with_reorder(["a", "b", "c"].into_iter().map(|c| { PhysicalSortExpr::new_default(col(c, schema.as_ref()).unwrap()) - })); + }))?; struct TestCase { name: &'static str, @@ -1194,17 +1194,18 @@ mod tests { eq_properties.add_constants([ConstExpr::from(Arc::clone(&col_a))])?; let sort_exprs = vec![ - PhysicalSortExpr::new_default(col_a), + PhysicalSortExpr::new_default(Arc::clone(&col_a)), PhysicalSortExpr::new_default(Arc::clone(&col_b)), ]; - let result = eq_properties.with_reorder(sort_exprs); + let result = eq_properties.with_reorder(sort_exprs)?; // Should only contain b since a is constant assert_eq!(result.oeq_class().len(), 1); let ordering = result.oeq_class().iter().next().unwrap(); - assert_eq!(ordering.len(), 1); - assert!(ordering[0].expr.eq(&col_b)); + assert_eq!(ordering.len(), 2); + assert!(ordering[0].expr.eq(&col_a)); + assert!(ordering[1].expr.eq(&col_b)); Ok(()) } @@ -1234,7 +1235,7 @@ mod tests { // New ordering: [a ASC] let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), asc)]; - let result = eq_properties.with_reorder(new_order); + let result = eq_properties.with_reorder(new_order)?; // Should only contain [a ASC, b DESC, c ASC] assert_eq!(result.oeq_class().len(), 1); @@ -1271,7 +1272,7 @@ mod tests { // New ordering: [b ASC] let new_order = vec![PhysicalSortExpr::new_default(Arc::clone(&col_b))]; - let result = eq_properties.with_reorder(new_order); + let result = eq_properties.with_reorder(new_order)?; // Should only contain [a/b ASC, c ASC] assert_eq!(result.oeq_class().len(), 1); @@ -1311,7 +1312,7 @@ mod tests { // New ordering: [a DESC] let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), desc)]; - let result = eq_properties.with_reorder(new_order.clone()); + let result = eq_properties.with_reorder(new_order.clone())?; // Should only contain the new ordering since options don't match assert_eq!(result.oeq_class().len(), 1); @@ -1353,25 +1354,10 @@ mod tests { PhysicalSortExpr::new_default(Arc::clone(&col_c)), ]; - let result = eq_properties.with_reorder(new_order); - - // Should preserve the original [d ASC, a ASC] ordering - assert_eq!(result.oeq_class().len(), 1); - let ordering = result.oeq_class().iter().next().unwrap(); - assert_eq!(ordering.len(), 2); - - // First expression should be either b or d (they're equivalent) - let asc = SortOptions::default(); - assert!( - ordering[0].expr.eq(&col_b) || ordering[0].expr.eq(&col_d), - "Expected b or d as first expression, got {:?}", - ordering[0].expr - ); - assert!(ordering[0].options.eq(&asc)); - - // Second expression should be a - assert!(ordering[1].expr.eq(&col_a)); - assert!(ordering[1].options.eq(&asc)); + let old_orderings = eq_properties.oeq_class().clone(); + let result = eq_properties.with_reorder(new_order)?; + // Original orderings should be preserved: + assert_eq!(result.oeq_class, old_orderings); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 449063dc92a6c..8f7002011d3f2 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -22,9 +22,9 @@ mod union; // Submodule containing calculate_union pub use joins::*; pub use union::*; +use std::fmt; use std::fmt::Display; use std::sync::Arc; -use std::{fmt, mem}; use self::dependency::{ construct_prefix_orderings, generate_dependency_orderings, referred_dependencies, @@ -205,14 +205,8 @@ impl EquivalenceProperties { /// Returns the output ordering of the properties. pub fn output_ordering(&self) -> Option { - let mut sort_exprs: Vec<_> = self.oeq_class.output_ordering()?.into(); - // Prune out constant expressions: - sort_exprs.retain(|sort_expr| { - self.eq_group - .get_equivalence_class(&sort_expr.expr) - .is_none_or(|cls| cls.constant.is_none()) - }); - LexOrdering::new(sort_exprs) + let concat = self.oeq_class.iter().flat_map(|o| o.iter().cloned()); + self.normalize_sort_exprs(concat) } /// Extends this `EquivalenceProperties` with the `other` object. @@ -385,31 +379,19 @@ impl EquivalenceProperties { pub fn with_reorder( mut self, ordering: impl IntoIterator, - ) -> Self { - // Normalize the given ordering and process: - if let Some(normal_ordering) = self.normalize_sort_exprs(ordering) { - // Preserve valid suffixes from existing orderings: - let mut orderings: Vec<_> = mem::take(&mut self.oeq_class).into(); - orderings.retain(|existing| { - // Check if the existing ordering is a prefix of the new ordering: - self.is_prefix_of(&normal_ordering, existing) - }); - if orderings.is_empty() { - orderings.push(normal_ordering); - } - self.oeq_class = OrderingEquivalenceClass::new(orderings); + ) -> Result { + let sort_exprs = ordering.into_iter().collect::>(); + // First, standardize the given ordering: + let Some(normal_ordering) = self.normalize_sort_exprs(sort_exprs.clone()) else { + // If the ordering vanishes after normalization, it is satisfied: + return Ok(self); + }; + if normal_ordering.len() != self.common_sort_prefix_length(normal_ordering)? { + // If the ordering is unsatisfied, replace existing orderings: + self.clear_orderings(); + self.add_ordering(sort_exprs); } - self - } - - /// Checks if the ordering `given` matches a prefix of the ordering - /// `reference` (considering expression equivalences). - fn is_prefix_of(&self, given: &LexOrdering, reference: &LexOrdering) -> bool { - given.len() <= reference.len() - && given.iter().zip(reference).all(|(new, existing)| { - self.eq_group.exprs_equal(&new.expr, &existing.expr) - && new.options == existing.options - }) + Ok(self) } /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 6bfca84183ae4..9ab9e25dae048 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -334,7 +334,7 @@ fn pushdown_requirement_to_children( let mut spm_eqs = plan.equivalence_properties().clone(); let old_ordering = spm_eqs.output_ordering().unwrap(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. - spm_eqs = spm_eqs.with_reorder(new_ordering); + spm_eqs = spm_eqs.with_reorder(new_ordering)?; if spm_eqs.ordering_satisfy(old_ordering)? { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. @@ -418,7 +418,7 @@ fn try_pushdown_requirements_to_join( .left() .equivalence_properties() .clone() - .with_reorder(sort_exprs); + .with_reorder(sort_exprs)?; let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; @@ -435,7 +435,7 @@ fn try_pushdown_requirements_to_join( .right() .equivalence_properties() .clone() - .with_reorder(sort_exprs); + .with_reorder(sort_exprs)?; let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); }; @@ -462,7 +462,7 @@ fn try_pushdown_requirements_to_join( let mut smj_eqs = smj.properties().equivalence_properties().clone(); if let Some(new_output_ordering) = new_output_ordering { // smj will have this ordering when its input changes. - smj_eqs = smj_eqs.with_reorder(new_output_ordering); + smj_eqs = smj_eqs.with_reorder(new_output_ordering)?; } let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required)?; Ok(should_pushdown.then(|| { diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 3df93fc4885c4..55ef4eec1868b 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -105,7 +105,8 @@ impl PartialSortExec { ) -> Self { debug_assert!(common_prefix_length > 0); let preserve_partitioning = false; - let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); + let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning) + .unwrap(); Self { input, expr, @@ -189,24 +190,24 @@ impl PartialSortExec { input: &Arc, sort_exprs: LexOrdering, preserve_partitioning: bool, - ) -> PlanProperties { + ) -> Result { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: let eq_properties = input .equivalence_properties() .clone() - .with_reorder(sort_exprs); + .with_reorder(sort_exprs)?; // Get output partitioning: let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); - PlanProperties::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, input.pipeline_behavior(), input.boundedness(), - ) + )) } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index dbed8104edd49..361f1bd3eb843 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1111,7 +1111,7 @@ impl SortExec { let eq_properties = input .equivalence_properties() .clone() - .with_reorder(sort_exprs); + .with_reorder(sort_exprs)?; // Get output partitioning: let output_partitioning = From 34f1afcb2c467a496667d20880ec7881eb093e8e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 19 May 2025 12:35:22 +0300 Subject: [PATCH 161/167] Use tee --- datafusion/physical-expr/src/equivalence/properties/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 8f7002011d3f2..f9e7c60a32776 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -380,16 +380,16 @@ impl EquivalenceProperties { mut self, ordering: impl IntoIterator, ) -> Result { - let sort_exprs = ordering.into_iter().collect::>(); + let (ordering, ordering_tee) = ordering.into_iter().tee(); // First, standardize the given ordering: - let Some(normal_ordering) = self.normalize_sort_exprs(sort_exprs.clone()) else { + let Some(normal_ordering) = self.normalize_sort_exprs(ordering) else { // If the ordering vanishes after normalization, it is satisfied: return Ok(self); }; if normal_ordering.len() != self.common_sort_prefix_length(normal_ordering)? { // If the ordering is unsatisfied, replace existing orderings: self.clear_orderings(); - self.add_ordering(sort_exprs); + self.add_ordering(ordering_tee); } Ok(self) } From d1b379a2505db6c63eed695721c049585f07960e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 19 May 2025 23:05:39 +0300 Subject: [PATCH 162/167] Fix reorder api --- .../src/equivalence/properties/dependency.rs | 44 +++++++++++-------- .../src/equivalence/properties/mod.rs | 18 ++++---- .../src/enforce_sorting/sort_pushdown.rs | 20 +++------ .../physical-plan/src/sorts/partial_sort.rs | 6 +-- datafusion/physical-plan/src/sorts/sort.rs | 6 +-- 5 files changed, 46 insertions(+), 48 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 7c421877c6bc5..a6718894ccc2c 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -922,10 +922,12 @@ mod tests { Field::new("b", DataType::Utf8, true), Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); - let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) - .with_reorder(["a", "b", "c"].into_iter().map(|c| { - PhysicalSortExpr::new_default(col(c, schema.as_ref()).unwrap()) - }))?; + let mut base_properties = EquivalenceProperties::new(Arc::clone(&schema)); + base_properties.reorder( + ["a", "b", "c"] + .into_iter() + .map(|c| PhysicalSortExpr::new_default(col(c, schema.as_ref()).unwrap())), + )?; struct TestCase { name: &'static str, @@ -1198,11 +1200,11 @@ mod tests { PhysicalSortExpr::new_default(Arc::clone(&col_b)), ]; - let result = eq_properties.with_reorder(sort_exprs)?; + let change = eq_properties.reorder(sort_exprs)?; + assert!(change); - // Should only contain b since a is constant - assert_eq!(result.oeq_class().len(), 1); - let ordering = result.oeq_class().iter().next().unwrap(); + assert_eq!(eq_properties.oeq_class().len(), 1); + let ordering = eq_properties.oeq_class().iter().next().unwrap(); assert_eq!(ordering.len(), 2); assert!(ordering[0].expr.eq(&col_a)); assert!(ordering[1].expr.eq(&col_b)); @@ -1235,11 +1237,12 @@ mod tests { // New ordering: [a ASC] let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), asc)]; - let result = eq_properties.with_reorder(new_order)?; + let change = eq_properties.reorder(new_order)?; + assert!(!change); // Should only contain [a ASC, b DESC, c ASC] - assert_eq!(result.oeq_class().len(), 1); - let ordering = result.oeq_class().iter().next().unwrap(); + assert_eq!(eq_properties.oeq_class().len(), 1); + let ordering = eq_properties.oeq_class().iter().next().unwrap(); assert_eq!(ordering.len(), 3); assert!(ordering[0].expr.eq(&col_a)); assert!(ordering[0].options.eq(&asc)); @@ -1272,14 +1275,15 @@ mod tests { // New ordering: [b ASC] let new_order = vec![PhysicalSortExpr::new_default(Arc::clone(&col_b))]; - let result = eq_properties.with_reorder(new_order)?; + let change = eq_properties.reorder(new_order)?; + assert!(!change); // Should only contain [a/b ASC, c ASC] - assert_eq!(result.oeq_class().len(), 1); + assert_eq!(eq_properties.oeq_class().len(), 1); // Verify orderings let asc = SortOptions::default(); - let ordering = result.oeq_class().iter().next().unwrap(); + let ordering = eq_properties.oeq_class().iter().next().unwrap(); assert_eq!(ordering.len(), 2); assert!(ordering[0].expr.eq(&col_a) || ordering[0].expr.eq(&col_b)); assert!(ordering[0].options.eq(&asc)); @@ -1312,11 +1316,12 @@ mod tests { // New ordering: [a DESC] let new_order = vec![PhysicalSortExpr::new(Arc::clone(&col_a), desc)]; - let result = eq_properties.with_reorder(new_order.clone())?; + let change = eq_properties.reorder(new_order.clone())?; + assert!(change); // Should only contain the new ordering since options don't match - assert_eq!(result.oeq_class().len(), 1); - let ordering = result.oeq_class().iter().next().unwrap(); + assert_eq!(eq_properties.oeq_class().len(), 1); + let ordering = eq_properties.oeq_class().iter().next().unwrap(); assert_eq!(ordering.to_vec(), new_order); Ok(()) @@ -1355,9 +1360,10 @@ mod tests { ]; let old_orderings = eq_properties.oeq_class().clone(); - let result = eq_properties.with_reorder(new_order)?; + let change = eq_properties.reorder(new_order)?; // Original orderings should be preserved: - assert_eq!(result.oeq_class, old_orderings); + assert!(!change); + assert_eq!(eq_properties.oeq_class, old_orderings); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index f9e7c60a32776..d7b43ed8328a5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -373,25 +373,27 @@ impl EquivalenceProperties { } /// Updates the ordering equivalence class within assuming that the table - /// is re-sorted according to the argument `ordering`. Note that equivalence - /// classes (and constants) do not change as they are unaffected by a re-sort. - /// If the given ordering is already satisfied, the function does nothing. - pub fn with_reorder( - mut self, + /// is re-sorted according to the argument `ordering`, and returns whether + /// this operation resulted in any change. Note that equivalence classes + /// (and constants) do not change as they are unaffected by a re-sort. If + /// the given ordering is already satisfied, the function does nothing. + pub fn reorder( + &mut self, ordering: impl IntoIterator, - ) -> Result { + ) -> Result { let (ordering, ordering_tee) = ordering.into_iter().tee(); // First, standardize the given ordering: let Some(normal_ordering) = self.normalize_sort_exprs(ordering) else { // If the ordering vanishes after normalization, it is satisfied: - return Ok(self); + return Ok(false); }; if normal_ordering.len() != self.common_sort_prefix_length(normal_ordering)? { // If the ordering is unsatisfied, replace existing orderings: self.clear_orderings(); self.add_ordering(ordering_tee); + return Ok(true); } - Ok(self) + Ok(false) } /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 9ab9e25dae048..bd7b0060c3be3 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -334,8 +334,8 @@ fn pushdown_requirement_to_children( let mut spm_eqs = plan.equivalence_properties().clone(); let old_ordering = spm_eqs.output_ordering().unwrap(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. - spm_eqs = spm_eqs.with_reorder(new_ordering)?; - if spm_eqs.ordering_satisfy(old_ordering)? { + let change = spm_eqs.reorder(new_ordering)?; + if !change || spm_eqs.ordering_satisfy(old_ordering)? { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. Ok(Some(vec![Some(parent_required)])) @@ -414,11 +414,8 @@ fn try_pushdown_requirements_to_join( let ordering = LexOrdering::new(sort_exprs.clone()); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { - let left_eq_properties = smj - .left() - .equivalence_properties() - .clone() - .with_reorder(sort_exprs)?; + let mut left_eq_properties = smj.left().equivalence_properties().clone(); + left_eq_properties.reorder(sort_exprs)?; let Some(left_requirement) = smj_required_orderings.swap_remove(0) else { return Ok(None); }; @@ -431,11 +428,8 @@ fn try_pushdown_requirements_to_join( (ordering.as_ref(), smj.right().output_ordering()) } JoinSide::Right => { - let right_eq_properties = smj - .right() - .equivalence_properties() - .clone() - .with_reorder(sort_exprs)?; + let mut right_eq_properties = smj.right().equivalence_properties().clone(); + right_eq_properties.reorder(sort_exprs)?; let Some(right_requirement) = smj_required_orderings.swap_remove(1) else { return Ok(None); }; @@ -462,7 +456,7 @@ fn try_pushdown_requirements_to_join( let mut smj_eqs = smj.properties().equivalence_properties().clone(); if let Some(new_output_ordering) = new_output_ordering { // smj will have this ordering when its input changes. - smj_eqs = smj_eqs.with_reorder(new_output_ordering)?; + smj_eqs.reorder(new_output_ordering)?; } let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required)?; Ok(should_pushdown.then(|| { diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 55ef4eec1868b..32b34a75cc763 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -193,10 +193,8 @@ impl PartialSortExec { ) -> Result { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: - let eq_properties = input - .equivalence_properties() - .clone() - .with_reorder(sort_exprs)?; + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.reorder(sort_exprs)?; // Get output partitioning: let output_partitioning = diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 361f1bd3eb843..dfd9633858100 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1108,10 +1108,8 @@ impl SortExec { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: - let eq_properties = input - .equivalence_properties() - .clone() - .with_reorder(sort_exprs)?; + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.reorder(sort_exprs)?; // Get output partitioning: let output_partitioning = From a87505eefa3512ee138f294cce2e99e3e10e1e45 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 20 May 2025 12:00:16 +0300 Subject: [PATCH 163/167] Comment grammar --- datafusion/physical-expr/src/equivalence/properties/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index d7b43ed8328a5..c1bd844363f17 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -484,7 +484,7 @@ impl EquivalenceProperties { let nullable = element.expr.nullable(schema).unwrap_or(true); options_compatible(&options, &opts, nullable) }), - // Singleton expressions satisfies any requirement. + // Singleton expressions satisfy any requirement. SortProperties::Singleton => true, SortProperties::Unordered => false, }; @@ -532,7 +532,7 @@ impl EquivalenceProperties { &element.options, element.expr.nullable(schema).unwrap_or(true), ), - // Singleton expressions satisfies any ordering. + // Singleton expressions satisfy any ordering. SortProperties::Singleton => true, SortProperties::Unordered => false, }; From 4ac3c2b43ab7982600a8f55a7ff16ce36c1fe036 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 20 May 2025 16:00:28 +0300 Subject: [PATCH 164/167] Remove unwraps --- .../tests/fuzz_cases/equivalence/properties.rs | 3 ++- .../src/equivalence/properties/dependency.rs | 10 +++++----- .../src/equivalence/properties/mod.rs | 8 +++----- datafusion/physical-expr/src/window/aggregate.rs | 5 +++-- datafusion/physical-expr/src/window/standard.rs | 13 +++++++++---- .../physical-optimizer/src/update_aggr_exprs.rs | 12 ++++-------- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 14 +++++++------- .../physical-plan/src/windows/window_agg_exec.rs | 2 +- 10 files changed, 36 insertions(+), 35 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 5166d7e9e75c4..e35ce3a6f8c99 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -70,7 +70,8 @@ fn test_find_longest_permutation_random() -> Result<()> { for n_req in 0..=exprs.len() { for exprs in exprs.iter().combinations(n_req) { let exprs = exprs.into_iter().cloned().collect::>(); - let (ordering, indices) = eq_properties.find_longest_permutation(&exprs); + let (ordering, indices) = + eq_properties.find_longest_permutation(&exprs)?; // Make sure that find_longest_permutation return values are consistent let ordering2 = indices .iter() diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index a6718894ccc2c..4262f98e2ef77 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -544,7 +544,7 @@ mod tests { PhysicalSortExpr::new(Arc::new(Column::new("b", 1)), sort_options_not), PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns)?; assert_eq!(idxs, vec![0, 1]); assert_eq!( result, @@ -573,7 +573,7 @@ mod tests { PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ], ]); - let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns)?; assert_eq!(idxs, vec![0, 1]); assert_eq!( result, @@ -600,7 +600,7 @@ mod tests { PhysicalSortExpr::new(Arc::new(Column::new("c", 2)), sort_options), PhysicalSortExpr::new(Arc::new(Column::new("a", 0)), sort_options), ]); - let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns)?; assert_eq!(idxs, vec![0]); Ok(()) @@ -767,7 +767,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let (actual, _) = eq_properties.find_longest_permutation(&exprs); + let (actual, _) = eq_properties.find_longest_permutation(&exprs)?; assert_eq!(actual, expected); } @@ -796,7 +796,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let (actual, _) = eq_properties.find_longest_permutation(&exprs); + let (actual, _) = eq_properties.find_longest_permutation(&exprs)?; assert_eq!(actual, expected); } diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index c1bd844363f17..a77ef69377a46 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -1043,7 +1043,7 @@ impl EquivalenceProperties { pub fn find_longest_permutation( &self, exprs: &[Arc], - ) -> (Vec, Vec) { + ) -> Result<(Vec, Vec)> { let mut eq_properties = self.clone(); let mut result = vec![]; // The algorithm is as follows: @@ -1091,15 +1091,13 @@ impl EquivalenceProperties { // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { let const_expr = ConstExpr::from(Arc::clone(expr)); - eq_properties - .add_constants(std::iter::once(const_expr)) - .unwrap(); + eq_properties.add_constants(std::iter::once(const_expr))?; search_indices.shift_remove(idx); } // Add new ordered section to the state. result.extend(ordered_exprs); } - result.into_iter().unzip() + Ok(result.into_iter().unzip()) } /// This function determines whether the provided expression is constant diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 07d392666f128..4161cbd3b1bbe 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -72,7 +72,7 @@ impl PlainAggregateWindowExpr { &self, eq_properties: &mut EquivalenceProperties, window_expr_index: usize, - ) { + ) -> Result<()> { if let Some(expr) = self .get_aggregate_expr() .get_result_ordering(window_expr_index) @@ -81,8 +81,9 @@ impl PlainAggregateWindowExpr { eq_properties, expr, &self.partition_by, - ); + )?; } + Ok(()) } } diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 26deee65925b1..57306efff2024 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -70,15 +70,19 @@ impl StandardWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + pub fn add_equal_orderings( + &self, + eq_properties: &mut EquivalenceProperties, + ) -> Result<()> { let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { add_new_ordering_expr_with_partition_by( eq_properties, fn_res_ordering, &self.partition_by, - ); + )?; } + Ok(()) } } @@ -279,7 +283,7 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( eqp: &mut EquivalenceProperties, expr: PhysicalSortExpr, partition_by: &[Arc], -) { +) -> Result<()> { if partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: eqp.add_ordering([expr]); @@ -290,10 +294,11 @@ pub(crate) fn add_new_ordering_expr_with_partition_by( // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - let (mut ordering, _) = eqp.find_longest_permutation(partition_by); + let (mut ordering, _) = eqp.find_longest_permutation(partition_by)?; if ordering.len() == partition_by.len() { ordering.push(expr); eqp.add_ordering(ordering); } } + Ok(()) } diff --git a/datafusion/physical-optimizer/src/update_aggr_exprs.rs b/datafusion/physical-optimizer/src/update_aggr_exprs.rs index f8ab40b58f985..61bc715592af6 100644 --- a/datafusion/physical-optimizer/src/update_aggr_exprs.rs +++ b/datafusion/physical-optimizer/src/update_aggr_exprs.rs @@ -25,11 +25,9 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; -use datafusion_physical_plan::aggregates::concat_slices; +use datafusion_physical_plan::aggregates::{concat_slices, AggregateExec}; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; -use datafusion_physical_plan::{ - aggregates::AggregateExec, ExecutionPlan, ExecutionPlanProperties, -}; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use crate::PhysicalOptimizerRule; @@ -93,14 +91,12 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { // If the existing ordering satisfies a prefix of the GROUP BY // expressions, prefix requirements with this section. In this // case, aggregation will work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, input); + let indices = get_ordered_partition_by_indices(&groupby_exprs, input)?; let requirement = indices .iter() .map(|&idx| { PhysicalSortRequirement::new( - Arc::::clone( - &groupby_exprs[idx], - ), + Arc::clone(&groupby_exprs[idx]), None, ) }) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3064c9c62d6c6..5592872024a58 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -477,7 +477,7 @@ impl AggregateExec { // If existing ordering satisfies a prefix of the GROUP BY expressions, // prefix requirements with this section. In this case, aggregation will // work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input)?; let mut new_requirements = indices .iter() .map(|&idx| { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 756a73e7c71e0..d851d08a101f8 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -114,7 +114,7 @@ impl BoundedWindowAggExec { let indices = get_ordered_partition_by_indices( window_expr[0].partition_by(), &input, - ); + )?; if indices.len() == partition_by_exprs.len() { indices } else { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 6a3e671ebd889..891a2176f7be1 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -316,11 +316,11 @@ pub(crate) fn calc_requirements< pub fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, -) -> Vec { +) -> Result> { let (_, indices) = input .equivalence_properties() - .find_longest_permutation(partition_by_exprs); - indices + .find_longest_permutation(partition_by_exprs)?; + Ok(indices) } pub(crate) fn get_partition_by_sort_exprs( @@ -336,7 +336,7 @@ pub(crate) fn get_partition_by_sort_exprs( assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); let (ordering, _) = input .equivalence_properties() - .find_longest_permutation(&ordered_partition_exprs); + .find_longest_permutation(&ordered_partition_exprs)?; if ordering.len() == ordered_partition_exprs.len() { Ok(ordering) } else { @@ -382,7 +382,7 @@ pub(crate) fn window_equivalence_properties( return Ok(window_eq_properties); } else if let Some(std_expr) = expr.as_any().downcast_ref::() { - std_expr.add_equal_orderings(&mut window_eq_properties); + std_expr.add_equal_orderings(&mut window_eq_properties)?; } else if let Some(plain_expr) = expr.as_any().downcast_ref::() { @@ -417,7 +417,7 @@ pub(crate) fn window_equivalence_properties( plain_expr.add_equal_orderings( &mut window_eq_properties, window_expr_indices[i], - ); + )?; } } else if let Some(sliding_expr) = expr.as_any().downcast_ref::() @@ -591,7 +591,7 @@ pub fn get_window_mode( input: &Arc, ) -> Result> { let mut input_eqs = input.equivalence_properties().clone(); - let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); + let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs)?; let partition_by_reqs = indices .iter() .map(|&idx| PhysicalSortRequirement { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index bfefd16164562..1b7cb9bb76e1b 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -81,7 +81,7 @@ impl WindowAggExec { let schema = Arc::new(schema); let ordered_partition_by_indices = - get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); + get_ordered_partition_by_indices(window_expr[0].partition_by(), &input)?; let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr)?; Ok(Self { input, From c704256f481b82f6630ccf373e33b9301d686f5f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 30 May 2025 14:26:38 +0300 Subject: [PATCH 165/167] Cache normalized orderings --- .../physical-expr/src/equivalence/class.rs | 32 ++- .../physical-expr/src/equivalence/ordering.rs | 27 ++ .../src/equivalence/properties/mod.rs | 239 ++++++++++++------ .../src/equivalence/properties/union.rs | 27 +- 4 files changed, 226 insertions(+), 99 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 8739d8c19dea0..0e722f3dc8246 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -356,11 +356,13 @@ impl EquivalenceGroup { /// Removes constant expressions that may change across partitions. /// This method should be used when merging data from different partitions. - pub fn clear_per_partition_constants(&mut self) { - let mut idx = 0; + /// Returns whether any change was made to the equivalence group. + pub fn clear_per_partition_constants(&mut self) -> bool { + let (mut idx, mut change) = (0, false); while idx < self.classes.len() { let cls = &mut self.classes[idx]; if let Some(AcrossPartitions::Heterogeneous) = cls.constant { + change = true; if cls.len() == 1 { // If this class becomes trivial, remove it entirely: self.remove_class_at_idx(idx); @@ -371,6 +373,7 @@ impl EquivalenceGroup { } idx += 1; } + change } /// Adds the equality `left` = `right` to this equivalence group. New @@ -454,45 +457,53 @@ impl EquivalenceGroup { } } - /// Removes redundant entries from this group. - fn remove_redundant_entries(&mut self) { + /// Removes redundant entries from this group. Returns whether any change + /// was made to the equivalence group. + fn remove_redundant_entries(&mut self) -> bool { // First, remove trivial equivalence classes: + let mut change = false; for idx in (0..self.classes.len()).rev() { if self.classes[idx].is_trivial() { self.remove_class_at_idx(idx); + change = true; } } // Then, unify/bridge groups that have common expressions: - self.bridge_classes() + self.bridge_classes() || change } /// This utility function unifies/bridges classes that have common expressions. /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all /// equal and belong to one class. This utility converts merges such classes. - fn bridge_classes(&mut self) { - let mut idx = 0; + /// Returns whether any change was made to the equivalence group. + fn bridge_classes(&mut self) -> bool { + let (mut idx, mut change) = (0, false); 'scan: while idx < self.classes.len() { for other_idx in (idx + 1..self.classes.len()).rev() { if self.classes[idx].contains_any(&self.classes[other_idx]) { let extension = self.remove_class_at_idx(other_idx); Self::update_lookup_table(&mut self.map, &extension, idx); self.classes[idx].extend(extension); + change = true; continue 'scan; } } idx += 1; } + change } /// Extends this equivalence group with the `other` equivalence group. - pub fn extend(&mut self, other: Self) { + /// Returns whether any equivalence classes were unified/bridged as a + /// result of the extension process. + pub fn extend(&mut self, other: Self) -> bool { for (idx, cls) in other.classes.iter().enumerate() { // Update the lookup table for the new class: Self::update_lookup_table(&mut self.map, cls, idx); } self.classes.extend(other.classes); - self.remove_redundant_entries(); + self.bridge_classes() } /// Normalizes the given physical expression according to this group. The @@ -964,8 +975,7 @@ mod tests { EquivalenceClass::new([lit(30)]), EquivalenceClass::new([c(2), c(3), c(4)]), ]; - let mut eq_groups = EquivalenceGroup::new(entries); - eq_groups.remove_redundant_entries(); + let eq_groups = EquivalenceGroup::new(entries); assert_eq!(eq_groups.classes, expected); Ok(()) } diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 651ca57dd83de..0ce7ccab4683d 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -20,9 +20,11 @@ use std::ops::Deref; use std::sync::Arc; use std::vec::IntoIter; +use crate::expressions::with_new_schema; use crate::{add_offset_to_physical_sort_exprs, LexOrdering, PhysicalExpr}; use arrow::compute::SortOptions; +use arrow::datatypes::SchemaRef; use datafusion_common::{HashSet, Result}; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; @@ -175,6 +177,31 @@ impl OrderingEquivalenceClass { Ok(()) } + /// Transforms this `OrderingEquivalenceClass` by mapping columns in the + /// original schema to columns in the new schema by index. The new schema + /// and the original schema needs to be aligned; i.e. they should have the + /// same number of columns, and fields at the same index have the same type + /// in both schemas. + pub fn with_new_schema(mut self, schema: &SchemaRef) -> Result { + self.orderings = self + .orderings + .into_iter() + .map(|ordering| { + ordering + .into_iter() + .map(|mut sort_expr| { + sort_expr.expr = with_new_schema(sort_expr.expr, schema)?; + Ok(sort_expr) + }) + .collect::>>() + // The following `unwrap` is safe because the vector will always + // be non-empty. + .map(|v| LexOrdering::new(v).unwrap()) + }) + .collect::>()?; + Ok(self) + } + /// Gets sort options associated with this expression if it is a leading /// ordering expression. Otherwise, returns `None`. pub fn get_options(&self, expr: &Arc) -> Option { diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index a77ef69377a46..0b46129525ff7 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -22,16 +22,17 @@ mod union; // Submodule containing calculate_union pub use joins::*; pub use union::*; -use std::fmt; -use std::fmt::Display; +use std::fmt::{self, Display}; +use std::mem; use std::sync::Arc; use self::dependency::{ construct_prefix_orderings, generate_dependency_orderings, referred_dependencies, Dependencies, DependencyMap, }; -use crate::equivalence::class::AcrossPartitions; -use crate::equivalence::{EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping}; +use crate::equivalence::{ + AcrossPartitions, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, +}; use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; use crate::{ ConstExpr, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, @@ -128,24 +129,75 @@ use itertools::Itertools; /// /// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], eq: [{members: [b@1], constant: (heterogeneous)}]"); /// ``` -#[derive(Debug, Clone)] +#[derive(Clone, Debug)] pub struct EquivalenceProperties { /// Distinct equivalence classes (i.e. expressions with the same value). eq_group: EquivalenceGroup, /// Equivalent sort expressions (i.e. those define the same ordering). oeq_class: OrderingEquivalenceClass, + /// Cache storing equivalent sort expressions in normal form (i.e. without + /// constants/duplicates and in standard form) and a map associating leading + /// terms with full sort expressions. + oeq_cache: OrderingEquivalenceCache, /// Table constraints that factor in equivalence calculations. constraints: Constraints, /// Schema associated with this object. schema: SchemaRef, } +/// This object serves as a cache for storing equivalent sort expressions +/// in normal form, and a map associating leading sort expressions with +/// full lexicographical orderings. With this information, DataFusion can +/// efficiently determine whether a given ordering is satisfied by the +/// existing orderings, and discover new orderings based on the existing +/// equivalence properties. +#[derive(Clone, Debug, Default)] +struct OrderingEquivalenceCache { + /// Equivalent sort expressions in normal form. + normal_cls: OrderingEquivalenceClass, + /// Map associating leading sort expressions with full lexicographical + /// orderings. Values are indices into `normal_cls`. + leading_map: HashMap, Vec>, +} + +impl OrderingEquivalenceCache { + /// Creates a new `OrderingEquivalenceCache` object with the given + /// equivalent orderings, which should be in normal form. + pub fn new( + orderings: impl IntoIterator>, + ) -> Self { + let mut cache = Self { + normal_cls: OrderingEquivalenceClass::new(orderings), + leading_map: HashMap::new(), + }; + cache.update_map(); + cache + } + + /// Updates/reconstructs the leading expression map according to the normal + /// ordering equivalence class within. + pub fn update_map(&mut self) { + self.leading_map.clear(); + for (idx, ordering) in self.normal_cls.iter().enumerate() { + let expr = Arc::clone(&ordering.first().expr); + self.leading_map.entry(expr).or_default().push(idx); + } + } + + /// Clears the cache, removing all orderings and leading expressions. + pub fn clear(&mut self) { + self.normal_cls.clear(); + self.leading_map.clear(); + } +} + impl EquivalenceProperties { /// Creates an empty `EquivalenceProperties` object. pub fn new(schema: SchemaRef) -> Self { Self { eq_group: EquivalenceGroup::default(), oeq_class: OrderingEquivalenceClass::default(), + oeq_cache: OrderingEquivalenceCache::default(), constraints: Constraints::default(), schema, } @@ -162,9 +214,18 @@ impl EquivalenceProperties { schema: SchemaRef, orderings: impl IntoIterator>, ) -> Self { + let eq_group = EquivalenceGroup::default(); + let oeq_class = OrderingEquivalenceClass::new(orderings); + // Here, we can avoid performing a full normalization, and get by with + // only removing constants because the equivalence group is empty. + let normal_orderings = oeq_class.iter().cloned().map(|o| { + o.into_iter() + .filter(|sort_expr| eq_group.is_expr_constant(&sort_expr.expr).is_none()) + }); Self { - eq_group: EquivalenceGroup::default(), - oeq_class: OrderingEquivalenceClass::new(orderings), + oeq_cache: OrderingEquivalenceCache::new(normal_orderings), + oeq_class, + eq_group, constraints: Constraints::default(), schema, } @@ -221,12 +282,21 @@ impl EquivalenceProperties { /// Call this method when existing orderings are invalidated. pub fn clear_orderings(&mut self) { self.oeq_class.clear(); + self.oeq_cache.clear(); } /// Removes constant expressions that may change across partitions. /// This method should be used when merging data from different partitions. pub fn clear_per_partition_constants(&mut self) { - self.eq_group.clear_per_partition_constants(); + if self.eq_group.clear_per_partition_constants() { + // Renormalize orderings if the equivalence group changes: + let normal_orderings = self + .oeq_class + .iter() + .cloned() + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_cache = OrderingEquivalenceCache::new(normal_orderings); + } } /// Adds new orderings into the existing ordering equivalence class. @@ -234,8 +304,21 @@ impl EquivalenceProperties { &mut self, orderings: impl IntoIterator>, ) { - // TODO: Normalization point. - self.oeq_class.add_orderings(orderings); + let orderings: Vec<_> = + orderings.into_iter().filter_map(LexOrdering::new).collect(); + let normal_orderings: Vec<_> = orderings + .iter() + .cloned() + .filter_map(|o| self.normalize_sort_exprs(o)) + .collect(); + if !normal_orderings.is_empty() { + self.oeq_class.extend(orderings); + // Normalize given orderings to update the cache: + self.oeq_cache.normal_cls.extend(normal_orderings); + // TODO: If no ordering is found to be redunant during extension, we + // can use a shortcut algorithm to update the leading map. + self.oeq_cache.update_map(); + } } /// Adds a single ordering to the existing ordering equivalence class. @@ -249,16 +332,37 @@ impl EquivalenceProperties { &mut self, other_eq_group: EquivalenceGroup, ) -> Result<()> { - self.eq_group.extend(other_eq_group); - // TODO: Normalization point. - // Discover any new orderings based on the new equivalence classes: - for ordering in self.normalized_oeq_class() { - let leading = Arc::clone(&ordering[0].expr); - self.discover_new_orderings(leading)?; + if !other_eq_group.is_empty() { + self.eq_group.extend(other_eq_group); + // Renormalize orderings if the equivalence group changes: + let normal_cls = mem::take(&mut self.oeq_cache.normal_cls); + let normal_orderings = normal_cls + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_cache.normal_cls = OrderingEquivalenceClass::new(normal_orderings); + self.oeq_cache.update_map(); + // Discover any new orderings based on the new equivalence classes: + let leading_exprs: Vec<_> = + self.oeq_cache.leading_map.keys().cloned().collect(); + for expr in leading_exprs { + self.discover_new_orderings(expr)?; + } } Ok(()) } + /// Returns the ordering equivalence class within in normal form. + /// Normalization standardizes expressions according to the equivalence + /// group within, and removes constants/duplicates. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { + self.oeq_class + .iter() + .cloned() + .filter_map(|ordering| self.normalize_sort_exprs(ordering)) + .collect::>() + .into() + } + /// Adds a new equality condition into the existing equivalence group. /// If the given equality defines a new equivalence class, adds this new /// equivalence class to the equivalence group. @@ -269,7 +373,13 @@ impl EquivalenceProperties { ) -> Result<()> { // Add equal expressions to the state: if self.eq_group.add_equal_conditions(Arc::clone(&left), right) { - // TODO: Normalization point. + // Renormalize orderings if the equivalence group changes: + let normal_cls = mem::take(&mut self.oeq_cache.normal_cls); + let normal_orderings = normal_cls + .into_iter() + .map(|o| self.eq_group.normalize_sort_exprs(o)); + self.oeq_cache.normal_cls = OrderingEquivalenceClass::new(normal_orderings); + self.oeq_cache.update_map(); // Discover any new orderings: self.discover_new_orderings(left)?; } @@ -285,48 +395,44 @@ impl EquivalenceProperties { for constant in constants { self.eq_group.add_constant(constant); } - // TODO: Normalization point. + // Renormalize the orderings after adding new constants by removing + // the constants from existing orderings: + let normal_cls = mem::take(&mut self.oeq_cache.normal_cls); + let normal_orderings = normal_cls.into_iter().map(|ordering| { + ordering.into_iter().filter(|sort_expr| { + self.eq_group.is_expr_constant(&sort_expr.expr).is_none() + }) + }); + self.oeq_cache.normal_cls = OrderingEquivalenceClass::new(normal_orderings); + self.oeq_cache.update_map(); // Discover any new orderings based on the constants: - for ordering in self.normalized_oeq_class() { - let leading = Arc::clone(&ordering[0].expr); - self.discover_new_orderings(leading)?; + let leading_exprs: Vec<_> = self.oeq_cache.leading_map.keys().cloned().collect(); + for expr in leading_exprs { + self.discover_new_orderings(expr)?; } Ok(()) } - /// Returns the ordering equivalence class within in normal form. - /// Normalization standardizes expressions according to the equivalence - /// group within, and removes constants/duplicates. - pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { - self.oeq_class - .iter() - .cloned() - .filter_map(|ordering| self.normalize_sort_exprs(ordering)) - .collect::>() - .into() - } - /// Discover new valid orderings in light of a new equality. Accepts a single /// argument (`expr`) which is used to determine the orderings to update. /// When constants or equivalence classes change, there may be new orderings /// that can be discovered with the new equivalence properties. /// For a discussion, see: - fn discover_new_orderings(&mut self, expr: Arc) -> Result<()> { - let normal_expr = self.eq_group.normalize_expr(expr); + fn discover_new_orderings( + &mut self, + normal_expr: Arc, + ) -> Result<()> { + let Some(ordering_idxs) = self.oeq_cache.leading_map.get(&normal_expr) else { + return Ok(()); + }; let eq_class = self .eq_group .get_equivalence_class(&normal_expr) - .map_or_else( - || vec![Arc::clone(&normal_expr)], - |class| class.clone().into(), - ); + .map_or_else(|| vec![normal_expr], |class| class.clone().into()); let mut new_orderings = vec![]; - for ordering in self.normalized_oeq_class() { - if !ordering[0].expr.eq(&normal_expr) { - continue; - } - + for idx in ordering_idxs { + let ordering = &self.oeq_cache.normal_cls[*idx]; let leading_ordering_options = ordering[0].options; 'exprs: for equivalent_expr in &eq_class { @@ -368,7 +474,9 @@ impl EquivalenceProperties { } } - self.add_orderings(new_orderings); + if !new_orderings.is_empty() { + self.add_orderings(new_orderings); + } Ok(()) } @@ -1020,9 +1128,14 @@ impl EquivalenceProperties { /// `output_schema`. pub fn project(&self, mapping: &ProjectionMapping, output_schema: SchemaRef) -> Self { let eq_group = self.eq_group.project(mapping); - let orderings = self.projected_orderings(mapping, self.normalized_oeq_class()); - // TODO: Normalization point. + let orderings = + self.projected_orderings(mapping, self.oeq_cache.normal_cls.clone()); + let normal_orderings = orderings + .iter() + .cloned() + .map(|o| eq_group.normalize_sort_exprs(o)); Self { + oeq_cache: OrderingEquivalenceCache::new(normal_orderings), oeq_class: OrderingEquivalenceClass::new(orderings), constraints: self.projected_constraints(mapping).unwrap_or_default(), schema: output_schema, @@ -1146,9 +1259,8 @@ impl EquivalenceProperties { .unwrap_or_else(|_| ExprProperties::new_unknown()) } - /// Transforms this `EquivalenceProperties` into a new `EquivalenceProperties` - /// by mapping columns in the original schema to columns in the new schema - /// by index. + /// Transforms this `EquivalenceProperties` by mapping columns in the + /// original schema to columns in the new schema by index. pub fn with_new_schema(mut self, schema: SchemaRef) -> Result { // The new schema and the original schema is aligned when they have the // same number of columns, and fields at the same index have the same @@ -1191,30 +1303,15 @@ impl EquivalenceProperties { } eq_classes.push(eq_class); } + self.eq_group = eq_classes.into(); // Rewrite orderings according to new schema: - let new_orderings = self - .oeq_class - .into_iter() - .map(|ordering| { - ordering - .into_iter() - .map(|mut sort_expr| { - sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; - Ok(sort_expr) - }) - .collect::>>() - // The following `unwrap` is safe because the vector will always - // be non-empty. - .map(|v| LexOrdering::new(v).unwrap()) - }) - .collect::>>()?; + self.oeq_class = self.oeq_class.with_new_schema(&schema)?; + self.oeq_cache.normal_cls = self.oeq_cache.normal_cls.with_new_schema(&schema)?; - // Update the schema, the equivalence group and the ordering equivalence - // class: + // Update the schema: self.schema = schema; - self.eq_group = eq_classes.into(); - self.oeq_class = new_orderings.into(); + Ok(self) } } @@ -1290,7 +1387,7 @@ fn update_properties( let normal_expr = eq_properties .eq_group .normalize_expr(Arc::clone(&node.expr)); - let oeq_class = eq_properties.normalized_oeq_class(); + let oeq_class = &eq_properties.oeq_cache.normal_cls; if eq_properties.is_expr_constant(&normal_expr).is_some() || oeq_class.is_expr_partial_const(&normal_expr) { diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index df6df1074cfeb..4f44b9b0c9d4a 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -26,15 +26,15 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -/// Calculates the union (in the sense of `UnionExec`) `EquivalenceProperties` -/// of `lhs` and `rhs` according to the schema of `lhs`. +/// Computes the union (in the sense of `UnionExec`) `EquivalenceProperties` +/// of `lhs` and `rhs` according to the schema of `lhs`. /// -/// Rules: The UnionExec does not interleave its inputs: instead it passes each -/// input partition from the children as its own output. +/// Rules: The `UnionExec` does not interleave its inputs, instead it passes +/// each input partition from the children as its own output. /// /// Since the output equivalence properties are properties that are true for /// *all* output partitions, that is the same as being true for all *input* -/// partitions +/// partitions. fn calculate_union_binary( lhs: EquivalenceProperties, mut rhs: EquivalenceProperties, @@ -147,7 +147,7 @@ impl UnionEquivalentOrderingBuilder { ) -> Result<()> { let constants = source.constants(); let properties_constants = properties.constants(); - for mut ordering in source.normalized_oeq_class() { + for mut ordering in source.oeq_cache.normal_cls.clone() { // Progressively shorten the ordering to search for a satisfied prefix: loop { ordering = match self.try_add_ordering( @@ -844,20 +844,13 @@ mod tests { constants: Vec<&str>, schema: &SchemaRef, ) -> Result { - let orderings = orderings - .iter() - .map(|ordering| { - ordering - .iter() - .map(|name| parse_sort_expr(name, schema)) - .collect::>() - }) - .collect::>(); + let orderings = orderings.iter().map(|ordering| { + ordering.iter().map(|name| parse_sort_expr(name, schema)) + }); let constants = constants .iter() - .map(|col_name| ConstExpr::from(col(col_name, schema).unwrap())) - .collect::>(); + .map(|col_name| ConstExpr::from(col(col_name, schema).unwrap())); let mut props = EquivalenceProperties::new_with_orderings(Arc::clone(schema), orderings); From e7bda2c03cfe0d46b8bd81e07ebe9c9f0b9a7490 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 3 Jun 2025 08:35:11 -0400 Subject: [PATCH 166/167] Minor: remove an unecessary clone in common_sort_prefix_length --- .../physical-expr/src/equivalence/properties/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 0b46129525ff7..6d18d34ca4ded 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -495,7 +495,7 @@ impl EquivalenceProperties { // If the ordering vanishes after normalization, it is satisfied: return Ok(false); }; - if normal_ordering.len() != self.common_sort_prefix_length(normal_ordering)? { + if normal_ordering.len() != self.common_sort_prefix_length(&normal_ordering)? { // If the ordering is unsatisfied, replace existing orderings: self.clear_orderings(); self.add_ordering(ordering_tee); @@ -538,7 +538,7 @@ impl EquivalenceProperties { // If the ordering vanishes after normalization, it is satisfied: return Ok(true); }; - Ok(normal_ordering.len() == self.common_sort_prefix_length(normal_ordering)?) + Ok(normal_ordering.len() == self.common_sort_prefix_length(&normal_ordering)?) } /// Iteratively checks whether the given sort requirement is satisfied by @@ -619,10 +619,10 @@ impl EquivalenceProperties { /// Returns the number of consecutive sort expressions (starting from the /// left) that are satisfied by the existing ordering. - fn common_sort_prefix_length(&self, normal_ordering: LexOrdering) -> Result { + fn common_sort_prefix_length(&self, normal_ordering: &LexOrdering) -> Result { let full_length = normal_ordering.len(); // Check whether the given ordering is satisfied by constraints: - if self.satisfied_by_constraints_ordering(&normal_ordering) { + if self.satisfied_by_constraints_ordering(normal_ordering) { // If constraints satisfy all sort expressions, return the full // length: return Ok(full_length); @@ -661,7 +661,7 @@ impl EquivalenceProperties { // From the analysis above, we know that `[a ASC]` is satisfied. Then, // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - let const_expr = ConstExpr::from(element.expr); + let const_expr = ConstExpr::from(Arc::clone(&element.expr)); eq_properties.add_constants(std::iter::once(const_expr))? } // All sort expressions are satisfied, return full length: @@ -680,7 +680,7 @@ impl EquivalenceProperties { // If the ordering vanishes after normalization, it is satisfied: return Ok((vec![], true)); }; - let prefix_len = self.common_sort_prefix_length(normal_ordering.clone())?; + let prefix_len = self.common_sort_prefix_length(&normal_ordering)?; let flag = prefix_len == normal_ordering.len(); let mut sort_exprs: Vec<_> = normal_ordering.into(); if !flag { From 74d023b218d0abf1e1434718f030ba76decf5670 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 4 Jun 2025 19:29:03 +0300 Subject: [PATCH 167/167] Address reviews --- .../functions-aggregate/src/first_last.rs | 8 ++++ .../physical-expr-common/src/sort_expr.rs | 44 ++++++++++++++----- .../physical-expr/src/equivalence/ordering.rs | 11 ++++- .../src/output_requirements.rs | 2 +- 4 files changed, 53 insertions(+), 12 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index a83d57856be37..42c0a57fbf28b 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -757,6 +757,10 @@ where } } +/// This accumulator is used when there is no ordering specified for the +/// `FIRST_VALUE` aggregation. It simply returns the first value it sees +/// according to the pre-existing ordering of the input data, and provides +/// a fast path for this case without needing to maintain any ordering state. #[derive(Debug)] pub struct TrivialFirstValueAccumulator { first: ScalarValue, @@ -1234,6 +1238,10 @@ impl AggregateUDFImpl for LastValue { } } +/// This accumulator is used when there is no ordering specified for the +/// `LAST_VALUE` aggregation. It simply updates the last value it sees +/// according to the pre-existing ordering of the input data, and provides +/// a fast path for this case without needing to maintain any ordering state. #[derive(Debug)] pub struct TrivialLastValueAccumulator { last: ScalarValue, diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 1192bd58f08ad..07edfb70f4aae 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -154,7 +154,8 @@ impl PhysicalSortExpr { /// Checks whether this sort expression satisfies the given `requirement`. /// If sort options are unspecified in `requirement`, only expressions are - /// compared for inequality. + /// compared for inequality. See [`options_compatible`] for details on + /// how sort options compare with one another. pub fn satisfy( &self, requirement: &PhysicalSortRequirement, @@ -171,6 +172,8 @@ impl PhysicalSortExpr { } /// Checks whether this sort expression satisfies the given `sort_expr`. + /// See [`options_compatible`] for details on how sort options compare with + /// one another. pub fn satisfy_expr(&self, sort_expr: &Self, schema: &Schema) -> bool { self.expr.eq(&sort_expr.expr) && options_compatible( @@ -200,7 +203,10 @@ impl Display for PhysicalSortExpr { } } -/// Returns whether the given two [`SortOptions`] are compatible. +/// Returns whether the given two [`SortOptions`] are compatible. Here, +/// compatibility means that they are either exactly equal, or they differ only +/// in whether NULL values come in first/last, which is immaterial because the +/// column in question is not nullable (specified by the `nullable` parameter). pub fn options_compatible( options_lhs: &SortOptions, options_rhs: &SortOptions, @@ -338,12 +344,22 @@ impl From for PhysicalSortExpr { /// /// For example, a `vec![a ASC, b DESC]` represents a lexicographical ordering /// that first sorts by column `a` in ascending order, then by column `b` in -/// descending order. The ordering is non-degenerate, meaning it contains at -/// least one element, and it is duplicate-free, meaning it does not contain -/// multiple entries for the same column. +/// descending order. +/// +/// # Invariants +/// +/// The following always hold true for a `LexOrdering`: +/// +/// 1. It is non-degenerate, meaning it contains at least one element. +/// 2. It is duplicate-free, meaning it does not contain multiple entries for +/// the same column. #[derive(Debug, Clone, PartialEq, Eq)] pub struct LexOrdering { + /// Vector of sort expressions representing the lexicographical ordering. exprs: Vec, + /// Set of expressions in the lexicographical ordering, used to ensure + /// that the ordering is duplicate-free. Note that the elements in this + /// set are the same underlying physical expressions as in `exprs`. set: HashSet>, } @@ -365,9 +381,7 @@ impl LexOrdering { /// Add all elements from `iter` to the `LexOrdering`. pub fn extend(&mut self, sort_exprs: impl IntoIterator) { for sort_expr in sort_exprs { - if self.set.insert(Arc::clone(&sort_expr.expr)) { - self.exprs.push(sort_expr); - } + self.push(sort_expr); } } @@ -598,7 +612,17 @@ impl From for LexOrdering { } /// Represents a plan's input ordering requirements. Vector elements represent -/// alternative ordering requirements in the order of preference. +/// alternative ordering requirements in the order of preference. The list of +/// alternatives can be either hard or soft, depending on whether the operator +/// can work without an input ordering. +/// +/// # Invariants +/// +/// The following always hold true for a `OrderingRequirements`: +/// +/// 1. It is non-degenerate, meaning it contains at least one ordering. The +/// absence of an input ordering requirement is represented by a `None` value +/// in `ExecutionPlan` APIs, which return an `Option`. #[derive(Debug, Clone, PartialEq)] pub enum OrderingRequirements { /// The operator is not able to work without one of these requirements. @@ -660,7 +684,7 @@ impl OrderingRequirements { /// Returns all alternatives as a vector of `LexRequirement` objects and a /// boolean value indicating softness/hardness of the requirements. - pub fn get_alternatives(self) -> (Vec, bool) { + pub fn into_alternatives(self) -> (Vec, bool) { match self { Self::Hard(alts) => (alts, false), Self::Soft(alts) => (alts, true), diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 0ce7ccab4683d..875c2a76e5eb2 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -44,6 +44,13 @@ use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; /// /// Here, both `[a ASC, b ASC]` and `[c DESC, d ASC]` describe the table /// ordering. In this case, we say that these orderings are equivalent. +/// +/// An `OrderingEquivalenceClass` is a set of such equivalent orderings, which +/// is represented by a vector of `LexOrdering`s. The set does not store any +/// redundant information by enforcing the invariant that no suffix of an +/// ordering in the equivalence class is a prefix of another ordering in the +/// equivalence class. The set can be empty, which means that there are no +/// orderings that describe the table. #[derive(Clone, Debug, Default, Eq, PartialEq)] pub struct OrderingEquivalenceClass { orderings: Vec, @@ -119,7 +126,9 @@ impl OrderingEquivalenceClass { } /// Trims `orderings[idx]` if some suffix of it overlaps with a prefix of - /// `orderings[pre_idx]`. Returns `true` if there is any overlap, `false` otherwise. + /// `orderings[pre_idx]`. If there is any overlap, returns a `Some(true)` + /// if any trimming took place, and `Some(false)` otherwise. If there is + /// no overlap, returns `None`. /// /// For example, if `orderings[idx]` is `[a ASC, b ASC, c DESC]` and /// `orderings[pre_idx]` is `[b ASC, c DESC]`, then the function will trim diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 2a5dc7dbc9ca0..044d27811be67 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -221,7 +221,7 @@ impl ExecutionPlan for OutputRequirementExec { let mut requirements = self.required_input_ordering().swap_remove(0); if let Some(reqs) = requirements { let mut updated_reqs = vec![]; - let (lexes, soft) = reqs.get_alternatives(); + let (lexes, soft) = reqs.into_alternatives(); for lex in lexes.into_iter() { let Some(updated_lex) = update_ordering_requirement(lex, proj_exprs)? else {