Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Feature: Monotonic Sets #14271

Open
wants to merge 17 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
260 changes: 258 additions & 2 deletions datafusion/core/tests/physical_optimizer/enforce_sorting.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,15 +37,15 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE
use datafusion_physical_plan::{get_plan_string, ExecutionPlan};
use datafusion_common::config::ConfigOptions;
use datafusion_common::tree_node::{TreeNode, TransformedResult};
use datafusion_physical_optimizer::test_utils::{check_integrity, bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec, coalesce_batches_exec, aggregate_exec, RequirementsTestExec};
use datafusion_physical_optimizer::test_utils::{check_integrity, bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec, coalesce_batches_exec, aggregate_exec, RequirementsTestExec, create_test_schema4, aggregate_exec_monotonic, aggregate_exec_non_monotonic, bounded_window_exec_with_partition, bounded_window_exec_non_monotonic,};
use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec};
use datafusion_execution::object_store::ObjectStoreUrl;
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::file_format::file_compression_type::FileCompressionType;

use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution;
use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec};
use datafusion_physical_plan::sorts::sort::SortExec;

use rstest::rstest;

/// Create a csv exec for tests
Expand Down Expand Up @@ -232,6 +232,262 @@ async fn test_remove_unnecessary_sort5() -> Result<()> {
Ok(())
}

#[tokio::test]
async fn test_aggregate_monotonic() -> Result<()> {
let schema = create_test_schema4()?;
let source = memory_exec(&schema);
let sort_exprs = vec![sort_expr("a", &schema)];
let sort = sort_exec(sort_exprs.clone(), source);

let aggregate = aggregate_exec_monotonic(sort);
let sort_exprs = LexOrdering::new(vec![sort_expr("count", &aggregate.schema())]);
let physical_plan: Arc<dyn ExecutionPlan> =
Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _;

let expected_input = [
"SortExec: expr=[count@0 ASC], preserve_partitioning=[false]",
" AggregateExec: mode=Single, gby=[], aggr=[count]",
" SortExec: expr=[a@0 ASC], preserve_partitioning=[false]",
" MemoryExec: partitions=1, partition_sizes=[0]",
];

let expected_optimized = [
"AggregateExec: mode=Single, gby=[], aggr=[count]",
" MemoryExec: partitions=1, partition_sizes=[0]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_aggregate_non_monotonic() -> Result<()> {
let schema = create_test_schema4()?;
let source = memory_exec(&schema);
let sort_exprs = vec![sort_expr("a", &schema)];
let sort = sort_exec(sort_exprs.clone(), source);

let aggregate = aggregate_exec_non_monotonic(sort);
let sort_exprs = LexOrdering::new(vec![sort_expr("avg", &aggregate.schema())]);
let physical_plan: Arc<dyn ExecutionPlan> =
Arc::new(SortExec::new(sort_exprs.clone(), aggregate)) as _;

let expected_input = [
"SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]",
" AggregateExec: mode=Single, gby=[], aggr=[avg]",
" SortExec: expr=[a@0 ASC], preserve_partitioning=[false]",
" MemoryExec: partitions=1, partition_sizes=[0]",
];

let expected_optimized = [
"SortExec: expr=[avg@0 ASC], preserve_partitioning=[false]",
" AggregateExec: mode=Single, gby=[], aggr=[avg]",
" MemoryExec: partitions=1, partition_sizes=[0]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_bounded_window_monotonic_sort() -> Result<()> {
let schema = create_test_schema()?;
let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let source = parquet_exec_sorted(&schema, sort_exprs.clone());
let sort = sort_exec(sort_exprs.clone(), source);

let bounded_window = bounded_window_exec("nullable_col", sort_exprs.clone(), sort);
let output_schema = bounded_window.schema();
let sort_exprs2 = vec![sort_expr_options(
"count",
&output_schema,
SortOptions {
descending: false,
nulls_first: false,
},
)];
let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window);

let expected_input = [
"SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]",
" BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]",
" SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
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]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_bounded_plain_window_monotonic_sort_with_partitions() -> Result<()> {
let schema = create_test_schema()?;
let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let source = parquet_exec_sorted(&schema, sort_exprs.clone());
let sort = sort_exec(sort_exprs.clone(), source);
let partition_bys = &[col("nullable_col", &schema)?];

let bounded_window = bounded_window_exec_with_partition(
"nullable_col",
sort_exprs.clone(),
partition_bys,
sort,
false,
);
let output_schema = bounded_window.schema();
let sort_exprs2 = vec![
sort_expr_options(
"nullable_col",
&output_schema,
SortOptions {
descending: true,
nulls_first: false,
},
),
sort_expr_options(
"count",
&output_schema,
SortOptions {
descending: false,
nulls_first: false,
},
),
];
let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window);

let expected_input = [
"SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]",
" BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]",
" SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
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]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_bounded_sliding_window_monotonic_sort_with_partitions() -> Result<()> {
let schema = create_test_schema()?;
let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let source = parquet_exec_sorted(&schema, sort_exprs.clone());
let sort = sort_exec(sort_exprs.clone(), source);
let partition_bys = &[col("nullable_col", &schema)?];

let bounded_window = bounded_window_exec_with_partition(
"nullable_col",
sort_exprs.clone(),
partition_bys,
sort,
true,
);
let output_schema = bounded_window.schema();
let sort_exprs2 = vec![
sort_expr_options(
"nullable_col",
&output_schema,
SortOptions {
descending: true,
nulls_first: false,
},
),
sort_expr_options(
"count",
&output_schema,
SortOptions {
descending: false,
nulls_first: false,
},
),
];
let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window);

let expected_input = [
"SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]",
" BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]",
" SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
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: CurrentRow, end_bound: Following(NULL), is_causal: false }], mode=[Sorted]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 DESC NULLS LAST]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_bounded_window_non_monotonic_sort() -> Result<()> {
let schema = create_test_schema4()?;
let sort_exprs = vec![sort_expr_options(
"a",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let source = parquet_exec_sorted(&schema, sort_exprs.clone());
let sort = sort_exec(sort_exprs.clone(), source);

let bounded_window = bounded_window_exec_non_monotonic("a", sort_exprs.clone(), sort);
let output_schema = bounded_window.schema();
let sort_exprs2 = vec![sort_expr_options(
"avg",
&output_schema,
SortOptions {
descending: false,
nulls_first: false,
},
)];
let physical_plan = sort_exec(sort_exprs2.clone(), bounded_window);

let expected_input = [
"SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]",
" BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, 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 DESC NULLS LAST], preserve_partitioning=[false]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]",
];
let expected_optimized = [
"SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]",
" BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]",
" ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_do_not_remove_sort_with_limit() -> Result<()> {
let schema = create_test_schema()?;
Expand Down
3 changes: 2 additions & 1 deletion datafusion/expr/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,8 @@ pub use partition_evaluator::PartitionEvaluator;
pub use sqlparser;
pub use table_source::{TableProviderFilterPushDown, TableSource, TableType};
pub use udaf::{
aggregate_doc_sections, AggregateUDF, AggregateUDFImpl, ReversedUDAF, StatisticsArgs,
aggregate_doc_sections, AggregateExprMonotonicity, AggregateUDF, AggregateUDFImpl,
ReversedUDAF, StatisticsArgs,
};
pub use udf::{
scalar_doc_sections, ReturnInfo, ReturnTypeArgs, ScalarFunctionArgs, ScalarUDF,
Expand Down
24 changes: 23 additions & 1 deletion datafusion/expr/src/test/function_stub.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,8 @@ use crate::{
expr::AggregateFunction,
function::{AccumulatorArgs, StateFieldsArgs},
utils::AggregateOrderSensitivity,
Accumulator, AggregateUDFImpl, Expr, GroupsAccumulator, ReversedUDAF, Signature,
Accumulator, AggregateExprMonotonicity, AggregateUDFImpl, Expr, GroupsAccumulator,
ReversedUDAF, Signature,
};

macro_rules! create_func {
Expand Down Expand Up @@ -203,6 +204,17 @@ impl AggregateUDFImpl for Sum {
fn order_sensitivity(&self) -> AggregateOrderSensitivity {
AggregateOrderSensitivity::Insensitive
}

fn monotonicity(&self, data_type: &DataType) -> AggregateExprMonotonicity {
// Sum is only monotonic if its input is unsigned
match data_type {
DataType::UInt8 => AggregateExprMonotonicity::MonotonicallyAscending,
DataType::UInt16 => AggregateExprMonotonicity::MonotonicallyAscending,
DataType::UInt32 => AggregateExprMonotonicity::MonotonicallyAscending,
DataType::UInt64 => AggregateExprMonotonicity::MonotonicallyAscending,
_ => AggregateExprMonotonicity::NotMonotonic,
}
}
}

/// Testing stub implementation of COUNT aggregate
Expand Down Expand Up @@ -278,6 +290,10 @@ impl AggregateUDFImpl for Count {
fn reverse_expr(&self) -> ReversedUDAF {
ReversedUDAF::Identical
}

fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity {
AggregateExprMonotonicity::MonotonicallyAscending
}
}

create_func!(Min, min_udaf);
Expand Down Expand Up @@ -363,6 +379,9 @@ impl AggregateUDFImpl for Min {
fn is_descending(&self) -> Option<bool> {
Some(false)
}
fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity {
AggregateExprMonotonicity::MonotonicallyDescending
}
}

create_func!(Max, max_udaf);
Expand Down Expand Up @@ -448,6 +467,9 @@ impl AggregateUDFImpl for Max {
fn is_descending(&self) -> Option<bool> {
Some(true)
}
fn monotonicity(&self, _data_type: &DataType) -> AggregateExprMonotonicity {
AggregateExprMonotonicity::MonotonicallyAscending
}
}

/// Testing stub implementation of avg aggregate
Expand Down
Loading
Loading