diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index 247727e1b484..b2038c432f77 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -22,7 +22,7 @@ use crate::util::{AccessLogOpt, BenchmarkRun, CommonOpt}; use arrow::util::pretty; use datafusion::common::Result; -use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_expr::{LexOrdering, LexOrderingRef, PhysicalSortExpr}; use datafusion::physical_plan::collect; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::prelude::{SessionConfig, SessionContext}; @@ -170,13 +170,13 @@ impl RunOpt { async fn exec_sort( ctx: &SessionContext, - expr: &[PhysicalSortExpr], + expr: LexOrderingRef<'_>, test_file: &TestParquetFile, debug: bool, ) -> Result<(usize, std::time::Duration)> { let start = Instant::now(); let scan = test_file.create_scan(ctx, None).await?; - let exec = Arc::new(SortExec::new(expr.to_owned(), scan)); + let exec = Arc::new(SortExec::new(LexOrdering::new(expr.to_owned()), scan)); let task_ctx = ctx.task_ctx(); let result = collect(exec, task_ctx).await?; let elapsed = start.elapsed(); diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 3ad71be1f447..349c2e438195 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -36,6 +36,7 @@ use datafusion::physical_plan::{ memory::MemoryExec, }; use datafusion::prelude::SessionContext; +use datafusion_physical_expr_common::sort_expr::LexOrdering; // Initialise the operator using the provided record batches and the sort key // as inputs. All record batches must have the same schema. @@ -52,7 +53,7 @@ fn sort_preserving_merge_operator( expr: col(name, &schema).unwrap(), options: Default::default(), }) - .collect::>(); + .collect::(); let exec = MemoryExec::try_new( &batches.into_iter().map(|rb| vec![rb]).collect::>(), diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 99a74b61b3e0..14e80ce364e3 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -89,6 +89,7 @@ use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::StreamExt; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -257,7 +258,7 @@ impl BenchCase { } /// Make sort exprs for each column in `schema` -fn make_sort_exprs(schema: &Schema) -> Vec { +fn make_sort_exprs(schema: &Schema) -> LexOrdering { schema .fields() .iter() diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ea2e098ef14e..15125fe5a090 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1283,13 +1283,16 @@ mod tests { // ok with one column ( vec![vec![col("string_col").sort(true, false)]], - Ok(vec![vec![PhysicalSortExpr { - expr: physical_col("string_col", &schema).unwrap(), - options: SortOptions { - descending: false, - nulls_first: false, - }, - }]]) + Ok(vec![LexOrdering { + inner: vec![PhysicalSortExpr { + expr: physical_col("string_col", &schema).unwrap(), + options: SortOptions { + descending: false, + nulls_first: false, + }, + }], + } + ]) ), // ok with two columns, different options ( @@ -1297,15 +1300,17 @@ mod tests { col("string_col").sort(true, false), col("int_col").sort(false, true), ]], - Ok(vec![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() - ]]) + Ok(vec![LexOrdering { + inner: 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() + ], + } + ]) ), ]; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 0ed53418fe32..ad369b75e130 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -62,7 +62,7 @@ fn create_ordering( for exprs in sort_order { // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = vec![]; + let mut sort_exprs = LexOrdering::default(); for sort in exprs { match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 96c0e452e29e..b32c39435c41 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -34,7 +34,7 @@ use arrow_array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, DataFusionError, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::LexOrdering; use log::warn; @@ -307,7 +307,7 @@ impl FileScanConfig { pub fn split_groups_by_statistics( table_schema: &SchemaRef, file_groups: &[Vec], - sort_order: &[PhysicalSortExpr], + sort_order: &LexOrdering, ) -> Result>> { let flattened_files = file_groups.iter().flatten().collect::>(); // First Fit: @@ -1129,7 +1129,7 @@ mod tests { let result = FileScanConfig::split_groups_by_statistics( &table_schema, &[partitioned_files.clone()], - &sort_order, + &LexOrdering::from(sort_order), ); let results_by_name = result .as_ref() diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 407a3b74f79f..9971e87282a5 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -65,6 +65,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::StreamExt; use log::debug; @@ -328,11 +329,11 @@ impl From for FileMeta { fn get_projected_output_ordering( base_config: &FileScanConfig, projected_schema: &SchemaRef, -) -> Vec> { +) -> Vec { let mut all_orderings = vec![]; for output_ordering in &base_config.output_ordering { - let mut new_ordering = vec![]; - for PhysicalSortExpr { expr, options } in output_ordering { + let mut new_ordering = LexOrdering::default(); + for PhysicalSortExpr { expr, options } in output_ordering.iter() { if let Some(col) = expr.as_any().downcast_ref::() { let name = col.name(); if let Some((idx, _)) = projected_schema.column_with_name(name) { diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index 3ca3ba89f4d9..8e6cfd799fa2 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -26,6 +26,8 @@ use std::sync::Arc; +use crate::datasource::listing::PartitionedFile; + use arrow::{ compute::SortColumn, row::{Row, Rows}, @@ -34,8 +36,7 @@ use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use datafusion_common::{DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; - -use crate::datasource::listing::PartitionedFile; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. /// The min/max values are ordered by [`Self::sort_order`]. @@ -43,13 +44,13 @@ use crate::datasource::listing::PartitionedFile; pub(crate) struct MinMaxStatistics { min_by_sort_order: Rows, max_by_sort_order: Rows, - sort_order: Vec, + sort_order: LexOrdering, } impl MinMaxStatistics { /// Sort order used to sort the statistics #[allow(unused)] - pub fn sort_order(&self) -> &[PhysicalSortExpr] { + pub fn sort_order(&self) -> &LexOrdering { &self.sort_order } @@ -65,8 +66,8 @@ impl MinMaxStatistics { } pub fn new_from_files<'a>( - projected_sort_order: &[PhysicalSortExpr], // Sort order with respect to projected schema - projected_schema: &SchemaRef, // Projected schema + projected_sort_order: &LexOrdering, // Sort order with respect to projected schema + projected_schema: &SchemaRef, // Projected schema projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) files: impl IntoIterator, ) -> Result { @@ -150,7 +151,7 @@ impl MinMaxStatistics { .unzip(); Self::new( - &min_max_sort_order, + &LexOrdering::from(min_max_sort_order), &min_max_schema, RecordBatch::try_new(Arc::clone(&min_max_schema), min_values).map_err( |e| { @@ -166,7 +167,7 @@ impl MinMaxStatistics { } pub fn new( - sort_order: &[PhysicalSortExpr], + sort_order: &LexOrdering, schema: &SchemaRef, min_values: RecordBatch, max_values: RecordBatch, @@ -256,7 +257,7 @@ impl MinMaxStatistics { Ok(Self { min_by_sort_order: min.map_err(|e| e.context("build min rows"))?, max_by_sort_order: max.map_err(|e| e.context("build max rows"))?, - sort_order: sort_order.to_vec(), + sort_order: LexOrdering::from(sort_order.clone()), }) } @@ -277,7 +278,7 @@ impl MinMaxStatistics { } fn sort_columns_from_physical_sort_exprs( - sort_order: &[PhysicalSortExpr], + sort_order: &LexOrdering, ) -> Option> { sort_order .iter() diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index aa4bcb683749..4deb042e84a3 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -52,11 +52,12 @@ use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, }; -use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; -use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; +use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -930,14 +931,14 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { // - 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 { - Arc::new(SortPreservingMergeExec::new( - input.plan.output_ordering().unwrap_or(&[]).to_vec(), - input.plan.clone(), - )) as _ + let ordering = input.plan.output_ordering().cloned(); + + let new_plan = if let Some(ordering) = ordering { + // should_preserve_ordering + Arc::new(SortPreservingMergeExec::new(ordering, input.plan.clone())) as _ } else { + // no ordering to preserve Arc::new(CoalescePartitionsExec::new(input.plan.clone())) as _ }; @@ -1417,7 +1418,6 @@ pub(crate) mod tests { use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, - PhysicalSortRequirement, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::PlanProperties; @@ -1434,7 +1434,7 @@ pub(crate) mod tests { impl SortRequiredExec { fn new_with_requirement( input: Arc, - requirement: Vec, + requirement: LexOrdering, ) -> Self { let cache = Self::compute_properties(&input); Self { @@ -1460,11 +1460,7 @@ pub(crate) mod tests { _t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!( - f, - "SortRequiredExec: [{}]", - PhysicalSortExpr::format_list(&self.expr) - ) + write!(f, "SortRequiredExec: [{}]", self.expr) } } @@ -1494,7 +1490,7 @@ pub(crate) mod tests { if self.expr.is_empty() { vec![None] } else { - vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] + vec![Some(LexRequirement::from(self.expr.clone()))] } } @@ -1539,7 +1535,7 @@ pub(crate) mod tests { /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1555,7 +1551,7 @@ pub(crate) mod tests { /// Created a sorted parquet exec with multiple files fn parquet_exec_multiple_sorted( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1572,7 +1568,7 @@ pub(crate) mod tests { csv_exec_with_sort(vec![]) } - fn csv_exec_with_sort(output_ordering: Vec>) -> Arc { + fn csv_exec_with_sort(output_ordering: Vec) -> Arc { Arc::new( CsvExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1595,9 +1591,7 @@ pub(crate) mod tests { } // Created a sorted parquet exec with multiple files - fn csv_exec_multiple_sorted( - output_ordering: Vec>, - ) -> Arc { + fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { Arc::new( CsvExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1727,7 +1721,7 @@ pub(crate) mod tests { } fn sort_exec( - sort_exprs: Vec, + sort_exprs: LexOrdering, input: Arc, preserve_partitioning: bool, ) -> Arc { @@ -1737,7 +1731,7 @@ pub(crate) mod tests { } fn sort_preserving_merge_exec( - sort_exprs: Vec, + sort_exprs: LexOrdering, input: Arc, ) -> Arc { Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) @@ -3073,7 +3067,7 @@ pub(crate) mod tests { // Only two RepartitionExecs added let expected = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortExec: expr=[b3@1 ASC,a3@0 ASC], preserve_partitioning=[true]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", @@ -3081,7 +3075,7 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b2@1 ASC,a2@0 ASC], preserve_partitioning=[true]", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", @@ -3093,9 +3087,9 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC,a3@0 ASC", + "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b3@1 ASC,a3@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", @@ -3104,9 +3098,9 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC,a2@0 ASC", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b2@1 ASC,a2@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", @@ -3124,10 +3118,10 @@ pub(crate) mod tests { fn merge_does_not_need_sort() -> Result<()> { // see https://github.com/apache/datafusion/issues/4331 let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // Scan some sorted parquet files let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); @@ -3326,10 +3320,10 @@ pub(crate) mod tests { #[test] fn repartition_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); let expected = &[ @@ -3348,10 +3342,10 @@ pub(crate) mod tests { #[test] fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_required_exec_with_req( filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), sort_key, @@ -3427,10 +3421,10 @@ pub(crate) mod tests { fn repartition_through_sort_preserving_merge() -> Result<()> { // sort preserving merge with non-sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); // need resort as the data was not sorted correctly @@ -3448,10 +3442,10 @@ pub(crate) mod tests { fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec( sort_key.clone(), parquet_exec_multiple_sorted(vec![sort_key]), @@ -3480,10 +3474,10 @@ pub(crate) mod tests { 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 = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let plan = sort_preserving_merge_exec(sort_key, input); @@ -3514,10 +3508,10 @@ pub(crate) mod tests { // SortRequired // Parquet(sorted) let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_required_exec_with_req( filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), sort_key, @@ -3549,10 +3543,10 @@ pub(crate) mod tests { // Parquet(unsorted) let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input1 = sort_required_exec_with_req( parquet_exec_with_sort(vec![sort_key.clone()]), sort_key, @@ -3591,10 +3585,10 @@ pub(crate) mod tests { )]; // non sorted input let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("sum", &proj.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec(sort_key, proj); let expected = &[ @@ -3624,10 +3618,10 @@ pub(crate) mod tests { #[test] fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -3657,10 +3651,10 @@ pub(crate) mod tests { #[test] fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -3690,10 +3684,10 @@ pub(crate) mod tests { #[test] fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); let expected = &[ @@ -3724,10 +3718,10 @@ pub(crate) mod tests { #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_exec( sort_key, projection_exec_with_alias( @@ -3794,10 +3788,10 @@ pub(crate) mod tests { #[test] fn parallelization_multiple_files() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); @@ -3958,10 +3952,10 @@ pub(crate) mod tests { #[test] fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); 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)); @@ -3990,10 +3984,10 @@ pub(crate) mod tests { #[test] fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_parquet = limit_exec(filter_exec(sort_exec( sort_key.clone(), parquet_exec(), @@ -4113,10 +4107,10 @@ pub(crate) mod tests { #[test] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // sort preserving merge already sorted input, let plan_parquet = sort_preserving_merge_exec( sort_key.clone(), @@ -4143,10 +4137,10 @@ pub(crate) mod tests { #[test] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // 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]); @@ -4177,10 +4171,10 @@ pub(crate) mod tests { #[test] fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // SortRequired // Parquet(sorted) let plan_parquet = sort_required_exec_with_req( @@ -4211,10 +4205,10 @@ pub(crate) mod tests { fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { // sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -4225,10 +4219,10 @@ pub(crate) mod tests { parquet_exec_with_sort(vec![sort_key]), alias_pairs, ); - let sort_key_after_projection = vec![PhysicalSortExpr { + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); let expected = &[ @@ -4252,10 +4246,10 @@ pub(crate) mod tests { fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { // sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -4265,10 +4259,10 @@ pub(crate) mod tests { let proj_csv = projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = vec![PhysicalSortExpr { + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_csv.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", @@ -4315,10 +4309,10 @@ pub(crate) mod tests { #[test] fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &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)); @@ -4340,10 +4334,10 @@ pub(crate) mod tests { #[test] fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &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)); @@ -4363,10 +4357,10 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + 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)); @@ -4395,10 +4389,10 @@ pub(crate) mod tests { #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &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)); @@ -4419,16 +4413,16 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + 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 = vec![PhysicalSortExpr { + 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 expected = &[ @@ -4457,10 +4451,10 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + 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 physical_plan = filter_exec(input); @@ -4478,10 +4472,10 @@ pub(crate) mod tests { #[test] fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec(); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); let expected = &[ @@ -4515,10 +4509,10 @@ pub(crate) mod tests { #[test] fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + 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_required_exec_with_req(filter_exec(input), sort_key); @@ -4552,10 +4546,10 @@ pub(crate) mod tests { #[test] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); 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); @@ -4575,10 +4569,10 @@ pub(crate) mod tests { #[test] fn do_not_add_unnecessary_hash2() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); 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/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index aa28f9d6b6aa..770e89658386 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -61,13 +61,14 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -220,7 +221,7 @@ 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 = PhysicalSortRequirement::from_sort_exprs(sort_plan.expr()); + let sort_req = LexRequirement::from(sort_plan.expr().clone()); let mut common_prefix_length = 0; while child_eq_properties @@ -231,7 +232,7 @@ fn replace_with_partial_sort( if common_prefix_length > 0 { return Ok(Arc::new( PartialSortExec::new( - sort_plan.expr().to_vec(), + LexOrdering::new(sort_plan.expr().to_vec()), sort_plan.input().clone(), common_prefix_length, ) @@ -274,8 +275,8 @@ fn parallelize_sorts( { // Take the initial sort expressions and requirements let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; - let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs); - let sort_exprs = sort_exprs.to_vec(); + let sort_reqs = LexRequirement::from(sort_exprs.clone()); + let sort_exprs = sort_exprs.clone(); // If there is a connection between a `CoalescePartitionsExec` and a // global sort that satisfy the requirements (i.e. intermediate @@ -390,15 +391,18 @@ fn analyze_immediate_sort_removal( 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.properties().output_ordering().unwrap_or(&[])) - { + if sort_input.equivalence_properties().ordering_satisfy( + &sort_exec + .properties() + .output_ordering() + .cloned() + .unwrap_or_default(), + ) { node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 { // Replace the sort with a sort-preserving merge: - let expr = sort_exec.expr().to_vec(); + let expr = LexOrdering::new(sort_exec.expr().to_vec()); Arc::new(SortPreservingMergeExec::new(expr, sort_input.clone())) as _ } else { // Remove the sort: @@ -619,7 +623,10 @@ fn remove_corresponding_sort_from_sub_plan( // `SortPreservingMergeExec` instead of a `CoalescePartitionsExec`. let plan = node.plan.clone(); let plan = if let Some(ordering) = plan.output_ordering() { - Arc::new(SortPreservingMergeExec::new(ordering.to_vec(), plan)) as _ + Arc::new(SortPreservingMergeExec::new( + LexOrdering::new(ordering.to_vec()), + plan, + )) as _ } else { Arc::new(CoalescePartitionsExec::new(plan)) as _ }; @@ -629,10 +636,10 @@ fn remove_corresponding_sort_from_sub_plan( Ok(node) } -/// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. +/// Converts an [ExecutionPlan] trait object to a [LexOrderingRef] when possible. fn get_sort_exprs( sort_any: &Arc, -) -> Result<(&[PhysicalSortExpr], Option)> { +) -> Result<(&LexOrdering, Option)> { if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { Ok((sort_exec.expr(), sort_exec.fetch())) } else if let Some(spm) = sort_any.as_any().downcast_ref::() @@ -645,7 +652,6 @@ fn get_sort_exprs( #[cfg(test)] mod tests { - use super::*; use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::test_utils::{ @@ -936,8 +942,8 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", @@ -961,10 +967,10 @@ mod tests { let sort = sort_exec(sort_exprs.clone(), source); let spm = sort_preserving_merge_exec(sort_exprs, sort); - let sort_exprs = vec![ + let sort_exprs = LexOrdering::new(vec![ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; + ]); let repartition_exec = repartition_exec(spm); let sort2 = Arc::new( SortExec::new(sort_exprs.clone(), repartition_exec) @@ -979,8 +985,8 @@ mod tests { // it with a `CoalescePartitionsExec` instead of directly removing it. let expected_input = [ "AggregateExec: mode=Final, gby=[], aggr=[]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[true]", + " 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=1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", @@ -1006,7 +1012,7 @@ mod tests { let source2 = repartition_exec(memory_exec(&schema)); let union = union_exec(vec![source1, source2]); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + 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), @@ -1029,7 +1035,7 @@ mod tests { // 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]", @@ -1039,8 +1045,8 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " MemoryExec: 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]", + 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", @@ -1085,8 +1091,11 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new( - SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source) - .with_fetch(Some(2)), + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + ) + .with_fetch(Some(2)), ); let physical_plan = sort_exec( vec![ @@ -1097,12 +1106,12 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1115,26 +1124,29 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ], + ]), source, )); let physical_plan = Arc::new( - SortExec::new(vec![sort_expr("non_nullable_col", &schema)], input) - .with_fetch(Some(2)), + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + input, + ) + .with_fetch(Some(2)), ) as Arc; let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "GlobalLimitExec: skip=0, fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1147,7 +1159,7 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new(SortExec::new( - vec![sort_expr("non_nullable_col", &schema)], + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), source, )); let limit = Arc::new(LocalLimitExec::new(input, 2)); @@ -1160,14 +1172,14 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "LocalLimitExec: fetch=2", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1181,7 +1193,7 @@ mod tests { let source = memory_exec(&schema); // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); let input = Arc::new(SortExec::new( - vec![sort_expr("non_nullable_col", &schema)], + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), source, )); let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; @@ -1253,24 +1265,24 @@ mod tests { let repartition = repartition_exec(union); let physical_plan = sort_preserving_merge_exec(sort_exprs, 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", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // We should keep the bottom `SortExec`. - 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]", + 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", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1288,12 +1300,12 @@ mod tests { let sort = sort_exec(vec![sort_exprs[0].clone()], source); let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1317,7 +1329,7 @@ mod tests { let expected_input = [ "SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ @@ -1409,17 +1421,17 @@ mod tests { // Input is an invalid plan. In this case rule should add required sorting in appropriate places. // First ParquetExec 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", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - 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]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1450,7 +1462,7 @@ mod tests { // 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]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -1490,20 +1502,20 @@ mod tests { // 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]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - 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]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1542,9 +1554,9 @@ mod tests { // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1588,7 +1600,7 @@ mod tests { " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Should adjust the requirement in the third input of the union so @@ -1625,9 +1637,9 @@ mod tests { // 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]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -1676,9 +1688,9 @@ mod tests { // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. let expected_input = ["UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. @@ -1744,10 +1756,10 @@ mod tests { async fn test_window_multi_path_sort2() -> Result<()> { let schema = create_test_schema()?; - let sort_exprs1 = vec![ + let sort_exprs1 = LexOrdering::new(vec![ 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()); @@ -1761,11 +1773,11 @@ mod tests { // 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]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; 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]", @@ -1810,11 +1822,11 @@ mod tests { // Should not change the unnecessarily fine `SortExec`s because there is `LimitExec` let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1822,7 +1834,7 @@ mod tests { " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1867,7 +1879,7 @@ mod tests { let join_plan2 = format!( " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); - 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]", join_plan2.as_str(), " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; @@ -1879,7 +1891,7 @@ mod tests { // can push down the sort requirements and save 1 SortExec vec![ join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", @@ -1888,7 +1900,7 @@ mod tests { _ => { // can not push down the sort requirements vec![ - "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -1938,9 +1950,9 @@ mod tests { ); let spm_plan = match join_type { JoinType::RightAnti => { - "SortPreservingMergeExec: [col_a@0 ASC,col_b@1 ASC]" + "SortPreservingMergeExec: [col_a@0 ASC, col_b@1 ASC]" } - _ => "SortPreservingMergeExec: [col_a@2 ASC,col_b@3 ASC]", + _ => "SortPreservingMergeExec: [col_a@2 ASC, col_b@3 ASC]", }; let join_plan2 = format!( " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" @@ -1956,14 +1968,14 @@ mod tests { join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC,col_b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { // can not push down the sort requirements for Left and Full join. vec![ - "SortExec: expr=[col_a@2 ASC,col_b@3 ASC], preserve_partitioning=[false]", + "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2001,13 +2013,13 @@ mod tests { ]; let physical_plan = sort_preserving_merge_exec(sort_exprs1, 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)]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // 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]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2023,13 +2035,13 @@ mod tests { ]; let physical_plan = sort_preserving_merge_exec(sort_exprs2, 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)]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // 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]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2069,7 +2081,7 @@ mod tests { 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]", " 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]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2124,7 +2136,7 @@ mod tests { let state = session_ctx.state(); let memory_exec = memory_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); let repartition = repartition_exec(window); @@ -2174,7 +2186,7 @@ mod tests { let repartition = repartition_exec(source); let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); let repartition = repartition_exec(coalesce_partitions); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); // Add local sort let sort = Arc::new( SortExec::new(sort_exprs.clone(), repartition) @@ -2332,11 +2344,11 @@ mod tests { let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2360,12 +2372,12 @@ mod tests { spm, ); - let expected_input = ["SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + 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", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], 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]", + 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", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2387,15 +2399,15 @@ mod tests { 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: [a@0 ASC,b@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC,b@1 ASC", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; 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=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -2418,11 +2430,11 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[a@0 ASC,c@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, c@2 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]" ]; let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC,c@2 ASC], common_prefix_length=[1]", + "PartialSortExec: expr=[a@0 ASC, c@2 ASC], common_prefix_length=[1]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2445,12 +2457,12 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[a@0 ASC,c@2 ASC,d@3 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]" ]; // let optimized let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC,c@2 ASC,d@3 ASC], common_prefix_length=[2]", + "PartialSortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], common_prefix_length=[2]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2472,7 +2484,7 @@ mod tests { parquet_input, ); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" ]; let expected_no_change = expected_input; @@ -2495,7 +2507,7 @@ mod tests { unbounded_input, ); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", + "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; @@ -2510,8 +2522,8 @@ mod tests { // SortExec: expr=[a] // MemoryExec let schema = create_test_schema3()?; - let sort_exprs_a = vec![sort_expr("a", &schema)]; - let sort_exprs_b = vec![sort_expr("b", &schema)]; + let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); + let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); let plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) @@ -2540,8 +2552,9 @@ mod tests { // SortExec: expr=[a] // MemoryExec let schema = create_test_schema3()?; - let sort_exprs_a = vec![sort_expr("a", &schema)]; - let sort_exprs_ab = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + 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 plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) @@ -2551,7 +2564,7 @@ mod tests { let plan = sort_exec(sort_exprs_ab, plan); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", @@ -2559,7 +2572,7 @@ mod tests { // should able to push shorts let expected = [ "RequiredInputOrderingExec", - " SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected, plan, true); diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 1c63df1f0281..4a25b3b0582d 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -40,7 +40,8 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{internal_err, JoinSide, JoinType}; use datafusion_expr::sort_properties::SortProperties; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; /// The [`JoinSelection`] rule tries to modify a given plan so that it can @@ -550,7 +551,7 @@ fn hash_join_convert_symmetric_subrule( // the function concludes that no specific order is required for the SymmetricHashJoinExec. This approach // ensures that the symmetric hash join operation only imposes ordering constraints when necessary, // based on the properties of the child nodes and the filter condition. - let determine_order = |side: JoinSide| -> Option> { + let determine_order = |side: JoinSide| -> Option { hash_join .filter() .map(|filter| { @@ -589,7 +590,7 @@ fn hash_join_convert_symmetric_subrule( JoinSide::Left => hash_join.left().output_ordering(), JoinSide::Right => hash_join.right().output_ordering(), } - .map(|p| p.to_vec()) + .map(|p| LexOrdering::new(p.to_vec())) }) .flatten() }; @@ -719,7 +720,6 @@ fn apply_subrules( #[cfg(test)] mod tests_statistical { - use super::*; use crate::{ physical_plan::{displayable, ColumnStatistics, Statistics}, diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index b4dd0a995d5f..584dd7229fab 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::Itertools; @@ -246,7 +246,7 @@ fn try_swapping_with_streaming_table( let mut lex_orderings = vec![]; for lex_ordering in streaming_table.projected_output_ordering().into_iter() { - let mut orderings = vec![]; + let mut orderings = LexOrdering::default(); for order in lex_ordering { let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? else { @@ -467,8 +467,8 @@ fn try_swapping_with_sort( return Ok(None); } - let mut updated_exprs = vec![]; - for sort in sort.expr() { + let mut updated_exprs = LexOrdering::default(); + for sort in sort.expr().iter() { let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { return Ok(None); }; @@ -497,8 +497,8 @@ fn try_swapping_with_sort_preserving_merge( return Ok(None); } - let mut updated_exprs = vec![]; - for sort in spm.expr() { + let mut updated_exprs = LexOrdering::default(); + for sort in spm.expr().iter() { let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? else { return Ok(None); @@ -915,8 +915,14 @@ fn try_swapping_with_sym_hash_join( new_filter, sym_join.join_type(), sym_join.null_equals_null(), - sym_join.right().output_ordering().map(|p| p.to_vec()), - sym_join.left().output_ordering().map(|p| p.to_vec()), + sym_join + .right() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), + sym_join + .left() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), sym_join.partition_mode(), )?))) } @@ -1863,7 +1869,7 @@ mod tests { }) as _], Some(&vec![0_usize, 2, 4, 3]), vec![ - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 2)), options: SortOptions::default(), @@ -1872,11 +1878,11 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), }, - ], - vec![PhysicalSortExpr { + ]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), options: SortOptions::default(), - }], + }]), ] .into_iter(), true, @@ -1923,7 +1929,7 @@ mod tests { assert_eq!( result.projected_output_ordering().into_iter().collect_vec(), vec![ - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 1)), options: SortOptions::default(), @@ -1932,11 +1938,11 @@ mod tests { expr: Arc::new(Column::new("a", 2)), options: SortOptions::default(), }, - ], - vec![PhysicalSortExpr { + ]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 0)), options: SortOptions::default(), - }], + }]), ] ); assert!(result.is_infinite()); @@ -2553,7 +2559,7 @@ mod tests { fn test_sort_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -2566,7 +2572,7 @@ mod tests { )), options: SortOptions::default(), }, - ], + ]), csv.clone(), )); let projection: Arc = Arc::new(ProjectionExec::try_new( @@ -2581,7 +2587,7 @@ mod tests { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[b@1 ASC, c@2 + a@0 ASC], preserve_partitioning=[false]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(initial, expected_initial); @@ -2590,7 +2596,7 @@ mod tests { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b@2 ASC, c@0 + new_a@1 ASC], preserve_partitioning=[false]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; @@ -2603,7 +2609,7 @@ mod tests { fn test_sort_preserving_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -2616,7 +2622,7 @@ mod tests { )), options: SortOptions::default(), }, - ], + ]), csv.clone(), )); let projection: Arc = Arc::new(ProjectionExec::try_new( @@ -2631,7 +2637,7 @@ mod tests { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " SortPreservingMergeExec: [b@1 ASC, c@2 + a@0 ASC]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(initial, expected_initial); @@ -2640,7 +2646,7 @@ mod tests { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "SortPreservingMergeExec: [b@2 ASC,c@0 + new_a@1 ASC]", + "SortPreservingMergeExec: [b@2 ASC, c@0 + new_a@1 ASC]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index a989be987d3d..01168b19d247 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -128,10 +128,10 @@ fn plan_with_order_preserving_variants( return Ok(sort_input); } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { let child = &sort_input.children[0].plan; - if let Some(ordering) = child.output_ordering().map(Vec::from) { + if let Some(ordering) = child.output_ordering() { // When the input of a `CoalescePartitionsExec` has an ordering, // replace it with a `SortPreservingMergeExec` if appropriate: - let spm = SortPreservingMergeExec::new(ordering, child.clone()); + let spm = SortPreservingMergeExec::new(ordering.clone(), child.clone()); sort_input.plan = Arc::new(spm) as _; sort_input.children[0].data = true; return Ok(sort_input); @@ -255,7 +255,13 @@ pub(crate) fn replace_with_order_preserving_variants( if alternate_plan .plan .equivalence_properties() - .ordering_satisfy(requirements.plan.output_ordering().unwrap_or(&[])) + .ordering_satisfy( + &requirements + .plan + .output_ordering() + .cloned() + .unwrap_or_default(), + ) { for child in alternate_plan.children.iter_mut() { child.data = false; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index c7677d725b03..e137ac101490 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -36,10 +36,8 @@ use datafusion_common::{plan_err, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{ - LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, -}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use hashbrown::HashSet; @@ -86,18 +84,15 @@ fn pushdown_sorts_helper( mut requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_reqs = requirements - .data - .ordering_requirement - .as_deref() - .unwrap_or(&[]); + let parent_reqs = requirements.data.ordering_requirement.unwrap_or_default(); let satisfy_parent = plan .equivalence_properties() - .ordering_satisfy_requirement(parent_reqs); + .ordering_satisfy_requirement(&parent_reqs); if is_sort(plan) { let required_ordering = plan .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs) + .cloned() + .map(LexRequirement::from) .unwrap_or_default(); if !satisfy_parent { // Make sure this `SortExec` satisfies parent requirements: @@ -235,7 +230,7 @@ fn pushdown_requirement_to_children( Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required, - &parent_required_expr, + parent_required_expr.as_ref(), JoinSide::Left, ), Some(JoinSide::Right) => { @@ -248,7 +243,7 @@ fn pushdown_requirement_to_children( try_pushdown_requirements_to_join( smj, parent_required, - &new_right_required_expr, + new_right_required_expr.as_ref(), JoinSide::Right, ) } @@ -277,7 +272,7 @@ fn pushdown_requirement_to_children( 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().unwrap_or(&[])) { + if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or_default()) { Ok(None) } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer @@ -344,10 +339,11 @@ fn determine_children_requirement( RequirementsCompatibility::NonCompatible } } + fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirementRef, - sort_expr: &[PhysicalSortExpr], + sort_expr: &LexOrdering, push_side: JoinSide, ) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); @@ -355,13 +351,13 @@ 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().unwrap_or(&[]); - let right_ordering = smj.right().output_ordering().unwrap_or(&[]); + let left_ordering = smj.left().output_ordering().unwrap_or_default(); + let right_ordering = smj.right().output_ordering().unwrap_or_default(); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { let left_eq_properties = left_eq_properties .clone() - .with_reorder(Vec::from(sort_expr)); + .with_reorder(LexOrdering::from_ref(sort_expr)); if left_eq_properties .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { @@ -374,7 +370,7 @@ fn try_pushdown_requirements_to_join( JoinSide::Right => { let right_eq_properties = right_eq_properties .clone() - .with_reorder(Vec::from(sort_expr)); + .with_reorder(LexOrdering::from_ref(sort_expr)); if right_eq_properties .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { @@ -416,7 +412,7 @@ fn try_pushdown_requirements_to_join( } fn expr_source_side( - required_exprs: &[PhysicalSortExpr], + required_exprs: &LexOrdering, join_type: JoinType, left_columns_len: usize, ) -> Option { diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 98f1a7c21a39..403936d3f355 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -57,7 +57,7 @@ use datafusion_physical_plan::{ use async_trait::async_trait; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr_common::sort_expr::{ - LexRequirement, PhysicalSortRequirement, + LexOrdering, LexRequirement, PhysicalSortRequirement, }; async fn register_current_csv( @@ -243,7 +243,7 @@ pub fn bounded_window_exec( sort_exprs: impl IntoIterator, input: Arc, ) -> Arc { - let sort_exprs: Vec<_> = sort_exprs.into_iter().collect(); + let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); Arc::new( @@ -364,7 +364,7 @@ pub fn sort_exec( /// A test [`ExecutionPlan`] whose requirements can be configured. #[derive(Debug)] pub struct RequirementsTestExec { - required_input_ordering: Vec, + required_input_ordering: LexOrdering, maintains_input_order: bool, input: Arc, } @@ -372,7 +372,7 @@ pub struct RequirementsTestExec { impl RequirementsTestExec { pub fn new(input: Arc) -> Self { Self { - required_input_ordering: vec![], + required_input_ordering: LexOrdering::default(), maintains_input_order: true, input, } @@ -381,7 +381,7 @@ impl RequirementsTestExec { /// sets the required input ordering pub fn with_required_input_ordering( mut self, - required_input_ordering: Vec, + required_input_ordering: LexOrdering, ) -> Self { self.required_input_ordering = required_input_ordering; self @@ -419,8 +419,9 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - let requirement = - PhysicalSortRequirement::from_sort_exprs(&self.required_input_ordering); + let requirement = PhysicalSortRequirement::from_sort_exprs( + self.required_input_ordering.as_ref().iter(), + ); vec![Some(requirement)] } diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 26cdd65883e4..de56e231630c 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -27,6 +27,7 @@ use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{ reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; @@ -138,12 +139,10 @@ fn try_convert_aggregate_if_better( aggr_exprs .into_iter() .map(|aggr_expr| { - let aggr_sort_exprs = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_sort_exprs = reverse_order_bys(aggr_sort_exprs); - let aggr_sort_reqs = - PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_sort_exprs); + let aggr_sort_exprs = aggr_expr.order_bys().cloned().unwrap_or_default(); + let reverse_aggr_sort_exprs = reverse_order_bys(&aggr_sort_exprs); + let aggr_sort_reqs = LexRequirement::from(aggr_sort_exprs); + let reverse_aggr_req = LexRequirement::from(reverse_aggr_sort_exprs); // If the aggregate expression benefits from input ordering, and // there is an actual ordering enabling this, try to update the diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 2c0d042281e6..9acd3f67c272 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -27,7 +27,8 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use datafusion_physical_expr::{LexRequirement, PhysicalSortRequirement}; +use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::tree_node::PlanContext; @@ -38,8 +39,8 @@ pub fn add_sort_above( sort_requirements: LexRequirement, fetch: Option, ) -> PlanContext { - let mut sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirements); - sort_expr.retain(|sort_expr| { + let mut sort_expr = LexOrdering::from(sort_requirements); + sort_expr.inner.retain(|sort_expr| { !node .plan .equivalence_properties() diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ffedc2d6b6ef..0e42b66b9f2c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1545,7 +1545,7 @@ type AggregateExprWithOptionalArgs = ( // The filter clause, if any Option>, // Ordering requirements, if any - Option>, + Option, ); /// Create an aggregate expression with a name from a logical expression @@ -1595,12 +1595,12 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( None => None, }; - let ordering_reqs: Vec = - physical_sort_exprs.clone().unwrap_or(vec![]); + 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.to_vec()) + .order_by(ordering_reqs) .schema(Arc::new(physical_input_schema.to_owned())) .alias(name) .with_ignore_nulls(ignore_nulls) @@ -1669,7 +1669,7 @@ pub fn create_physical_sort_exprs( 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 28901b14b5b7..a162ba6819ce 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -39,15 +39,15 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; +use crate::fuzz_cases::aggregation_fuzzer::{ + AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, +}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use tokio::task::JoinSet; -use crate::fuzz_cases::aggregation_fuzzer::{ - AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, -}; - // ======================================================================== // The new aggregation fuzz tests based on [`AggregationFuzzer`] // ======================================================================== @@ -232,7 +232,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 = vec![]; + let mut sort_keys = LexOrdering::default(); for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).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 ef9b5a7f355a..1cb5a8b0fd3a 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -25,6 +25,7 @@ use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::sorts::sort::sort_batch; use rand::{ rngs::{StdRng, ThreadRng}, @@ -140,8 +141,8 @@ impl DatasetGenerator { let col_expr = col(key, schema)?; Ok(PhysicalSortExpr::new_default(col_expr)) }) - .collect::>>()?; - let sorted_batch = sort_batch(&base_batch, &sort_exprs, None)?; + .collect::>()?; + let sorted_batch = sort_batch(&base_batch, sort_exprs.as_ref(), 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 94157e11702c..525baadd14a5 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -25,7 +25,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -62,7 +62,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), table_data_with_properties.clone(), @@ -74,7 +74,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.as_ref()), expected, "{}", err_msg @@ -135,7 +135,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), table_data_with_properties.clone(), @@ -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.as_ref()), (expected | false), "{}", err_msg @@ -311,7 +311,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expr: Arc::clone(expr), options, }) - .collect::>(); + .collect::(); // Check expected result with experimental result. assert_eq!( @@ -322,7 +322,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expected ); assert_eq!( - eq_properties.ordering_satisfy(&required), + eq_properties.ordering_satisfy(required.as_ref()), expected, "{err_msg}" ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index c0c8517a612b..3df3e0348e42 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -25,7 +25,7 @@ use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -173,7 +173,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), projected_batch.clone(), @@ -185,7 +185,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.as_ref()), expected, "{}", err_msg diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index e704fcacc328..82586bd79eda 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -23,7 +23,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -76,7 +76,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" diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index acc45fe0e591..35da8b596380 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -223,7 +223,7 @@ fn add_equal_conditions_test() -> Result<()> { /// If the table remains the same after sorting with the added unique column, it indicates that the table was /// already sorted according to `required_ordering` to begin with. pub fn is_table_same_after_sort( - mut required_ordering: Vec, + mut required_ordering: LexOrdering, batch: RecordBatch, ) -> Result { // Clone the original schema and columns @@ -444,7 +444,7 @@ pub fn generate_table_for_orderings( assert!(!orderings.is_empty()); // Sort the inner vectors by their lengths (longest first) - orderings.sort_by_key(|v| std::cmp::Reverse(v.len())); + orderings.sort_by_key(|v| std::cmp::Reverse(v.inner.len())); let arrays = schema .fields @@ -459,13 +459,13 @@ 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])?; + let sort_columns = get_sort_columns(&batch, orderings[0].as_ref())?; let sort_indices = lexsort_to_indices(&sort_columns, None)?; let mut batch = take_record_batch(&batch, &sort_indices)?; // prune out rows that is invalid according to remaining orderings. for ordering in orderings.iter().skip(1) { - let sort_columns = get_sort_columns(&batch, ordering)?; + let sort_columns = get_sort_columns(&batch, ordering.as_ref())?; // Collect sort options and values into separate vectors. let (sort_options, sort_col_values): (Vec<_>, Vec<_>) = sort_columns @@ -495,7 +495,7 @@ pub fn generate_table_for_orderings( // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], -) -> Vec { +) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { @@ -508,7 +508,7 @@ pub fn convert_to_sort_exprs( // Convert each inner tuple to PhysicalSortExpr pub fn convert_to_orderings( orderings: &[Vec<(&Arc, SortOptions)>], -) -> Vec> { +) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 4eb1070e6c85..4e895920dd3d 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -31,6 +31,7 @@ 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}; #[tokio::test] @@ -107,13 +108,13 @@ async fn run_merge_test(input: Vec>) { .expect("at least one batch"); let schema = first_batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("x", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&input, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 4ba06ef1d2a6..e4acb96f4930 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -30,6 +30,7 @@ use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use rand::Rng; use std::sync::Arc; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; @@ -114,13 +115,13 @@ impl SortTest { .expect("at least one batch"); let schema = first_batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("x", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&input, schema, None).unwrap(); let sort = Arc::new(SortExec::new(sort, Arc::new(exec))); 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 353db8668363..73f4a569954e 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -45,6 +45,7 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; + use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -345,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 = vec![]; + let mut sort_keys = LexOrdering::default(); for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 61b4e32ad6c9..5bfb4d97ed70 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -47,6 +47,7 @@ use test_utils::add_empty_batches; use datafusion::functions_window::row_number::row_number_udwf; use datafusion_functions_window::lead_lag::{lag_udwf, lead_udwf}; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; @@ -251,7 +252,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { ]; let partitionby_exprs = vec![]; - let orderby_exprs = vec![]; + let orderby_exprs = LexOrdering::default(); // Window frame starts with "UNBOUNDED PRECEDING": let start_bound = WindowFrameBound::Preceding(ScalarValue::UInt64(None)); @@ -284,7 +285,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { fn_name.to_string(), &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame), &extended_schema, false, @@ -599,7 +600,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 = vec![]; + let mut orderby_exprs = LexOrdering::default(); for column in &orderby_columns { orderby_exprs.push(PhysicalSortExpr { expr: col(column, &schema)?, @@ -607,27 +608,27 @@ 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 = LexOrdering::new(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 = vec![]; + let mut sort_keys = LexOrdering::default(); for partition_by_expr in &partitionby_exprs { sort_keys.push(PhysicalSortExpr { expr: partition_by_expr.clone(), options: SortOptions::default(), }) } - for order_by_expr in &orderby_exprs { + for order_by_expr in &orderby_exprs.inner { if !sort_keys.contains(order_by_expr) { sort_keys.push(order_by_expr.clone()) } } let concat_input_record = concat_batches(&schema, &input1)?; - let source_sort_keys = vec![ + let source_sort_keys = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: Default::default(), @@ -640,7 +641,7 @@ async fn run_window_test( expr: col("c", &schema)?, options: Default::default(), }, - ]; + ]); let mut exec1 = Arc::new( MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None)? .try_with_sort_information(vec![source_sort_keys.clone()])?, @@ -659,7 +660,7 @@ async fn run_window_test( fn_name.clone(), &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame.clone()), &extended_schema, false, @@ -677,7 +678,7 @@ async fn run_window_test( fn_name, &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame.clone()), &extended_schema, false, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index fc2fb9afb5f9..6817969580da 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -238,15 +238,15 @@ async fn sort_preserving_merge() { // SortPreservingMergeExec (not a Sort which would compete // with the SortPreservingMergeExec for memory) &[ - "+---------------+-----------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+-----------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 |", - "| | MemoryExec: partitions=2, partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", - "| | |", - "+---------------+-----------------------------------------------------------------------------------------------------------+", + "+---------------+------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 |", + "| | MemoryExec: partitions=2, partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+------------------------------------------------------------------------------------------------------------+", ] ) .run() @@ -281,15 +281,15 @@ async fn sort_spill_reservation() { // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC], preserve_partitioning=[false] |", - "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", - "| | |", - "+---------------+--------------------------------------------------------------------------------------------------------+", + "+---------------+---------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+---------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", + "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+---------------------------------------------------------------------------------------------------------+", ] ); @@ -654,7 +654,7 @@ impl Scenario { descending: false, nulls_first: false, }; - let sort_information = vec![vec![ + let sort_information = vec![LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema).unwrap(), options, @@ -663,7 +663,7 @@ impl Scenario { expr: col("b", &schema).unwrap(), options, }, - ]]; + ])]; let table = SortedTableProvider::new(batches, sort_information); Arc::new(table) diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 6859e2f1468c..6910db6285a3 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::{ expressions::{cast, col}, PhysicalExpr, PhysicalSortExpr, }; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::{ limited_distinct_aggregation::LimitedDistinctAggregation, PhysicalOptimizerRule, }; @@ -407,10 +408,10 @@ fn test_has_filter() -> Result<()> { #[test] fn test_has_order_by() -> Result<()> { - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs index 131b887c4ec7..12cd08fb3db3 100644 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ b/datafusion/core/tests/physical_optimizer/test_util.rs @@ -25,11 +25,11 @@ use datafusion::datasource::{ physical_plan::{FileScanConfig, ParquetExec}, }; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index ddf0085b9de4..a230bb028909 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -18,9 +18,8 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_physical_expr_common::{ - physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, -}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; /// [`AccumulatorArgs`] contains information about how an aggregate @@ -53,7 +52,7 @@ pub struct AccumulatorArgs<'a> { /// ``` /// /// If no `ORDER BY` is specified, `ordering_req` will be empty. - pub ordering_req: &'a [PhysicalSortExpr], + pub ordering_req: &'a LexOrdering, /// 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 4fba772d8ddc..e440abe2de69 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::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( @@ -88,7 +88,7 @@ pub fn adjust_output_array(data_type: &DataType, array: ArrayRef) -> Result Vec { @@ -107,7 +107,7 @@ pub fn ordering_fields( } /// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { +pub fn get_sort_options(ordering_req: &LexOrdering) -> Vec { ordering_req.iter().map(|item| item.options).collect() } diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 65956cb8a1de..715e019bf3e6 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -23,15 +23,17 @@ 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 { let schema = Arc::new(Schema::new(vec![Field::new("f", DataType::Int32, true)])); + let empty_ordering = LexOrdering::default(); let accumulator_args = AccumulatorArgs { return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: &empty_ordering, 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 652d447129dc..72664723a2a3 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -23,15 +23,17 @@ 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 { let schema = Arc::new(Schema::new(vec![Field::new("f", data_type.clone(), true)])); + let empty_ordering = LexOrdering::default(); let accumulator_args = AccumulatorArgs { return_type: data_type, schema: &schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: &empty_ordering, 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 b3e04c5584ef..0e5e533a8fbd 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -135,7 +135,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + acc_args.ordering_req.clone(), 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 da3fc62f8c8c..5471c2e0a4d3 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ ExprFunctionExt, Signature, SortExpr, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; create_func!(FirstValue, first_value_udaf); @@ -130,7 +130,7 @@ impl AggregateUDFImpl for FirstValue { FirstValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + acc_args.ordering_req.clone(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -453,7 +453,7 @@ impl AggregateUDFImpl for LastValue { LastValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + acc_args.ordering_req.clone(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -719,10 +719,7 @@ fn filter_states_according_to_is_set( } /// Combines array refs and their corresponding orderings to construct `SortColumn`s. -fn convert_to_sort_cols( - arrs: &[ArrayRef], - sort_exprs: &[PhysicalSortExpr], -) -> Vec { +fn convert_to_sort_cols(arrs: &[ArrayRef], sort_exprs: &LexOrdering) -> Vec { arrs.iter() .zip(sort_exprs.iter()) .map(|(item, sort_expr)| SortColumn { @@ -740,10 +737,18 @@ mod tests { #[test] fn test_first_last_value_value() -> Result<()> { - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + 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)]; @@ -780,14 +785,22 @@ mod tests { .collect::>(); // FirstValueAccumulator - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; first_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = first_accumulator.state()?; - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; first_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = first_accumulator.state()?; @@ -802,22 +815,34 @@ mod tests { ])?); } - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + 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, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = last_accumulator.state()?; - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = last_accumulator.state()?; @@ -832,8 +857,12 @@ mod tests { ])?); } - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.merge_batch(&states)?; let merged_state = last_accumulator.state()?; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 2a1778d8b232..7d66db3af3b2 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -133,7 +133,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &data_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + acc_args.ordering_req.clone(), ) .map(|acc| Box::new(acc) as _) } diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 355d1d5ad2db..06680a4acaae 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -410,6 +410,7 @@ 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] @@ -457,11 +458,12 @@ mod tests { agg2: Arc, schema: &Schema, ) -> Result { + let empty_ordering = LexOrdering::default(); let args1 = AccumulatorArgs { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: &empty_ordering, name: "a", is_distinct: false, is_reversed: false, @@ -472,7 +474,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: &empty_ordering, name: "a", is_distinct: false, is_reversed: false, diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index 45ccb08e52e9..bd699da397d8 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -41,4 +41,5 @@ arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } +itertools = {workspace = true} rand = { workspace = true } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index d825bfe7e264..83b580339f9b 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -17,18 +17,20 @@ //! Sort expressions +use crate::physical_expr::PhysicalExpr; +use std::fmt; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; -use std::ops::Deref; +use std::ops::{Deref, Index, Range, RangeFrom, RangeTo}; use std::sync::Arc; - -use crate::physical_expr::PhysicalExpr; +use std::vec::IntoIter; 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 /// @@ -143,7 +145,7 @@ impl Hash for PhysicalSortExpr { } impl Display for PhysicalSortExpr { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!(f, "{} {}", self.expr, to_str(&self.options)) } } @@ -183,26 +185,6 @@ impl PhysicalSortExpr { .map_or(true, |opts| self.options.descending == opts.descending) } } - - /// Returns a [`Display`]able list of `PhysicalSortExpr`. - pub fn format_list(input: &[PhysicalSortExpr]) -> impl Display + '_ { - struct DisplayableList<'a>(&'a [PhysicalSortExpr]); - impl<'a> Display for DisplayableList<'a> { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - let mut first = true; - for sort_expr in self.0 { - if first { - first = false; - } else { - write!(f, ",")?; - } - write!(f, "{}", sort_expr)?; - } - Ok(()) - } - } - DisplayableList(input) - } } /// Represents sort requirement associated with a plan @@ -260,7 +242,7 @@ impl PartialEq for PhysicalSortRequirement { } impl Display for PhysicalSortRequirement { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let opts_string = self.options.as_ref().map_or("NA", to_str); write!(f, "{} {}", self.expr, opts_string) } @@ -274,7 +256,7 @@ pub fn format_physical_sort_requirement_list( ) -> impl Display + '_ { struct DisplayWrapper<'a>(&'a [PhysicalSortRequirement]); impl<'a> Display for DisplayWrapper<'a> { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { let mut iter = self.0.iter(); write!(f, "[")?; if let Some(expr) = iter.next() { @@ -325,16 +307,12 @@ impl PhysicalSortRequirement { /// use implementing [`ExecutionPlan::required_input_ordering`]. /// /// [`ExecutionPlan::required_input_ordering`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.required_input_ordering + #[deprecated(since = "43.0.0", note = "use LexRequirement::from_lex_ordering")] pub fn from_sort_exprs<'a>( ordering: impl IntoIterator, ) -> LexRequirement { - LexRequirement::new( - ordering - .into_iter() - .cloned() - .map(PhysicalSortRequirement::from) - .collect(), - ) + let ordering = ordering.into_iter().cloned().collect(); + LexRequirement::from_lex_ordering(ordering) } /// Converts an iterator of [`PhysicalSortRequirement`] into a Vec @@ -343,13 +321,12 @@ impl PhysicalSortRequirement { /// This function converts `PhysicalSortRequirement` to `PhysicalSortExpr` /// for each entry in the input. If required ordering is None for an entry /// default ordering `ASC, NULLS LAST` if given (see the `PhysicalSortExpr::from`). + #[deprecated(since = "43.0.0", note = "use LexOrdering::from_lex_requirement")] pub fn to_sort_exprs( requirements: impl IntoIterator, - ) -> Vec { - requirements - .into_iter() - .map(PhysicalSortExpr::from) - .collect() + ) -> LexOrdering { + let requirements = requirements.into_iter().collect(); + LexOrdering::from_lex_requirement(requirements) } } @@ -364,12 +341,173 @@ fn to_str(options: &SortOptions) -> &str { } } -///`LexOrdering` is an alias for the type `Vec`, which represents +///`LexOrdering` contains a `Vec`, which represents /// a lexicographical ordering. -pub type LexOrdering = Vec; +#[derive(Debug, Default, Clone, PartialEq, Eq, Hash)] +pub struct LexOrdering { + pub inner: Vec, +} + +impl LexOrdering { + // Creates a new [`LexOrdering`] from a vector + pub fn new(inner: Vec) -> Self { + Self { inner } + } + + pub fn capacity(&self) -> usize { + self.inner.capacity() + } + + pub fn clear(&mut self) { + self.inner.clear() + } + + pub fn contains(&self, expr: &PhysicalSortExpr) -> bool { + self.inner.contains(expr) + } + + pub fn extend>(&mut self, iter: I) { + self.inner.extend(iter) + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } + + pub fn len(&self) -> usize { + self.inner.len() + } + + pub fn pop(&mut self) -> Option { + self.inner.pop() + } + + pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { + self.inner.push(physical_sort_expr) + } + + pub fn retain(&mut self, f: impl FnMut(&PhysicalSortExpr) -> bool) { + self.inner.retain(f) + } + + pub fn truncate(&mut self, len: usize) { + self.inner.truncate(len) + } + + /// Merge the contents of `other` into `self`, removing duplicates. + pub fn merge(mut self, other: LexOrdering) -> Self { + self.inner = self.inner.into_iter().chain(other).unique().collect(); + self + } + + /// Converts a `LexRequirement` into a `LexOrdering`. + /// + /// This function converts `PhysicalSortRequirement` to `PhysicalSortExpr` + /// for each entry in the input. If required ordering is None for an entry + /// default ordering `ASC, NULLS LAST` if given (see the `PhysicalSortExpr::from`). + pub fn from_lex_requirement(requirements: LexRequirement) -> LexOrdering { + requirements + .into_iter() + .map(PhysicalSortExpr::from) + .collect() + } +} + +impl From> for LexOrdering { + fn from(value: Vec) -> Self { + Self::new(value) + } +} + +impl From for LexOrdering { + fn from(value: LexRequirement) -> Self { + Self::from_lex_requirement(value) + } +} + +impl Deref for LexOrdering { + type Target = [PhysicalSortExpr]; + + fn deref(&self) -> &Self::Target { + self.inner.as_slice() + } +} + +impl Display for LexOrdering { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + let mut first = true; + for sort_expr in &self.inner { + if first { + first = false; + } else { + write!(f, ", ")?; + } + write!(f, "{}", sort_expr)?; + } + Ok(()) + } +} + +impl FromIterator for LexOrdering { + fn from_iter>(iter: T) -> Self { + let mut lex_ordering = LexOrdering::default(); + + for i in iter { + lex_ordering.push(i); + } + + lex_ordering + } +} + +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; + + fn into_iter(self) -> Self::IntoIter { + self.inner.into_iter() + } +} ///`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 @@ -384,6 +522,10 @@ impl LexRequirement { Self { inner } } + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + pub fn iter(&self) -> impl Iterator { self.inner.iter() } @@ -391,6 +533,30 @@ impl LexRequirement { pub fn push(&mut self, physical_sort_requirement: PhysicalSortRequirement) { self.inner.push(physical_sort_requirement) } + + /// Create a new [`LexRequirement`] from a vector of [`PhysicalSortExpr`]s. + /// + /// Returns [`PhysicalSortRequirement`] that requires the exact + /// sort of the [`PhysicalSortExpr`]s in `ordering` + /// + /// This method takes `&'a PhysicalSortExpr` to make it easy to + /// use implementing [`ExecutionPlan::required_input_ordering`]. + /// + /// [`ExecutionPlan::required_input_ordering`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.required_input_ordering + pub fn from_lex_ordering(ordering: LexOrdering) -> Self { + Self::new( + ordering + .into_iter() + .map(PhysicalSortRequirement::from) + .collect(), + ) + } +} + +impl From for LexRequirement { + fn from(value: LexOrdering) -> Self { + Self::from_lex_ordering(value) + } } impl Deref for LexRequirement { @@ -415,7 +581,7 @@ impl FromIterator for LexRequirement { impl IntoIterator for LexRequirement { type Item = PhysicalSortRequirement; - type IntoIter = std::vec::IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.inner.into_iter() diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index d2c9bf1a2408..ffdab6c6d385 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -24,7 +24,7 @@ use datafusion_common::Result; use datafusion_expr_common::sort_properties::ExprProperties; use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::PhysicalSortExpr; +use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; use crate::tree_node::ExprContext; /// Represents a [`PhysicalExpr`] node with associated properties (order and @@ -96,7 +96,7 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { /// 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: &[PhysicalSortExpr]) -> Vec { +pub fn reverse_order_bys(order_bys: &LexOrdering) -> LexOrdering { order_bys .iter() .map(|e| PhysicalSortExpr::new(e.expr.clone(), !e.options)) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 6330c240241a..8bde6af9d7ac 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -45,7 +45,7 @@ use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; 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::LexOrdering; use datafusion_physical_expr_common::utils::reverse_order_bys; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; @@ -81,7 +81,7 @@ impl AggregateExprBuilder { args, alias: None, schema: Arc::new(Schema::empty()), - ordering_req: vec![], + ordering_req: LexOrdering::default(), ignore_nulls: false, is_distinct: false, is_reversed: false, @@ -291,7 +291,7 @@ impl AggregateFunctionExpr { /// 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<&[PhysicalSortExpr]> { + pub fn order_bys(&self) -> Option<&LexOrdering> { if self.ordering_req.is_empty() { return None; } @@ -340,7 +340,7 @@ impl AggregateFunctionExpr { }; AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) - .order_by(self.ordering_req.to_vec()) + .order_by(self.ordering_req.clone()) .schema(Arc::new(self.schema.clone())) .alias(self.name().to_string()) .with_ignore_nulls(self.ignore_nulls) @@ -473,7 +473,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(reverse_ordering_req.to_vec()) + .order_by(reverse_ordering_req) .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) @@ -489,7 +489,7 @@ 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.order_bys().unwrap_or(&[]); + let order_bys = self.order_bys().cloned().unwrap_or_default(); let order_by_exprs = order_bys .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index c1851ddb22b5..dd7bc31e103c 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -21,8 +21,8 @@ use std::sync::Arc; use super::{add_offset_to_expr, collapse_lex_req, ProjectionMapping}; use crate::{ expressions::Column, physical_expr::deduplicate_physical_exprs, - physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, - LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, + physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexRequirement, + LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; @@ -475,13 +475,13 @@ 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: LexOrderingRef) -> LexOrdering { + pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { // Convert sort expressions to sort requirements: - let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + let sort_reqs = LexRequirement::from_lex_ordering(sort_exprs.clone()); // Normalize the requirements: let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); // Convert sort requirements back to sort expressions: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs.inner) + LexOrdering::from_lex_requirement(normalized_sort_reqs) } /// This function applies the `normalize_sort_requirement` function for all diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 95bb93d6ca57..902e53a7f236 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -80,6 +80,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use arrow_schema::{SchemaRef, SortOptions}; use datafusion_common::{plan_datafusion_err, Result}; + use datafusion_physical_expr_common::sort_expr::LexOrdering; pub fn output_schema( mapping: &ProjectionMapping, @@ -184,7 +185,7 @@ mod tests { // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], - ) -> Vec { + ) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { @@ -197,7 +198,7 @@ mod tests { // Convert each inner tuple to PhysicalSortExpr pub fn convert_to_orderings( orderings: &[Vec<(&Arc, SortOptions)>], - ) -> Vec> { + ) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) @@ -207,20 +208,22 @@ mod tests { // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs_owned( in_data: &[(Arc, SortOptions)], - ) -> Vec { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr), - options: *options, - }) - .collect() + ) -> 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> { + ) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d71f3b037fb1..31b21110d008 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use std::vec::IntoIter; use crate::equivalence::add_offset_to_expr; -use crate::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; +use crate::{LexOrdering, PhysicalExpr}; use arrow_schema::SortOptions; /// An `OrderingEquivalenceClass` object keeps track of different alternative @@ -146,7 +146,12 @@ 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(); + let output_ordering = self + .orderings + .iter() + .flat_map(|ordering| ordering.as_ref()) + .cloned() + .collect(); let output_ordering = collapse_lex_ordering(output_ordering); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -169,7 +174,7 @@ impl OrderingEquivalenceClass { for idx in 0..n_ordering { // Calculate cross product index let idx = outer_idx * n_ordering + idx; - self.orderings[idx].extend(ordering.iter().cloned()); + self.orderings[idx].inner.extend(ordering.iter().cloned()); } } self @@ -179,7 +184,7 @@ impl OrderingEquivalenceClass { /// ordering equivalence class. pub fn add_offset(&mut self, offset: usize) { for ordering in self.orderings.iter_mut() { - for sort_expr in ordering { + for sort_expr in ordering.inner.iter_mut() { sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); } } @@ -211,10 +216,10 @@ impl IntoIterator for OrderingEquivalenceClass { /// duplicate entries that have same physical expression inside. For example, /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { - let mut output = Vec::::new(); - for item in input { + let mut output = LexOrdering::default(); + for item in input.iter() { if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); + output.push(item.clone()); } } output @@ -239,10 +244,10 @@ impl Display for OrderingEquivalenceClass { write!(f, "[")?; let mut iter = self.orderings.iter(); if let Some(ordering) = iter.next() { - write!(f, "[{}]", PhysicalSortExpr::format_list(ordering))?; + write!(f, "[{}]", ordering)?; } for ordering in iter { - write!(f, ", [{}]", PhysicalSortExpr::format_list(ordering))?; + write!(f, ", [{}]", ordering)?; } write!(f, "]")?; Ok(()) @@ -268,6 +273,7 @@ mod tests { use arrow_schema::SortOptions; use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; + use datafusion_physical_expr_common::sort_expr::LexOrdering; #[test] fn test_ordering_satisfy() -> Result<()> { @@ -275,11 +281,11 @@ mod tests { Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), ])); - let crude = vec![PhysicalSortExpr { + let crude = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), - }]; - let finer = vec![ + }]); + let finer = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -288,7 +294,7 @@ mod tests { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }, - ]; + ]); // finer ordering satisfies, crude ordering should return true let mut eq_properties_finer = EquivalenceProperties::new(Arc::clone(&input_schema)); diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 9a16b205ae25..5c892aa2dbbb 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -30,9 +30,8 @@ use crate::equivalence::{ }; use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; use crate::{ - physical_exprs_contains, ConstExpr, LexOrdering, LexOrderingRef, LexRequirement, - LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_contains, ConstExpr, LexOrdering, LexRequirement, LexRequirementRef, + PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow_schema::{SchemaRef, SortOptions}; @@ -103,7 +102,7 @@ use itertools::Itertools; /// # use arrow_schema::{Schema, Field, DataType, SchemaRef}; /// # use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; /// # use datafusion_physical_expr::expressions::col; -/// use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +/// use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// # let schema: SchemaRef = Arc::new(Schema::new(vec![ /// # Field::new("a", DataType::Int32, false), /// # Field::new("b", DataType::Int32, false), @@ -116,12 +115,12 @@ 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(vec![ +/// eq_properties.add_new_ordering(LexOrdering::new(vec![ /// 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]") +/// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], const: [b@1]") /// ``` #[derive(Debug, Clone)] pub struct EquivalenceProperties { @@ -185,6 +184,7 @@ impl EquivalenceProperties { let mut output_ordering = self.oeq_class().output_ordering().unwrap_or_default(); // Prune out constant expressions output_ordering + .inner .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -351,7 +351,7 @@ impl EquivalenceProperties { .iter() .filter(|ordering| ordering[0].expr.eq(&normalized_expr)) // First expression after leading ordering - .filter_map(|ordering| Some(ordering).zip(ordering.get(1))) + .filter_map(|ordering| Some(ordering).zip(ordering.inner.get(1))) { let leading_ordering = ordering[0].options; // Currently, we only handle expressions with a single child. @@ -378,7 +378,7 @@ impl EquivalenceProperties { // then we can deduce that ordering `[b ASC]` is also valid. // Hence, ordering `[b ASC]` can be added to the state as valid ordering. // (e.g. existing ordering where leading ordering is removed) - new_orderings.push(ordering[1..].to_vec()); + new_orderings.push(LexOrdering::new(ordering[1..].to_vec())); break; } } @@ -391,7 +391,7 @@ impl EquivalenceProperties { /// Updates the ordering equivalence group within assuming that the table /// is re-sorted according to the argument `sort_exprs`. Note that constants /// and equivalence classes are unchanged as they are unaffected by a re-sort. - pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { + pub fn with_reorder(mut self, sort_exprs: LexOrdering) -> Self { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); self @@ -407,13 +407,13 @@ 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: LexOrderingRef) -> LexOrdering { + fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { // Convert sort expressions to sort requirements: - let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + let sort_reqs = LexRequirement::from_lex_ordering(sort_exprs.clone()); // Normalize the requirements: let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); // Convert sort requirements back to sort expressions: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + normalized_sort_reqs.into() } /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the @@ -455,15 +455,15 @@ impl EquivalenceProperties { /// Checks whether the given ordering is satisfied by any of the existing /// orderings. - pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + pub fn ordering_satisfy(&self, given: &LexOrdering) -> bool { // Convert the given sort expressions to sort requirements: - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); + let sort_requirements = LexRequirement::from(given.clone()); self.ordering_satisfy_requirement(&sort_requirements) } /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + pub fn ordering_satisfy_requirement(&self, reqs: &LexRequirement) -> bool { let mut eq_properties = self.clone(); // First, standardize the given requirement: let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); @@ -547,15 +547,15 @@ impl EquivalenceProperties { /// the latter. pub fn get_finer_ordering( &self, - lhs: LexOrderingRef, - rhs: LexOrderingRef, + lhs: &LexOrdering, + rhs: &LexOrdering, ) -> Option { // Convert the given sort expressions to sort requirements: - let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); - let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); + 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(PhysicalSortRequirement::to_sort_exprs) + finer.map(LexOrdering::from) } /// Returns the finer ordering among the requirements `lhs` and `rhs`, @@ -605,8 +605,8 @@ impl EquivalenceProperties { pub fn substitute_ordering_component( &self, mapping: &ProjectionMapping, - sort_expr: &[PhysicalSortExpr], - ) -> Result>> { + sort_expr: &LexOrdering, + ) -> Result> { let new_orderings = sort_expr .iter() .map(|sort_expr| { @@ -616,7 +616,7 @@ impl EquivalenceProperties { .filter(|source| expr_refers(source, &sort_expr.expr)) .cloned() .collect(); - let mut res = vec![sort_expr.clone()]; + let mut res = LexOrdering::new(vec![sort_expr.clone()]); // TODO: Add one-to-ones analysis for ScalarFunctions. for r_expr in referring_exprs { // we check whether this expression is substitutable or not @@ -639,7 +639,9 @@ impl EquivalenceProperties { // Generate all valid orderings, given substituted expressions. let res = new_orderings .into_iter() + .map(|ordering| ordering.inner) .multi_cartesian_product() + .map(LexOrdering::new) .collect::>(); Ok(res) } @@ -836,7 +838,7 @@ impl EquivalenceProperties { if prefixes.is_empty() { // If prefix is empty, there is no dependency. Insert // empty ordering: - prefixes = vec![vec![]]; + prefixes = vec![LexOrdering::default()]; } // Append current ordering on top its dependencies: for ordering in prefixes.iter_mut() { @@ -986,7 +988,8 @@ impl EquivalenceProperties { // Add new ordered section to the state. result.extend(ordered_exprs); } - result.into_iter().unzip() + let (left, right) = result.into_iter().unzip(); + (LexOrdering::new(left), right) } /// This function determines whether the provided expression is constant @@ -1076,6 +1079,7 @@ impl EquivalenceProperties { let mut new_orderings = vec![]; for ordering in self.oeq_class.orderings { let new_ordering = ordering + .inner .into_iter() .map(|mut sort_expr| { sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; @@ -1313,7 +1317,7 @@ fn construct_prefix_orderings( /// Generates all possible orderings where dependencies are satisfied for the /// current projection expression. /// -/// # Examaple +/// # Example /// If `dependences` is `a + b ASC` and the dependency map holds dependencies /// * `a ASC` --> `[c ASC]` /// * `b ASC` --> `[d DESC]`, @@ -1348,7 +1352,7 @@ fn generate_dependency_orderings( // No dependency, dependent is a leading ordering. if relevant_prefixes.is_empty() { // Return an empty ordering: - return vec![vec![]]; + return vec![LexOrdering::default()]; } relevant_prefixes @@ -1358,7 +1362,12 @@ fn generate_dependency_orderings( prefix_orderings .iter() .permutations(prefix_orderings.len()) - .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) + .map(|prefixes| { + prefixes + .into_iter() + .flat_map(|ordering| ordering.inner.clone()) + .collect() + }) .collect::>() }) .collect() @@ -1651,7 +1660,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![vec![target_sort_expr.clone()]]; + return vec![LexOrdering::new(vec![target_sort_expr.clone()])]; }; node.dependencies @@ -2022,9 +2031,9 @@ impl UnionEquivalentOrderingBuilder { 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(); + let mut augmented_ordering = LexOrdering::default(); + let mut sort_expr_iter = ordering.inner.iter().peekable(); + let mut existing_sort_expr_iter = existing_ordering.inner.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() @@ -2170,20 +2179,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(vec![ + input_properties.add_new_ordering(LexOrdering::new(vec![ 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(vec![ + input_properties.add_new_ordering(LexOrdering::new(vec![ 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![ @@ -2197,7 +2206,7 @@ mod tests { assert_eq!( out_properties.to_string(), - "order: [[a@0 ASC,c@2 ASC,b@1 ASC,d@3 ASC], [a@0 ASC,b@1 ASC,c@2 ASC,d@3 ASC]]" + "order: [[a@0 ASC, c@2 ASC, b@1 ASC, d@3 ASC], [a@0 ASC, b@1 ASC, c@2 ASC, d@3 ASC]]" ); Ok(()) @@ -2403,27 +2412,27 @@ mod tests { eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr)?; let others = vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(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([ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), options: sort_options, - }], + }]), ]); let oeq_class = eq_properties.oeq_class().clone(); @@ -2446,7 +2455,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([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2455,12 +2464,12 @@ 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!( result, - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -2469,7 +2478,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ] + ]) ); let schema = Schema::new(vec![ @@ -2482,11 +2491,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([ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, - }], - vec![ + }]), + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2495,13 +2504,13 @@ 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!( result, - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -2510,7 +2519,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ] + ]) ); let required_columns = [ @@ -2525,7 +2534,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); // not satisfied orders - eq_properties.add_new_orderings([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2538,7 +2547,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]); @@ -2567,14 +2576,14 @@ mod tests { eq_properties.add_equal_conditions(col_b, col_a)?; // [b ASC], [d ASC] eq_properties.add_new_orderings(vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_b), options: option_asc, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, - }], + }]), ]); let test_cases = vec![ @@ -2605,7 +2614,7 @@ mod tests { let leading_orderings = eq_properties .oeq_class() .iter() - .flat_map(|ordering| ordering.first().cloned()) + .flat_map(|ordering| ordering.inner.first().cloned()) .collect::>(); let expr_props = eq_properties.get_expr_properties(Arc::clone(&expr)); let err_msg = format!( @@ -2649,7 +2658,7 @@ mod tests { nulls_first: true, }; // [d ASC, h DESC] also satisfies schema. - eq_properties.add_new_orderings([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, @@ -2658,7 +2667,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)]), @@ -2940,7 +2949,7 @@ mod tests { Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) - .with_reorder( + .with_reorder(LexOrdering::new( ["a", "b", "c"] .into_iter() .map(|c| { @@ -2953,7 +2962,7 @@ mod tests { }) }) .collect::>>()?, - ); + )); struct TestCase { name: &'static str, @@ -3042,7 +3051,7 @@ mod tests { options: SortOptions::default(), }) }) - .collect::>>()?; + .collect::>()?; assert_eq!( properties.ordering_satisfy(&sort), @@ -3564,7 +3573,7 @@ mod tests { ordering .iter() .map(|name| parse_sort_expr(name, schema)) - .collect::>() + .collect::() }) .collect::>(); diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e7c2b4119c5a..8cc770d7280e 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,7 +54,7 @@ pub use physical_expr::{ pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; pub use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, + LexOrdering, LexRequirement, LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 4bd022975ac3..3b9af8888629 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -35,7 +35,6 @@ use datafusion_common::tree_node::{ use datafusion_common::Result; use datafusion_expr::Operator; -use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -244,18 +243,6 @@ pub fn reassign_predicate_columns( .data() } -/// Merge left and right sort expressions, checking for duplicates. -pub fn merge_vectors( - left: &[PhysicalSortExpr], - right: &[PhysicalSortExpr], -) -> Vec { - left.iter() - .cloned() - .chain(right.iter().cloned()) - .unique() - .collect() -} - #[cfg(test)] pub(crate) mod tests { use std::any::Any; diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 3fe5d842dfd1..d8af5c320c94 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -25,16 +25,16 @@ use arrow::array::Array; use arrow::record_batch::RecordBatch; use arrow::{array::ArrayRef, datatypes::Field}; -use datafusion_common::ScalarValue; -use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::{Accumulator, WindowFrame}; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +use crate::{reverse_order_bys, PhysicalExpr}; +use datafusion_common::ScalarValue; +use datafusion_common::{DataFusionError, Result}; +use datafusion_expr::{Accumulator, WindowFrame}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A window expr that takes the form of an aggregate function. /// @@ -43,7 +43,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; pub struct PlainAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -52,13 +52,13 @@ impl PlainAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: order_by.clone(), window_frame, } } @@ -124,7 +124,7 @@ impl WindowExpr for PlainAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { + fn order_by(&self) -> &LexOrdering { &self.order_by } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 8ff277db37df..6c2dd83a9652 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -22,7 +22,6 @@ use std::ops::Range; use std::sync::Arc; use super::{BuiltInWindowFunctionExpr, WindowExpr}; -use crate::expressions::PhysicalSortExpr; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; @@ -34,13 +33,14 @@ 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; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. #[derive(Debug)] pub struct BuiltInWindowExpr { expr: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -49,13 +49,13 @@ impl BuiltInWindowExpr { pub fn new( expr: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { expr, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: order_by.clone(), window_frame, } } @@ -76,7 +76,8 @@ impl BuiltInWindowExpr { if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eq_properties.add_new_orderings([vec![fn_res_ordering]]); + eq_properties + .add_new_orderings([LexOrdering::new(vec![fn_res_ordering])]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -117,7 +118,7 @@ impl WindowExpr for BuiltInWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { + fn order_by(&self) -> &LexOrdering { &self.order_by } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index b889ec8c5d98..9d3dcf87d700 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -25,15 +25,15 @@ use arrow::array::{Array, ArrayRef}; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; -use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::{Accumulator, WindowFrame}; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +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 +43,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; pub struct SlidingAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -52,13 +52,13 @@ impl SlidingAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: order_by.clone(), window_frame, } } @@ -108,7 +108,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { + fn order_by(&self) -> &LexOrdering { &self.order_by } @@ -157,7 +157,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { expr: new_expr, options: req.options, }) - .collect::>(); + .collect::(); Some(Arc::new(SlidingAggregateWindowExpr { aggregate: self .aggregate diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 46c46fab68c5..b1e8a6e3dfe4 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::{LexOrderingRef, PhysicalExpr, PhysicalSortExpr}; +use crate::PhysicalExpr; use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; @@ -34,6 +34,7 @@ use datafusion_expr::window_state::{ }; use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame, WindowFrameBound}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use indexmap::IndexMap; /// Common trait for [window function] implementations @@ -109,7 +110,7 @@ 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) -> &[PhysicalSortExpr]; + fn order_by(&self) -> &LexOrdering; /// Get order by columns, empty if absent fn order_by_columns(&self, batch: &RecordBatch) -> Result> { @@ -344,7 +345,7 @@ pub(crate) fn is_end_bound_safe( window_frame_ctx: &WindowFrameContext, order_bys: &[ArrayRef], most_recent_order_bys: Option<&[ArrayRef]>, - sort_exprs: LexOrderingRef, + sort_exprs: &LexOrdering, idx: usize, ) -> Result { if sort_exprs.is_empty() { diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 4f6f91a2348f..6c8e76bff82b 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_physical_plan::{ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; -use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; +use datafusion_physical_expr::{Distribution, LexRequirement}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; @@ -256,13 +256,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 = PhysicalSortRequirement::from_sort_exprs(req_ordering); + let reqs = LexRequirement::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 = PhysicalSortRequirement::from_sort_exprs(spm.expr()); + let reqs = LexRequirement::from(spm.expr().clone()); Ok(( Arc::new(OutputRequirementExec::new( plan, diff --git a/datafusion/physical-optimizer/src/topk_aggregation.rs b/datafusion/physical-optimizer/src/topk_aggregation.rs index c8a28ed0ec0b..0e5fb82d9e93 100644 --- a/datafusion/physical-optimizer/src/topk_aggregation.rs +++ b/datafusion/physical-optimizer/src/topk_aggregation.rs @@ -25,6 +25,7 @@ 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; @@ -126,7 +127,7 @@ impl TopKAggregation { Ok(Transformed::no(plan)) }; let child = Arc::clone(child).transform_down(closure).data().ok()?; - let sort = SortExec::new(sort.expr().to_vec(), child) + let sort = SortExec::new(LexOrdering::new(sort.expr().to_vec()), 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 48a03af19dbd..01d6213445b7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -937,10 +937,10 @@ fn get_aggregate_expr_req( // 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 vec![]; + return LexOrdering::default(); } - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + let mut req = aggr_expr.order_bys().cloned().unwrap_or_default(); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we @@ -1014,7 +1014,7 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result { - let mut requirement = vec![]; + let mut requirement = LexOrdering::default(); for aggr_expr in aggr_exprs.iter_mut() { if let Some(finer_ordering) = finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) @@ -1074,7 +1074,7 @@ pub fn get_finer_aggregate_exprs_requirement( ); } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) + Ok(LexRequirement::from(requirement)) } /// Returns physical expressions for arguments to evaluate against a batch. @@ -2088,7 +2088,7 @@ mod tests { let args = [col("b", schema)?]; AggregateExprBuilder::new(first_value_udaf(), args.to_vec()) - .order_by(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::new(schema.clone())) .alias(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2106,7 +2106,7 @@ mod tests { }]; let args = [col("b", schema)?]; AggregateExprBuilder::new(last_value_udaf(), args.to_vec()) - .order_by(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::new(schema.clone())) .alias(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2272,7 +2272,7 @@ mod tests { ]), ]; - let common_requirement = vec![ + let common_requirement = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_a), options: options1, @@ -2281,14 +2281,14 @@ mod tests { expr: Arc::clone(col_c), options: options1, }, - ]; + ]); 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(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::clone(&test_schema)) .build() .map(Arc::new) @@ -2302,7 +2302,7 @@ mod tests { &eq_properties, &AggregateMode::Partial, )?; - let res = PhysicalSortRequirement::to_sort_exprs(res); + let res = LexOrdering::from(res); assert_eq!(res, common_requirement); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index accb2fda1131..7d9a50e20ae0 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -19,7 +19,7 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::mem::size_of; mod full; @@ -45,7 +45,7 @@ impl GroupOrdering { pub fn try_new( input_schema: &Schema, mode: &InputOrderMode, - ordering: &[PhysicalSortExpr], + ordering: &LexOrdering, ) -> Result { match mode { InputOrderMode::Linear => Ok(GroupOrdering::None), diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 2dd1ea8a5449..5a05b88798ef 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -21,7 +21,7 @@ use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::mem::size_of; use std::sync::Arc; @@ -107,7 +107,7 @@ impl GroupOrderingPartial { pub fn try_new( input_schema: &Schema, order_indices: &[usize], - ordering: &[PhysicalSortExpr], + ordering: &LexOrdering, ) -> Result { assert!(!order_indices.is_empty()); assert!(order_indices.len() <= ordering.len()); diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7d21cc2f1944..1a2413920e05 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -48,14 +48,14 @@ use datafusion_expr::{EmitTo, GroupsAccumulator}; 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; -use super::order::GroupOrdering; -use super::AggregateExec; - #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) pub(crate) enum ExecutionState { @@ -80,7 +80,7 @@ struct SpillState { // the execution. // ======================================================================== /// Sorting expression for spilling batches - spill_expr: Vec, + spill_expr: LexOrdering, /// Schema for spilling batches spill_schema: SchemaRef, @@ -508,11 +508,8 @@ impl GroupedHashAggregateStream { .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_slice(), - )?; + let group_ordering = + GroupOrdering::try_new(&group_schema, &agg.input_order_mode, &ordering)?; let group_values = new_group_values(group_schema)?; timer.done(); diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index e79b3c817bd1..9f3a76e28577 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -25,7 +25,7 @@ use arrow_schema::SchemaRef; use datafusion_common::display::{GraphvizBuilder, PlanType, StringifiedPlan}; use datafusion_expr::display_schema; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::LexOrdering; use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; @@ -459,23 +459,6 @@ impl<'a> fmt::Display for ProjectSchemaDisplay<'a> { } } -/// A wrapper to customize output ordering display. -#[derive(Debug)] -pub struct OutputOrderingDisplay<'a>(pub &'a [PhysicalSortExpr]); - -impl<'a> fmt::Display for OutputOrderingDisplay<'a> { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - write!(f, "[")?; - for (i, e) in self.0.iter().enumerate() { - if i > 0 { - write!(f, ", ")? - } - write!(f, "{e}")?; - } - write!(f, "]") - } -} - pub fn display_orderings(f: &mut Formatter, orderings: &[LexOrdering]) -> fmt::Result { if let Some(ordering) = orderings.first() { if !ordering.is_empty() { @@ -489,8 +472,8 @@ pub fn display_orderings(f: &mut Formatter, orderings: &[LexOrdering]) -> fmt::R orderings.iter().enumerate().filter(|(_, o)| !o.is_empty()) { match idx { - 0 => write!(f, "{}", OutputOrderingDisplay(ordering))?, - _ => write!(f, ", {}", OutputOrderingDisplay(ordering))?, + 0 => write!(f, "[{}]", ordering)?, + _ => write!(f, ", [{}]", ordering)?, } } let end = if orderings.len() == 1 { "" } else { "]" }; diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index e6484452d43e..7220e7594ea6 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -37,7 +37,7 @@ pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, udf, Distribution, Partitioning, PhysicalExpr, }; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::coalesce_partitions::CoalescePartitionsExec; @@ -443,7 +443,7 @@ pub trait ExecutionPlanProperties { /// For example, `SortExec` (obviously) produces sorted output as does /// `SortPreservingMergeStream`. Less obviously, `Projection` produces sorted /// output if its input is sorted as it does not reorder the input rows. - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; + fn output_ordering(&self) -> Option<&LexOrdering>; /// Get the [`EquivalenceProperties`] within the plan. /// @@ -474,7 +474,7 @@ impl ExecutionPlanProperties for Arc { self.properties().execution_mode() } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + fn output_ordering(&self) -> Option<&LexOrdering> { self.properties().output_ordering() } @@ -492,7 +492,7 @@ impl ExecutionPlanProperties for &dyn ExecutionPlan { self.properties().execution_mode() } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + fn output_ordering(&self) -> Option<&LexOrdering> { self.properties().output_ordering() } @@ -643,8 +643,8 @@ impl PlanProperties { &self.partitioning } - pub fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.output_ordering.as_deref() + pub fn output_ordering(&self) -> Option<&LexOrdering> { + self.output_ordering.as_ref() } pub fn execution_mode(&self) -> ExecutionMode { diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 358ff02473a6..e7c0dd4b80f5 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -858,7 +858,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 rstest::rstest; @@ -888,7 +888,7 @@ pub(crate) mod tests { let mut exec = MemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { - let mut sort_info = Vec::new(); + let mut sort_info = LexOrdering::default(); for name in sorted_column_names { let index = schema.index_of(name).unwrap(); let sort_expr = PhysicalSortExpr { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index b299b495c504..4152d0e746d1 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -49,8 +49,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, PhysicalSortRequirement}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -86,9 +86,9 @@ pub struct SortMergeJoinExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// The left SortExpr - left_sort_exprs: Vec, + left_sort_exprs: LexOrdering, /// The right SortExpr - right_sort_exprs: Vec, + right_sort_exprs: LexOrdering, /// Sort options of join columns used in sorting left and right execution plans pub sort_options: Vec, /// If null_equals_null is true, null == null else null != null @@ -157,8 +157,8 @@ impl SortMergeJoinExec { join_type, schema, metrics: ExecutionPlanMetricsSet::new(), - left_sort_exprs, - right_sort_exprs, + left_sort_exprs: LexOrdering::new(left_sort_exprs), + right_sort_exprs: LexOrdering::new(right_sort_exprs), sort_options, null_equals_null, cache, @@ -292,12 +292,8 @@ impl ExecutionPlan for SortMergeJoinExec { fn required_input_ordering(&self) -> Vec> { vec![ - Some(PhysicalSortRequirement::from_sort_exprs( - &self.left_sort_exprs, - )), - Some(PhysicalSortRequirement::from_sort_exprs( - &self.right_sort_exprs, - )), + Some(LexRequirement::from(self.left_sort_exprs.clone())), + Some(LexRequirement::from(self.right_sort_exprs.clone())), ] } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 02c71dab3df2..4f7ad1835679 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -40,6 +40,7 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::raw::RawTable; use hashbrown::HashSet; @@ -744,8 +745,8 @@ pub fn prepare_sorted_exprs( filter: &JoinFilter, left: &Arc, right: &Arc, - left_sort_exprs: &[PhysicalSortExpr], - right_sort_exprs: &[PhysicalSortExpr], + left_sort_exprs: &LexOrdering, + right_sort_exprs: &LexOrdering, ) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { let err = || { datafusion_common::plan_datafusion_err!("Filter does not include the child order") diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index eb6a30d17e92..103031c83294 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -48,7 +48,6 @@ use crate::joins::utils::{ }; use crate::{ execution_mode_from_children, - expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, @@ -70,10 +69,10 @@ 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, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExprRef; use ahash::RandomState; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; use hashbrown::HashSet; use parking_lot::Mutex; @@ -186,9 +185,9 @@ pub struct SymmetricHashJoinExec { /// If null_equals_null is true, null == null else null != null pub(crate) null_equals_null: bool, /// Left side sort expression(s) - pub(crate) left_sort_exprs: Option>, + pub(crate) left_sort_exprs: Option, /// Right side sort expression(s) - pub(crate) right_sort_exprs: Option>, + pub(crate) right_sort_exprs: Option, /// Partition Mode mode: StreamJoinPartitionMode, /// Cache holding plan properties like equivalences, output partitioning etc. @@ -210,8 +209,8 @@ impl SymmetricHashJoinExec { filter: Option, join_type: &JoinType, null_equals_null: bool, - left_sort_exprs: Option>, - right_sort_exprs: Option>, + left_sort_exprs: Option, + right_sort_exprs: Option, mode: StreamJoinPartitionMode, ) -> Result { let left_schema = left.schema(); @@ -318,13 +317,13 @@ impl SymmetricHashJoinExec { } /// Get left_sort_exprs - pub fn left_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { - self.left_sort_exprs.as_deref() + pub fn left_sort_exprs(&self) -> Option<&LexOrdering> { + self.left_sort_exprs.as_ref() } /// Get right_sort_exprs - pub fn right_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { - self.right_sort_exprs.as_deref() + pub fn right_sort_exprs(&self) -> Option<&LexOrdering> { + self.right_sort_exprs.as_ref() } /// Check if order information covers every column in the filter expression. @@ -416,10 +415,12 @@ impl ExecutionPlan for SymmetricHashJoinExec { vec![ self.left_sort_exprs .as_ref() - .map(PhysicalSortRequirement::from_sort_exprs), + .cloned() + .map(LexRequirement::from), self.right_sort_exprs .as_ref() - .map(PhysicalSortRequirement::from_sort_exprs), + .cloned() + .map(LexRequirement::from), ] } @@ -1626,6 +1627,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 once_cell::sync::Lazy; use rstest::*; @@ -1725,7 +1727,7 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: binary( col("la1", left_schema)?, Operator::Plus, @@ -1733,11 +1735,11 @@ mod tests { left_schema, )?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -1803,14 +1805,14 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -1943,20 +1945,20 @@ 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 = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1_des", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1_des", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2001,20 +2003,20 @@ 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 = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_first", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_first", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2059,20 +2061,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_last", left_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_last", right_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2119,20 +2121,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_desc_null_first", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_desc_null_first", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2180,15 +2182,15 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]; + }]); - let right_sorted = vec![PhysicalSortExpr { + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2238,20 +2240,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let left_sorted = vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: col("la2", left_schema)?, options: SortOptions::default(), - }], + }]), ]; - let right_sorted = vec![PhysicalSortExpr { + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, @@ -2317,20 +2319,20 @@ 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 = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("lt1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("rt1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2398,20 +2400,20 @@ 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 = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("li1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ri1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2472,14 +2474,14 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_float", left_schema)?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_float", right_schema)?, options: SortOptions::default(), - }]; + }]); 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 090d60f0bac3..421fd0da808c 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -101,8 +101,10 @@ pub async fn partitioned_sym_join_with_filter( filter, join_type, null_equals_null, - left.output_ordering().map(|p| p.to_vec()), - right.output_ordering().map(|p| p.to_vec()), + left.output_ordering().map(|p| LexOrdering::new(p.to_vec())), + right + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), StreamJoinPartitionMode::Partitioned, )?; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 090cf9aa628a..d531e0270b5e 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -48,9 +48,9 @@ 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::utils::collect_columns; use datafusion_physical_expr::{ - LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, + LexOrdering, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -448,10 +448,10 @@ pub fn adjust_right_output_partitioning( /// 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 [PhysicalSortExpr], + right_ordering: &mut LexOrdering, ) -> Result<()> { for (left_col, right_col) in on_columns { - for item in right_ordering.iter_mut() { + for item in right_ordering.inner.iter_mut() { let new_expr = Arc::clone(&item.expr) .transform(|e| { if e.eq(right_col) { @@ -468,10 +468,10 @@ fn replace_on_columns_of_right_ordering( } fn offset_ordering( - ordering: LexOrderingRef, + ordering: &LexOrdering, join_type: &JoinType, offset: usize, -) -> Vec { +) -> 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. @@ -482,14 +482,14 @@ fn offset_ordering( options: sort_expr.options, }) .collect(), - _ => ordering.to_vec(), + _ => ordering.clone(), } } /// Calculate the output ordering of a given join operation. pub fn calculate_join_output_ordering( - left_ordering: LexOrderingRef, - right_ordering: LexOrderingRef, + left_ordering: &LexOrdering, + right_ordering: &LexOrdering, join_type: JoinType, on_columns: &[(PhysicalExprRef, PhysicalExprRef)], left_columns_len: usize, @@ -502,15 +502,17 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( on_columns, - &mut right_ordering.to_vec(), + &mut right_ordering.clone(), ) .ok()?; - merge_vectors( - left_ordering, - &offset_ordering(right_ordering, &join_type, left_columns_len), - ) + + left_ordering.clone().merge(offset_ordering( + right_ordering, + &join_type, + left_columns_len, + )) } else { - left_ordering.to_vec() + left_ordering.clone() } } [false, true] => { @@ -518,13 +520,14 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( on_columns, - &mut right_ordering.to_vec(), + &mut right_ordering.clone(), ) .ok()?; - merge_vectors( - &offset_ordering(right_ordering, &join_type, left_columns_len), - left_ordering, - ) + right_ordering.clone().merge(offset_ordering( + right_ordering, + &join_type, + left_columns_len, + )) } else { offset_ordering(right_ordering, &join_type, left_columns_len) } @@ -2586,7 +2589,7 @@ mod tests { #[test] fn test_calculate_join_output_ordering() -> Result<()> { let options = SortOptions::default(); - let left_ordering = vec![ + let left_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options, @@ -2599,8 +2602,8 @@ mod tests { expr: Arc::new(Column::new("d", 3)), options, }, - ]; - let right_ordering = vec![ + ]); + let right_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 2)), options, @@ -2609,7 +2612,7 @@ mod tests { expr: Arc::new(Column::new("y", 1)), options, }, - ]; + ]); let join_type = JoinType::Inner; let on_columns = [( Arc::new(Column::new("b", 1)) as _, @@ -2620,7 +2623,7 @@ mod tests { let probe_sides = [Some(JoinSide::Left), Some(JoinSide::Right)]; let expected = [ - Some(vec![ + Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options, @@ -2641,8 +2644,8 @@ mod tests { expr: Arc::new(Column::new("y", 6)), options, }, - ]), - Some(vec![ + ])), + Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 7)), options, @@ -2663,7 +2666,7 @@ mod tests { expr: Arc::new(Column::new("d", 3)), options, }, - ]), + ])), ]; for (i, (maintains_input_order, probe_side)) in diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index dd4868d1bfcc..56ed144845a0 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -22,7 +22,6 @@ use std::fmt; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::{ common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -79,10 +78,7 @@ impl DisplayAs for MemoryExec { .sort_information .first() .map(|output_ordering| { - format!( - ", output_ordering={}", - PhysicalSortExpr::format_list(output_ordering) - ) + format!(", output_ordering={}", output_ordering) }) .unwrap_or_default(); @@ -216,7 +212,7 @@ impl MemoryExec { let fields = self.schema.fields(); let ambiguous_column = sort_information .iter() - .flatten() + .flat_map(|ordering| ordering.inner.clone()) .flat_map(|expr| collect_columns(&expr.expr)) .find(|col| { fields @@ -365,6 +361,7 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_expr_common::sort_expr::LexOrdering; #[test] fn test_memory_order_eq() -> datafusion_common::Result<()> { @@ -373,7 +370,7 @@ mod tests { Field::new("b", DataType::Int64, false), Field::new("c", DataType::Int64, false), ])); - let sort1 = vec![ + let sort1 = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), @@ -382,12 +379,12 @@ mod tests { expr: col("b", &schema)?, options: SortOptions::default(), }, - ]; - let sort2 = vec![PhysicalSortExpr { + ]); + let sort2 = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema)?, options: SortOptions::default(), - }]; - let mut expected_output_order = vec![]; + }]); + let mut expected_output_order = LexOrdering::default(); expected_output_order.extend(sort1.clone()); expected_output_order.extend(sort2.clone()); @@ -396,8 +393,8 @@ mod tests { .try_with_sort_information(sort_information)?; assert_eq!( - mem_exec.properties().output_ordering().unwrap(), - expected_output_order + mem_exec.properties().output_ordering().unwrap().to_vec(), + expected_output_order.inner ); let eq_properties = mem_exec.properties().equivalence_properties(); assert!(eq_properties.oeq_class().contains(&sort1)); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 601c1e873152..209a5efc1309 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -47,9 +47,10 @@ use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use crate::execution_plan::CardinalityEffect; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use hashbrown::HashMap; @@ -502,11 +503,7 @@ impl DisplayAs for RepartitionExec { } if let Some(sort_exprs) = self.sort_exprs() { - write!( - f, - ", sort_exprs={}", - PhysicalSortExpr::format_list(sort_exprs) - )?; + write!(f, ", sort_exprs={sort_exprs}")?; } Ok(()) } @@ -575,7 +572,7 @@ impl ExecutionPlan for RepartitionExec { let schema_captured = Arc::clone(&schema); // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().unwrap_or(&[]).to_owned(); + let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); @@ -759,7 +756,7 @@ impl RepartitionExec { } /// Return the sort expressions that are used to merge - fn sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { + fn sort_exprs(&self) -> Option<&LexOrdering> { if self.preserve_order { self.input.output_ordering() } else { @@ -1561,10 +1558,10 @@ mod tests { mod test { use arrow_schema::{DataType, Field, Schema, SortOptions}; - use datafusion_physical_expr::expressions::col; - use crate::memory::MemoryExec; use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use super::*; @@ -1659,12 +1656,12 @@ mod test { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } - fn sort_exprs(schema: &Schema) -> Vec { + fn sort_exprs(schema: &Schema) -> LexOrdering { let options = SortOptions::default(); - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("c0", schema).unwrap(), options, - }] + }]) } fn memory_exec(schema: &SchemaRef) -> Arc { @@ -1673,7 +1670,7 @@ mod test { fn sorted_memory_exec( schema: &SchemaRef, - sort_exprs: Vec, + sort_exprs: LexOrdering, ) -> Arc { Arc::new( MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 649c05d52e8b..7164ce8de78a 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -57,7 +57,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; use crate::{ @@ -82,7 +81,7 @@ pub struct PartialSortExec { /// Input schema pub(crate) input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Length of continuous matching columns of input that satisfy /// the required ordering for the sort common_prefix_length: usize, @@ -100,7 +99,7 @@ pub struct PartialSortExec { impl PartialSortExec { /// Create a new partial sort execution plan pub fn new( - expr: Vec, + expr: LexOrdering, input: Arc, common_prefix_length: usize, ) -> Self { @@ -159,7 +158,7 @@ impl PartialSortExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { + pub fn expr(&self) -> &LexOrdering { &self.expr } @@ -212,13 +211,12 @@ impl DisplayAs for PartialSortExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr = PhysicalSortExpr::format_list(&self.expr); let common_prefix_length = self.common_prefix_length; match self.fetch { Some(fetch) => { - write!(f, "PartialSortExec: TopK(fetch={fetch}), expr=[{expr}], common_prefix_length=[{common_prefix_length}]", ) + write!(f, "PartialSortExec: TopK(fetch={fetch}), expr=[{}], common_prefix_length=[{common_prefix_length}]", self.expr) } - None => write!(f, "PartialSortExec: expr=[{expr}], common_prefix_length=[{common_prefix_length}]"), + None => write!(f, "PartialSortExec: expr=[{}], common_prefix_length=[{common_prefix_length}]", self.expr), } } } @@ -315,7 +313,7 @@ struct PartialSortStream { /// The input plan input: SendableRecordBatchStream, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Length of prefix common to input ordering and required ordering of plan /// should be more than 0 otherwise PartialSort is not applicable common_prefix_length: usize, @@ -448,6 +446,7 @@ mod tests { use crate::collect; use crate::expressions::col; + use crate::expressions::PhysicalSortExpr; use crate::memory::MemoryExec; use crate::sorts::sort::SortExec; use crate::test; @@ -475,7 +474,7 @@ mod tests { }; let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -488,7 +487,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&source), 2, )) as Arc; @@ -539,7 +538,7 @@ mod tests { for common_prefix_length in [1, 2] { let partial_sort_exec = Arc::new( PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -552,7 +551,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&source), common_prefix_length, ) @@ -611,7 +610,7 @@ mod tests { [(1, &source_tables[0]), (2, &source_tables[1])] { let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -624,7 +623,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(source), common_prefix_length, )); @@ -701,7 +700,7 @@ mod tests { }; let schema = mem_exec.schema(); let partial_sort_executor = PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -714,7 +713,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&mem_exec), 1, ); @@ -762,7 +761,7 @@ mod tests { (Some(250), vec![0, 125, 125]), ] { let partial_sort_executor = PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -775,7 +774,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&mem_exec), 1, ) @@ -834,10 +833,10 @@ mod tests { )?); let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }], + }]), input, 1, )); @@ -923,7 +922,7 @@ mod tests { )?; let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -936,7 +935,7 @@ mod tests { expr: col("c", &schema)?, options: option_desc, }, - ], + ]), Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), 2, )); @@ -1000,10 +999,10 @@ 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( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, 1, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 921678a4ad92..9e4306987068 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -52,7 +52,7 @@ 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::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::execution_plan::CardinalityEffect; use futures::{StreamExt, TryStreamExt}; @@ -243,7 +243,7 @@ impl ExternalSorter { pub fn new( partition_id: usize, schema: SchemaRef, - expr: Vec, + expr: LexOrdering, batch_size: usize, fetch: Option, sort_spill_reservation_bytes: usize, @@ -265,7 +265,7 @@ impl ExternalSorter { in_mem_batches: vec![], in_mem_batches_sorted: true, spills: vec![], - expr: expr.into(), + expr: expr.inner.into(), metrics, fetch, reservation, @@ -342,10 +342,12 @@ 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(&self.expr) + .with_expressions(&expressions) .with_metrics(self.metrics.baseline.clone()) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -534,10 +536,12 @@ impl ExternalSorter { }) .collect::>()?; + let expressions: LexOrdering = self.expr.iter().cloned().collect(); + StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(&self.expr) + .with_expressions(&expressions) .with_metrics(metrics) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -559,7 +563,7 @@ impl ExternalSorter { let schema = batch.schema(); let fetch = self.fetch; - let expressions = Arc::clone(&self.expr); + let expressions: LexOrdering = self.expr.iter().cloned().collect(); let stream = futures::stream::once(futures::future::lazy(move |_| { let timer = metrics.elapsed_compute().timer(); let sorted = sort_batch(&batch, &expressions, fetch)?; @@ -601,7 +605,7 @@ impl Debug for ExternalSorter { pub fn sort_batch( batch: &RecordBatch, - expressions: &[PhysicalSortExpr], + expressions: &LexOrdering, fetch: Option, ) -> Result { let sort_columns = expressions @@ -678,7 +682,7 @@ pub struct SortExec { /// Input schema pub(crate) input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Containing all metrics set created during sort metrics_set: ExecutionPlanMetricsSet, /// Preserve partitions of input plan. If false, the input partitions @@ -693,7 +697,7 @@ pub struct SortExec { impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. - pub fn new(expr: Vec, input: Arc) -> Self { + pub fn new(expr: LexOrdering, input: Arc) -> Self { let preserve_partitioning = false; let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { @@ -760,7 +764,7 @@ impl SortExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { + pub fn expr(&self) -> &LexOrdering { &self.expr } @@ -788,11 +792,10 @@ impl SortExec { preserve_partitioning: bool, ) -> PlanProperties { // Determine execution mode: - let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()) - .inner - .as_slice(), - ); + let requirement = LexRequirement::from(sort_exprs); + let sort_satisfied = input + .equivalence_properties() + .ordering_satisfy_requirement(&requirement); let mode = match input.execution_mode() { ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, ExecutionMode::Bounded => ExecutionMode::Bounded, @@ -801,6 +804,7 @@ 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() @@ -818,13 +822,12 @@ impl DisplayAs for SortExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr = PhysicalSortExpr::format_list(&self.expr); let preserve_partitioning = self.preserve_partitioning; match self.fetch { Some(fetch) => { - write!(f, "SortExec: TopK(fetch={fetch}), expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]",) + write!(f, "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr) } - None => write!(f, "SortExec: expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]"), + None => write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr), } } } @@ -889,11 +892,7 @@ impl ExecutionPlan for SortExec { let sort_satisfied = self .input .equivalence_properties() - .ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(self.expr.iter()) - .inner - .as_slice(), - ); + .ordering_satisfy_requirement(&LexRequirement::from(self.expr.clone())); match (sort_satisfied, self.fetch.as_ref()) { (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( @@ -1027,9 +1026,9 @@ mod tests { impl SortedUnboundedExec { fn compute_properties(schema: SchemaRef) -> PlanProperties { let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new_default( - Arc::new(Column::new("c1", 0)), - )]]); + eq_properties.add_new_orderings(vec![LexOrdering::new(vec![ + PhysicalSortExpr::new_default(Arc::new(Column::new("c1", 0))), + ])]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) } @@ -1123,10 +1122,10 @@ mod tests { let schema = csv.schema(); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(csv)), )); @@ -1166,10 +1165,10 @@ mod tests { let schema = input.schema(); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(input)), )); @@ -1245,10 +1244,10 @@ mod tests { let sort_exec = Arc::new( SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(csv)), ) .with_fetch(fetch), @@ -1294,10 +1293,10 @@ mod tests { ); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }], + }]), input, )); @@ -1345,7 +1344,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1360,7 +1359,7 @@ mod tests { nulls_first: false, }, }, - ], + ]), Arc::new(MemoryExec::try_new( &[vec![batch]], Arc::clone(&schema), @@ -1435,7 +1434,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1450,7 +1449,7 @@ mod tests { nulls_first: false, }, }, - ], + ]), Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), )); @@ -1514,10 +1513,10 @@ 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( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, )); @@ -1545,10 +1544,10 @@ mod tests { RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options) .unwrap(); - let expressions = vec![PhysicalSortExpr { + let expressions = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), options: SortOptions::default(), - }]; + }]); let result = sort_batch(&batch, &expressions, None).unwrap(); assert_eq!(result.num_rows(), 1); @@ -1564,9 +1563,9 @@ mod tests { cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), }; let mut plan = SortExec::new( - vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))], + )))]), 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 31a4ed61cf9e..9ac0ece4c031 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::expressions::PhysicalSortExpr; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge::StreamingMergeBuilder; @@ -33,9 +32,8 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -75,7 +73,7 @@ pub struct SortPreservingMergeExec { /// Input plan input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Optional number of rows to fetch. Stops producing rows after this fetch @@ -86,7 +84,7 @@ pub struct SortPreservingMergeExec { impl SortPreservingMergeExec { /// Create a new sort execution plan - pub fn new(expr: Vec, input: Arc) -> Self { + pub fn new(expr: LexOrdering, input: Arc) -> Self { let cache = Self::compute_properties(&input, expr.clone()); Self { input, @@ -108,7 +106,7 @@ impl SortPreservingMergeExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { + pub fn expr(&self) -> &LexOrdering { &self.expr } @@ -120,7 +118,7 @@ impl SortPreservingMergeExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( input: &Arc, - ordering: Vec, + ordering: LexOrdering, ) -> PlanProperties { let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_per_partition_constants(); @@ -141,11 +139,7 @@ impl DisplayAs for SortPreservingMergeExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "SortPreservingMergeExec: [{}]", - PhysicalSortExpr::format_list(&self.expr) - )?; + write!(f, "SortPreservingMergeExec: [{}]", self.expr)?; if let Some(fetch) = self.fetch { write!(f, ", fetch={fetch}")?; }; @@ -194,7 +188,7 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn required_input_ordering(&self) -> Vec> { - vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] + vec![Some(LexRequirement::from(self.expr.clone()))] } fn maintains_input_order(&self) -> Vec { @@ -335,6 +329,7 @@ mod tests { 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 tokio::time::timeout; @@ -393,7 +388,7 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a)]).unwrap(); let schema = batch.schema(); - let sort = vec![]; // no sort expressions + let sort = LexOrdering::default(); // no sort expressions let exec = MemoryExec::try_new(&[vec![batch.clone()], vec![batch]], schema, None) .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -572,7 +567,7 @@ mod tests { context: Arc, ) { let schema = partitions[0][0].schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -581,7 +576,7 @@ mod tests { expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]; + ]); let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -591,7 +586,7 @@ mod tests { async fn sorted_merge( input: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let merge = Arc::new(SortPreservingMergeExec::new(sort, input)); @@ -602,7 +597,7 @@ mod tests { async fn partition_sort( input: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let sort_exec = @@ -612,7 +607,7 @@ mod tests { async fn basic_sort( src: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let merge = Arc::new(CoalescePartitionsExec::new(src)); @@ -629,13 +624,13 @@ mod tests { let csv = test::scan_partitioned(partitions); let schema = csv.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let basic = basic_sort(Arc::clone(&csv), sort.clone(), Arc::clone(&task_ctx)).await; @@ -680,7 +675,7 @@ mod tests { } async fn sorted_partitioned_input( - sort: Vec, + sort: LexOrdering, sizes: &[usize], context: Arc, ) -> Result> { @@ -699,10 +694,10 @@ mod tests { async fn test_partition_sort_streaming_input() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: Default::default(), - }]; + }]); let input = sorted_partitioned_input(sort.clone(), &[10, 3, 11], Arc::clone(&task_ctx)) @@ -729,10 +724,10 @@ mod tests { #[tokio::test] async fn test_partition_sort_streaming_input_output() -> Result<()> { let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: Default::default(), - }]; + }]); // Test streaming with default batch size let task_ctx = Arc::new(TaskContext::default()); @@ -804,7 +799,7 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); let schema = b1.schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { @@ -819,7 +814,7 @@ mod tests { nulls_first: false, }, }, - ]; + ]); let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -855,13 +850,13 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new( SortPreservingMergeExec::new(sort, Arc::new(exec)).with_fetch(Some(2)), @@ -891,13 +886,13 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -924,10 +919,10 @@ mod tests { async fn test_async() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let batches = sorted_partitioned_input(sort.clone(), &[5, 7, 3], Arc::clone(&task_ctx)) @@ -963,7 +958,7 @@ mod tests { let merge_stream = StreamingMergeBuilder::new() .with_streams(streams) .with_schema(batches.schema()) - .with_expressions(sort.as_slice()) + .with_expressions(&sort) .with_metrics(BaselineMetrics::new(&metrics, 0)) .with_batch_size(task_ctx.session_config().batch_size()) .with_fetch(fetch) @@ -1003,10 +998,10 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = b1.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), - }]; + }]); let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -1062,10 +1057,10 @@ 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( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, )); @@ -1110,13 +1105,13 @@ mod tests { let schema = partitions[0][0].schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("value", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -1179,7 +1174,7 @@ mod tests { eq_properties.add_new_orderings(vec![columns .iter() .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))) - .collect::>()]); + .collect::()]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::Hash(columns, 3), mode) } @@ -1288,9 +1283,9 @@ mod tests { congestion_cleared: Arc::new(Mutex::new(false)), }; let spm = SortPreservingMergeExec::new( - vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))], + )))]), Arc::new(source), ); let spm_task = SpawnedTask::spawn(collect(Arc::new(spm), task_ctx)); diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index c7924edfb1eb..7c57fdf9baae 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -24,6 +24,7 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::{Fuse, StreamExt}; use std::marker::PhantomData; use std::sync::Arc; @@ -92,7 +93,7 @@ pub struct RowCursorStream { impl RowCursorStream { pub fn try_new( schema: &Schema, - expressions: &[PhysicalSortExpr], + expressions: &LexOrdering, streams: Vec, reservation: MemoryReservation, ) -> Result { diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index ad640d8e8470..7f910053e514 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -23,11 +23,13 @@ use crate::sorts::{ merge::SortPreservingMergeStream, stream::{FieldCursorStream, RowCursorStream}, }; -use crate::{PhysicalSortExpr, SendableRecordBatchStream}; +use crate::SendableRecordBatchStream; use arrow::datatypes::{DataType, SchemaRef}; use arrow_array::*; use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use std::sync::OnceLock; macro_rules! primitive_merge_helper { ($t:ty, $($v:ident),+) => { @@ -49,17 +51,32 @@ macro_rules! merge_helper { }}; } -#[derive(Default)] pub struct StreamingMergeBuilder<'a> { streams: Vec, schema: Option, - expressions: &'a [PhysicalSortExpr], + expressions: &'a LexOrdering, metrics: Option, batch_size: Option, fetch: Option, reservation: Option, } +static EMPTY_ORDER: OnceLock = OnceLock::new(); + +impl<'a> Default for StreamingMergeBuilder<'a> { + fn default() -> Self { + Self { + streams: vec![], + schema: None, + expressions: EMPTY_ORDER.get_or_init(LexOrdering::default), + metrics: None, + batch_size: None, + fetch: None, + reservation: None, + } + } +} + impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { Self::default() @@ -75,7 +92,7 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_expressions(mut self, expressions: &'a [PhysicalSortExpr]) -> Self { + pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { self.expressions = expressions; self } diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 9b46ad2ec7b1..14469ab6c0d9 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -24,6 +24,7 @@ use arrow::{ use std::mem::size_of; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; +use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; use datafusion_common::Result; @@ -32,10 +33,9 @@ use datafusion_execution::{ runtime_env::RuntimeEnv, }; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; -use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; - use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; /// Global TopK @@ -101,7 +101,7 @@ impl TopK { pub fn try_new( partition_id: usize, schema: SchemaRef, - expr: Vec, + expr: LexOrdering, k: usize, batch_size: usize, runtime: Arc, @@ -111,7 +111,7 @@ impl TopK { let reservation = MemoryConsumer::new(format!("TopK[{partition_id}]")) .register(&runtime.memory_pool); - let expr: Arc<[PhysicalSortExpr]> = expr.into(); + let expr: Arc<[PhysicalSortExpr]> = expr.inner.into(); let sort_fields: Vec<_> = expr .iter() diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 433dda870def..69cc63f8f65d 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -607,6 +607,7 @@ mod tests { use datafusion_common::ScalarValue; 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 { @@ -625,14 +626,14 @@ mod tests { // Convert each tuple to PhysicalSortExpr fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], - ) -> Vec { + ) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(*expr), options: *options, }) - .collect::>() + .collect::() } #[tokio::test] 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 6495657339fa..8536a78b9ca5 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::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -60,7 +59,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; @@ -149,7 +148,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, @@ -261,7 +260,7 @@ impl ExecutionPlan for BoundedWindowAggExec { .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys)] + vec![calc_requirements(partition_bys, order_keys.iter())] } fn required_input_distribution(&self) -> Vec { @@ -707,7 +706,7 @@ impl LinearSearch { /// when computing partitions. pub struct SortedSearch { /// Stores partition by columns and their ordering information - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, /// 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 @@ -1160,6 +1159,7 @@ mod tests { use std::time::Duration; use crate::common::collect; + use crate::expressions::PhysicalSortExpr; use crate::memory::MemoryExec; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; @@ -1184,7 +1184,7 @@ mod tests { use datafusion_physical_expr::window::{ BuiltInWindowExpr, BuiltInWindowFunctionExpr, }; - use datafusion_physical_expr::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; + use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; @@ -1286,10 +1286,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 = vec![PhysicalSortExpr { + let orderby_exprs = LexOrdering::new(vec![PhysicalSortExpr { expr: col(order_by, &schema)?, options: SortOptions::default(), - }]; + }]); let window_frame = WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::CurrentRow, @@ -1403,13 +1403,13 @@ mod tests { } fn schema_orders(schema: &SchemaRef) -> Result> { - let orderings = vec![vec![PhysicalSortExpr { + let orderings = vec![LexOrdering::new(vec![PhysicalSortExpr { expr: col("sn", schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]]; + }])]; Ok(orderings) } @@ -1552,7 +1552,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( last_value_func, &[], - &[], + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1563,7 +1563,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func1, &[], - &[], + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1574,7 +1574,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func2, &[], - &[], + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1716,8 +1716,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: [[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]", + "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]", " 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 7ebb7e71ec57..8a00358a5d77 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -98,7 +98,7 @@ pub fn create_window_expr( name: String, args: &[Arc], partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: &LexOrdering, window_frame: Arc, input_schema: &Schema, ignore_nulls: bool, @@ -139,7 +139,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: &[PhysicalSortExpr], + order_by: &LexOrdering, window_frame: Arc, aggregate: Arc, ) -> Arc { @@ -497,7 +497,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: &[PhysicalSortExpr], + orderby_keys: &LexOrdering, input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); @@ -516,9 +516,8 @@ 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 order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); - let reverse_order_by_reqs = - PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); + 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)] { @@ -699,7 +698,7 @@ mod tests { "count".to_owned(), &[col("a", &schema)?], &[], - &[], + &LexOrdering::default(), Arc::new(WindowFrame::new(None)), schema.as_ref(), false, @@ -896,7 +895,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = vec![]; + let mut order_by_exprs = LexOrdering::default(); for col_name in order_by_params { let expr = col(col_name, &test_schema)?; // Give default ordering, this is same with input ordering direction @@ -1058,7 +1057,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = vec![]; + let mut order_by_exprs = LexOrdering::default(); for (col_name, descending, nulls_first) in order_by_params { let expr = col(col_name, &test_schema)?; let options = SortOptions { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index afe9700ed08c..1318f36f269e 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -23,7 +23,6 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -43,7 +42,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::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; /// Window execution plan @@ -105,7 +104,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, @@ -195,13 +194,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)] + 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)] + vec![calc_requirements(partition_bys, order_keys.iter())] } } @@ -282,7 +281,7 @@ pub struct WindowAggStream { batches: Vec, finished: bool, window_expr: Vec>, - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, baseline_metrics: BaselineMetrics, ordered_partition_by_indices: Vec, } @@ -294,7 +293,7 @@ impl WindowAggStream { window_expr: Vec>, input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, 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 20ec5eeaeaf8..316166042fc4 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}; 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, @@ -99,13 +99,13 @@ pub fn parse_physical_sort_exprs( registry: &dyn FunctionRegistry, input_schema: &Schema, codec: &dyn PhysicalExtensionCodec, -) -> Result> { +) -> Result { proto .iter() .map(|sort_expr| { parse_physical_sort_expr(sort_expr, registry, input_schema, codec) }) - .collect::>>() + .collect::>() } /// Parses a physical window expr from a protobuf. @@ -175,7 +175,7 @@ pub fn parse_physical_window_expr( name, &window_node_expr, &partition_by, - &order_by, + order_by.as_ref(), Arc::new(window_frame), &extended_schema, false, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 326c7acab392..e84eae2b9082 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -35,7 +35,7 @@ use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; -use datafusion::physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; +use datafusion::physical_expr::{LexOrdering, PhysicalExprRef, PhysicalSortRequirement}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -501,8 +501,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ExprType::AggregateExpr(agg_node) => { let input_phy_expr: Vec> = agg_node.expr.iter() .map(|e| parse_physical_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; - let ordering_req: Vec = agg_node.ordering_req.iter() - .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; + let ordering_req: LexOrdering = agg_node.ordering_req.iter() + .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)) + .collect::>()?; agg_node.aggregate_function.as_ref().map(|func| { match func { AggregateFunction::UserDefinedAggrFunction(udaf_name) => { @@ -874,7 +875,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) } }) - .collect::, _>>()?; + .collect::>()?; let fetch = if sort.fetch < 0 { None } else { @@ -921,7 +922,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) } }) - .collect::, _>>()?; + .collect::>()?; let fetch = if sort.fetch < 0 { None } else { @@ -1036,7 +1037,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| PhysicalSortRequirement::from_sort_exprs(&item.inner)) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1066,7 +1067,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| PhysicalSortRequirement::from_sort_exprs(&item.inner)) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1103,7 +1104,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| { + PhysicalSortRequirement::from_sort_exprs(&item.inner) + }) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 89a2403922e9..4bf7e353326e 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; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; -use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, Literal, NegativeExpr, NotExpr, NthValue, TryCastExpr, @@ -52,7 +52,10 @@ pub fn serialize_physical_aggr_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; - let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); + let ordering_req = match aggr_expr.order_bys() { + Some(order) => LexOrdering::from_ref(order), + None => LexOrdering::default(), + }; let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; let name = aggr_expr.fun().name().to_string(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 4a9bf6afb49e..1e078ee410c6 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -52,7 +52,8 @@ use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{ - LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, + LexOrdering, LexOrderingRef, LexRequirement, PhysicalSortRequirement, + ScalarFunctionExpr, }; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -307,7 +308,7 @@ fn roundtrip_window() -> Result<()> { .build() .map(Arc::new)?, &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), )); @@ -327,7 +328,7 @@ fn roundtrip_window() -> Result<()> { let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( sum_expr, &[], - &[], + LexOrderingRef::default(), Arc::new(window_frame), )); @@ -459,13 +460,13 @@ fn rountrip_aggregate_with_sort() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let sort_exprs = vec![PhysicalSortExpr { + let sort_exprs = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let aggregates = vec![ @@ -585,7 +586,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 = vec![ + let sort_exprs = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -600,7 +601,7 @@ fn roundtrip_sort() -> Result<()> { nulls_first: true, }, }, - ]; + ]); roundtrip_test(Arc::new(SortExec::new( sort_exprs, Arc::new(EmptyExec::new(schema)), @@ -612,7 +613,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 = vec![ + let sort_exprs = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -627,7 +628,7 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { nulls_first: true, }, }, - ]; + ]); roundtrip_test(Arc::new(SortExec::new( sort_exprs.clone(), @@ -1013,7 +1014,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), &[col("author", &schema)?], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), ))], filter, @@ -1074,7 +1075,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr, &[col("author", &schema)?], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), ))], filter, @@ -1298,17 +1299,17 @@ fn roundtrip_sym_hash_join() -> Result<()> { ] { for left_order in &[ None, - Some(vec![PhysicalSortExpr { + Some(LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_left.index_of("col")?)), options: Default::default(), - }]), + }])), ] { for right_order in &[ None, - Some(vec![PhysicalSortExpr { + Some(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( diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index f03c3700ab9f..4e075878b31b 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -146,7 +146,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] -04)------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST], preserve_partitioning=[true] +04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 60569803322c..53ca8d81b9e4 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -158,7 +158,7 @@ logical_plan 07)--------Filter: balances.time < Int64(10) 08)----------TableScan: balances physical_plan -01)SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false 03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true 04)----CoalescePartitionsExec diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index 176331f570b0..e802ddfe6524 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -320,6 +320,41 @@ ORDER BY 2023-12-20T01:30:00 1000 f1 32.0 2023-12-20T01:30:00 1000 f2 foo +# Window Functions +query TTTT +SELECT "tag_id", "type", + lead("type") OVER (PARTITION BY "tag_id" ORDER BY "time") as "next_type", + lag("type") OVER (PARTITION BY "tag_id" ORDER BY "time") as "last_type" +FROM m2; +---- +1000 active active NULL +1000 active active active +1000 active active active +1000 active active active +1000 active active active +1000 active passive active +1000 passive passive active +1000 passive passive passive +1000 passive passive passive +1000 passive NULL passive + +query TTTT +SELECT "tag_id", "type", + lead("type") OVER (PARTITION BY "tag_id" ORDER BY "time" RANGE BETWEEN 2 PRECEDING AND CURRENT ROW) as "next_type", + lag("type") OVER (PARTITION BY "tag_id" ORDER BY "time") as "last_type" +FROM m2; +---- +1000 active active NULL +1000 active active active +1000 active active active +1000 active active active +1000 active active active +1000 active passive active +1000 passive passive active +1000 passive passive passive +1000 passive passive passive +1000 passive NULL passive + # Cleanup statement ok drop view m1; @@ -379,7 +414,6 @@ GROUP BY column3; ---- 1 2 - # Cleanup statement ok drop table m3; @@ -444,3 +478,14 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 03)----MemoryExec: partitions=1, partition_sizes=[1] + +## Test that filtering on dictionary columns coerces the filter value to the dictionary type +statement error DataFusion error: SQL error: ParserError\("Expected: an SQL statement, found: \#"\) +create table test as values + ('row1', arrow_cast('1', 'Dictionary(Int32, Utf8)')), + ('row2', arrow_cast('2', 'Dictionary(Int32, Utf8)')), + ('row3', arrow_cast('3', 'Dictionary(Int32, Utf8)')) +; +# Cleanup +statement ok +drop table test; diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index 3b4deae3326f..d96044fda8c0 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -37,7 +37,7 @@ logical_plan 02)--Filter: data.ticker = Utf8("A") 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [date@0 ASC NULLS LAST,time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -105,7 +105,7 @@ logical_plan 02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [time@2 ASC NULLS LAST,date@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -146,7 +146,7 @@ logical_plan 02)--Filter: data.date = Date32("2006-01-02") 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST,time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 61b3ad73cd0a..daf270190870 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2250,7 +2250,7 @@ logical_plan 01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -01)PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +01)PartialSortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] 02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT @@ -2263,7 +2263,7 @@ logical_plan 01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] 02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT @@ -2275,7 +2275,7 @@ logical_plan 01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST 02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan -01)SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], preserve_partitioning=[false] 02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true query TT @@ -2288,7 +2288,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a, b, d] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], ordering_mode=Sorted -02)--PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] +02)--PartialSortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, d@2 ASC NULLS LAST], common_prefix_length=[2] 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] # as can be seen in the result below d is indeed ordered. @@ -2535,7 +2535,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted -03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST,amount@1 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2573,7 +2573,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, sum(s.amount)@3 as sum1] 02)--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=PartiallySorted([0]) -03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST,amount@2 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST, amount@2 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort @@ -2646,7 +2646,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted -03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST,amount@1 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -4328,7 +4328,7 @@ logical_plan 02)--Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }"), unbounded_csv_with_timestamps2.ts) AS time_chunks 03)----TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] physical_plan -01)SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 +01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 230ea4d98fc3..804612287246 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -69,7 +69,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -129,7 +129,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -180,7 +180,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index c40f62c3ba80..35decd728eed 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -126,7 +126,7 @@ logical_plan 03)----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) -02)--SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC], preserve_partitioning=[false] +02)--SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] 03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] 04)------ValuesExec @@ -358,7 +358,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -419,7 +419,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index c56c59b1bd78..cf897d628da5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -94,7 +94,7 @@ logical_plan 07)--------Filter: annotated_data.d = Int32(3) 08)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan -01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 +01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index bc40f845cc8a..93bb1f1f548e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3243,13 +3243,13 @@ physical_plan 01)SortPreservingMergeExec: [rn1@5 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 +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)--------------CsvExec: 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], 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 +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 11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 12)----------CsvExec: 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], has_header=true @@ -3277,11 +3277,11 @@ physical_plan 01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] 02)--SortMergeJoin: join_type=Right, 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 +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 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CsvExec: 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], has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 -08)------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 +08)------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 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)----------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] 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] @@ -3315,8 +3315,8 @@ logical_plan 09)--------WindowAggr: windowExpr=[[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 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] +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 diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index d41b78dcd3f2..abf48fac5364 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -44,7 +44,7 @@ logical_plan 02)--Projection: CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST, b@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -60,7 +60,7 @@ logical_plan 02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -81,7 +81,7 @@ logical_plan 02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST,b@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -132,8 +132,8 @@ logical_plan 02)--Projection: CAST(multiple_ordered_table.a AS Utf8) AS a_str, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] -02)--SortExec: expr=[a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 6cc7ee0403f2..4368b9a535d2 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -165,7 +165,7 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortExec: expr=[c2@1 ASC NULLS LAST,c3@2 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] 03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true query II @@ -683,7 +683,7 @@ logical_plan 01)Sort: t1.id DESC NULLS FIRST, t1.name ASC NULLS LAST 02)--TableScan: t1 projection=[id, name] physical_plan -01)SortExec: expr=[id@0 DESC,name@1 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[id@0 DESC, name@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--MemoryExec: partitions=1, partition_sizes=[1] query IT @@ -702,7 +702,7 @@ logical_plan 01)Sort: t1.id ASC NULLS LAST, t1.name ASC NULLS LAST 02)--TableScan: t1 projection=[id, name] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST,name@1 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[id@0 ASC NULLS LAST, name@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--MemoryExec: partitions=1, partition_sizes=[1] @@ -768,8 +768,8 @@ logical_plan 13)------------Projection: column1 AS t 14)--------------Values: (Int64(0)), (Int64(1)) physical_plan -01)SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] -02)--SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [m@0 ASC NULLS LAST, t@1 ASC NULLS LAST] +02)--SortExec: expr=[m@0 ASC NULLS LAST, t@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----InterleaveExec 04)------ProjectionExec: expr=[0 as m, t@0 as t] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] @@ -1229,12 +1229,12 @@ logical_plan 09)----------TableScan: ordered_table projection=[a0, b, c, d] physical_plan 01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] -02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], fetch=2 +02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], fetch=2 03)----UnionExec -04)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 05)--------ProjectionExec: expr=[b@1 as b, c@2 as c, a@0 as a, NULL as a0, d@3 as d] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true -07)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] 09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index bf68a1851137..8fda0599c278 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -89,8 +89,8 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] # Tear down test_table: @@ -119,7 +119,7 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] # Add another file to the directory underlying test_table @@ -141,8 +141,8 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index f2ab4135aaa7..30c54bd7e274 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1439,7 +1439,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1481,7 +1481,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1502,7 +1502,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -01)SortPreservingMergeExec: [a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1553,7 +1553,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(*)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] -07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], has_header=true # FilterExec can track equality of non-column expressions. diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index e4360a9269ca..a3717dd838d6 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -65,8 +65,8 @@ logical_plan 05)--------TableScan: sink_table projection=[c1, c2, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST,c9@3 ASC NULLS LAST], preserve_partitioning=[false] -03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC,c3@2 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[false] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], has_header=true @@ -98,7 +98,7 @@ logical_plan 07)------------TableScan: sink_table projection=[c1, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, r@1 as r] -02)--SortExec: TopK(fetch=2), expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST,c9@3 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: TopK(fetch=2), expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Utf8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] @@ -118,8 +118,8 @@ logical_plan 07)------------TableScan: sink_table projection=[c1, c2, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST,c3@2 DESC,c9@3 ASC NULLS LAST] -03)----SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 DESC,c9@3 ASC NULLS LAST], preserve_partitioning=[true] +02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST] +03)----SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@1 as c1, first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@2 as c2, first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@3 as c3, first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@4 as c9] 05)--------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 06)----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 8cfd25d26c07..4d4323e93e9e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -47,8 +47,8 @@ logical_plan 05)--------Filter: lineitem.l_shipdate <= Date32("1998-09-02") 06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] physical_plan -01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] -02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] +02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index bb32fb209700..2a9fb12a31c2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -53,8 +53,8 @@ logical_plan 11)------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") 12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -01)SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] 04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 8058371764f2..6b2c2f7fdc3e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -65,8 +65,8 @@ logical_plan 13)--------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") 14)----------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] physical_plan -01)SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] 04)------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index e78b0d87f651..c80352c5d36a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -67,8 +67,8 @@ logical_plan 14)------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[sum(lineitem.l_quantity)]] 15)--------------TableScan: lineitem projection=[l_orderkey, l_quantity] physical_plan -01)SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] -02)--SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] +02)--SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 17f3b78a089d..23ffa0d226b8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -99,8 +99,8 @@ logical_plan 34)------------------Filter: region.r_name = Utf8("EUROPE") 35)--------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -01)SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 5cf069ec7248..93dcd4c68052 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -90,8 +90,8 @@ logical_plan 30)----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate 31)------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -01)SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] -02)--SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] 04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 16a1c2b6ebb1..289e9c7732bb 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -58,8 +58,8 @@ logical_plan 14)------------Filter: lineitem.l_shipdate > Date32("1995-03-15") 15)--------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("1995-03-15")] physical_plan -01)SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 5a6cef5311d4..a16af4710478 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -84,8 +84,8 @@ logical_plan 24)--------------Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") 25)----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE")] physical_plan -01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] -02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] +02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] 04)------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index b3631f07cc8f..c4910beb842b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -75,8 +75,8 @@ logical_plan 21)----------------TableScan: orders projection=[o_orderkey, o_orderdate] 22)------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 4a2d9e1d6864..ce251d201f0b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -361,7 +361,7 @@ physical_plan 03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[max(d.a), max(d.seq)], ordering_mode=Sorted 04)------ProjectionExec: expr=[row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] 05)--------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[b@1 ASC NULLS LAST, a@0 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 09)----------------UnionExec @@ -1244,7 +1244,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----ProjectionExec: expr=[c9@1 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c8@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true @@ -1265,7 +1265,7 @@ physical_plan 02)--WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, 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 }] 03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -1288,9 +1288,9 @@ physical_plan 02)--ProjectionExec: expr=[c2@0 as c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, 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 }] 04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # test_window_partition_by_order_by @@ -1312,12 +1312,12 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -03)----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=4096 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 07)------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------CoalesceBatchesExec: target_batch_size=4096 10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1470,10 +1470,10 @@ physical_plan 01)ProjectionExec: expr=[c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -04)------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: expr=[c9@2 ASC NULLS LAST, c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c9@2 DESC,c1@0 DESC], preserve_partitioning=[false] +07)------------SortExec: expr=[c9@2 DESC, c1@0 DESC], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII @@ -1554,17 +1554,17 @@ physical_plan 03)----WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +06)----------SortExec: expr=[c3@2 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 07)------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC], preserve_partitioning=[false] +08)--------------SortExec: expr=[c3@2 ASC NULLS LAST, c1@0 ASC], preserve_partitioning=[false] 09)----------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC], preserve_partitioning=[false] +10)------------------SortExec: expr=[c3@2 ASC NULLS LAST, c1@0 DESC], preserve_partitioning=[false] 11)--------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] 12)----------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 13)------------------------SortExec: expr=[c3@2 DESC NULLS LAST], preserve_partitioning=[false] 14)--------------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 15)----------------------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -16)------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST], preserve_partitioning=[false] +16)------------------------------SortExec: expr=[c3@2 DESC, c1@0 ASC NULLS LAST], preserve_partitioning=[false] 17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII @@ -1639,7 +1639,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true @@ -1683,7 +1683,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III @@ -1730,8 +1730,8 @@ physical_plan 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, c3@2 as c3, c9@3 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -07)------------SortExec: expr=[__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST] +07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true @@ -1823,13 +1823,13 @@ physical_plan 01)SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[c3@0 as c3, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -04)------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[true] +04)------SortExec: expr=[c3@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 08)--------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 09)----------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true @@ -1991,7 +1991,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, rn1@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() PARTITION BY [aggregate_test_100.c1] 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] 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -2023,7 +2023,7 @@ physical_plan 04)------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] 05)--------SortExec: expr=[c9@1 ASC NULLS LAST], preserve_partitioning=[true] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[true] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------CoalesceBatchesExec: target_batch_size=4096 09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2112,7 +2112,7 @@ physical_plan 05)--------WindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 07)------------WindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST], preserve_partitioning=[false] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2168,7 +2168,7 @@ physical_plan 06)----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] 07)------------BoundedWindowAggExec: wdw=[sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 08)--------------WindowAggExec: wdw=[sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST], preserve_partitioning=[false] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] 10)------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2211,7 +2211,7 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: true }], mode=[Sorted] 05)--------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true query RR @@ -2356,7 +2356,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST, c9@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] @@ -3051,15 +3051,15 @@ physical_plan 01)SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] 03)----BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Sorted] -04)------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: expr=[d@4 ASC NULLS LAST, a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +06)----------SortExec: expr=[b@2 ASC NULLS LAST, a@1 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +08)--------------SortExec: expr=[b@2 ASC NULLS LAST, a@1 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 09)----------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +10)------------------SortExec: expr=[a@1 ASC NULLS LAST, d@4 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 11)--------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] -12)----------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +12)----------------------SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 13)------------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 14)--------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3144,7 +3144,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC], preserve_partitioning=[false] +01)SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST, c9@0 DESC], preserve_partitioning=[false] 02)--ProjectionExec: expr=[c9@0 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] @@ -3264,17 +3264,17 @@ physical_plan 01)ProjectionExec: expr=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] 02)--BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=__common_expr_1@0 ASC NULLS LAST,a@1 ASC NULLS LAST +04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=__common_expr_1@0 ASC NULLS LAST, a@1 ASC NULLS LAST 05)--------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------CoalesceBatchesExec: target_batch_size=4096 -08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 09)----------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] 10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 12)----------------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 13)------------------------CoalesceBatchesExec: target_batch_size=4096 -14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -3624,7 +3624,7 @@ physical_plan 02)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] 03)----BoundedWindowAggExec: wdw=[avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] 04)------CoalesceBatchesExec: target_batch_size=4096 -05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]]