Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
34 commits
Select commit Hold shift + click to select a range
a2919b6
add monotonic function definitions for aggregate expressions
Jan 16, 2025
14109e6
fix benchmark results
Jan 16, 2025
b3d75ba
set prefer_existing_sort to true in sqllogictests
Jan 17, 2025
549502e
set prefer_existing_sort to true in sqllogictests
Jan 17, 2025
623e0c5
fix typo
Jan 17, 2025
6a9d24e
Merge branch 'refs/heads/apache_main' into feature/monotonic-sets
Jan 20, 2025
53ee3de
re-add test_utils.rs changes to the new file
Jan 20, 2025
97d8951
clone input with Arc
Jan 20, 2025
cc33031
Merge branch 'refs/heads/apache_main' into feature/monotonic-sets
Jan 22, 2025
41d9430
Merge branch 'refs/heads/apache_main' into feature/monotonic-sets
Jan 23, 2025
e988dcf
inject aggr expr indices
Jan 23, 2025
906245e
remove redundant file
Jan 23, 2025
475fe2d
add Sum monotonicity
Jan 24, 2025
57e000e
fix sql logic tests
Jan 24, 2025
ca57f46
fix sql logic tests
Jan 24, 2025
6cf9644
Merge branch 'refs/heads/apache_main' into feature/monotonic-sets
Jan 24, 2025
072e6ef
update docs
Jan 24, 2025
7d62cb0
Merge branch 'apache_main' into feature/monotonic-sets
berkaysynnada Jan 28, 2025
491aabe
review part 1
berkaysynnada Jan 28, 2025
972c56f
fix the tests
berkaysynnada Jan 29, 2025
4b946b3
revert slt's
berkaysynnada Jan 29, 2025
481b5b4
simplify terms
berkaysynnada Jan 29, 2025
29af731
Update mod.rs
berkaysynnada Jan 29, 2025
1f02953
remove unnecessary computations
berkaysynnada Jan 29, 2025
79dd942
remove index calc
berkaysynnada Jan 29, 2025
247d5fe
Update mod.rs
berkaysynnada Jan 29, 2025
16bdac4
Apply suggestions from code review
ozankabak Jan 29, 2025
1875336
add slt
berkaysynnada Jan 29, 2025
ba7b94f
remove aggregate changes, tests already give expected results
berkaysynnada Jan 30, 2025
2152b7f
fix clippy
berkaysynnada Jan 30, 2025
7822613
remove one row sorts
berkaysynnada Jan 30, 2025
5e9b2db
Improve comments
ozankabak Jan 30, 2025
54d62d6
Use a short name for set monotonicity
ozankabak Jan 30, 2025
1146811
Merge branch 'main' into feature/monotonic-sets
ozankabak Jan 31, 2025
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
214 changes: 209 additions & 5 deletions datafusion/core/tests/physical_optimizer/enforce_sorting.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,14 @@
use std::sync::Arc;

use crate::physical_optimizer::test_utils::{
aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec,
aggregate_exec, bounded_window_exec, bounded_window_exec_non_set_monotonic,
bounded_window_exec_with_partition, check_integrity, coalesce_batches_exec,
coalesce_partitions_exec, create_test_schema, create_test_schema2,
create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec,
local_limit_exec, memory_exec, parquet_exec, 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, RequirementsTestExec,
create_test_schema3, create_test_schema4, filter_exec, global_limit_exec,
hash_join_exec, limit_exec, local_limit_exec, memory_exec, parquet_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,
RequirementsTestExec,
};

use datafusion_physical_plan::displayable;
Expand Down Expand Up @@ -238,6 +240,208 @@ async fn test_remove_unnecessary_sort5() -> Result<()> {
Ok(())
}

#[tokio::test]
async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> {
let schema = create_test_schema()?;

let source = parquet_exec_sorted(&schema, vec![]);

let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let sort = sort_exec(sort_exprs.clone(), source);

let bounded_window = bounded_window_exec("nullable_col", vec![], 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]",
];
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]",
];
assert_optimized!(expected_input, expected_optimized, physical_plan, true);

Ok(())
}

#[tokio::test]
async fn test_bounded_plain_window_set_monotonic_with_partitions() -> Result<()> {
let schema = create_test_schema()?;

let source = parquet_exec_sorted(&schema, vec![]);

let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let sort = sort_exec(sort_exprs.clone(), source);

let partition_bys = &[col("nullable_col", &schema)?];
let bounded_window = bounded_window_exec_with_partition(
"non_nullable_col",
vec![],
partition_bys,
sort,
false,
);

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]",
];
let expected_optimized = [
"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 ASC 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);

Ok(())
}

#[tokio::test]
async fn test_bounded_plain_window_set_monotonic_with_partitions_partial() -> Result<()> {
let schema = create_test_schema()?;

let source = parquet_exec_sorted(&schema, vec![]);

let sort_exprs = vec![sort_expr_options(
"nullable_col",
&schema,
SortOptions {
descending: true,
nulls_first: false,
},
)];
let sort = sort_exec(sort_exprs.clone(), source);

let partition_bys = &[col("nullable_col", &schema)?];
let bounded_window = bounded_window_exec_with_partition(
"non_nullable_col",
vec![],
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]",
];
let expected_optimized = [
"BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]",
" SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC 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);

Ok(())
}

#[tokio::test]
async fn test_bounded_window_non_set_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_set_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
59 changes: 57 additions & 2 deletions datafusion/core/tests/physical_optimizer/test_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ use datafusion_common::{JoinType, Result};
use datafusion_execution::object_store::ObjectStoreUrl;
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
use datafusion_expr::{WindowFrame, WindowFunctionDefinition};
use datafusion_functions_aggregate::average::avg_udaf;
use datafusion_functions_aggregate::count::count_udaf;
use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr};
use datafusion_physical_expr::expressions::col;
Expand Down Expand Up @@ -120,6 +121,17 @@ pub fn create_test_schema3() -> Result<SchemaRef> {
Ok(schema)
}

// Generate a schema which consists of 5 columns (a, b, c, d, e) of Uint64
pub fn create_test_schema4() -> Result<SchemaRef> {
let a = Field::new("a", DataType::UInt64, true);
let b = Field::new("b", DataType::UInt64, false);
let c = Field::new("c", DataType::UInt64, true);
let d = Field::new("d", DataType::UInt64, false);
let e = Field::new("e", DataType::Int64, false);
let schema = Arc::new(Schema::new(vec![a, b, c, d, e]));
Ok(schema)
}

pub fn sort_merge_join_exec(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
Expand Down Expand Up @@ -188,15 +200,58 @@ pub fn bounded_window_exec(
col_name: &str,
sort_exprs: impl IntoIterator<Item = PhysicalSortExpr>,
input: Arc<dyn ExecutionPlan>,
) -> Arc<dyn ExecutionPlan> {
bounded_window_exec_with_partition(col_name, sort_exprs, &[], input, false)
}

pub fn bounded_window_exec_with_partition(
col_name: &str,
sort_exprs: impl IntoIterator<Item = PhysicalSortExpr>,
partition_by: &[Arc<dyn PhysicalExpr>],
input: Arc<dyn ExecutionPlan>,
should_reverse: bool,
) -> Arc<dyn ExecutionPlan> {
let sort_exprs: LexOrdering = sort_exprs.into_iter().collect();
let schema = input.schema();
let mut window_expr = create_window_expr(
&WindowFunctionDefinition::AggregateUDF(count_udaf()),
"count".to_owned(),
&[col(col_name, &schema).unwrap()],
partition_by,
sort_exprs.as_ref(),
Arc::new(WindowFrame::new(Some(false))),
schema.as_ref(),
false,
)
.unwrap();
if should_reverse {
window_expr = window_expr.get_reverse_expr().unwrap();
}

Arc::new(
BoundedWindowAggExec::try_new(
vec![window_expr],
Arc::clone(&input),
vec![],
InputOrderMode::Sorted,
)
.unwrap(),
)
}

pub fn bounded_window_exec_non_set_monotonic(
col_name: &str,
sort_exprs: impl IntoIterator<Item = PhysicalSortExpr>,
input: Arc<dyn ExecutionPlan>,
) -> Arc<dyn ExecutionPlan> {
let sort_exprs: LexOrdering = sort_exprs.into_iter().collect();
let schema = input.schema();

Arc::new(
BoundedWindowAggExec::try_new(
vec![create_window_expr(
&WindowFunctionDefinition::AggregateUDF(count_udaf()),
"count".to_owned(),
&WindowFunctionDefinition::AggregateUDF(avg_udaf()),
"avg".to_owned(),
&[col(col_name, &schema).unwrap()],
&[],
sort_exprs.as_ref(),
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, AggregateUDF, AggregateUDFImpl, ReversedUDAF,
SetMonotonicity, StatisticsArgs,
};
pub use udf::{
scalar_doc_sections, ReturnInfo, ReturnTypeArgs, ScalarFunctionArgs, ScalarUDF,
Expand Down
29 changes: 28 additions & 1 deletion datafusion/expr/src/udaf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -389,7 +389,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync {

/// Whether the aggregate function is nullable.
///
/// Nullable means that that the function could return `null` for any inputs.
/// Nullable means that the function could return `null` for any inputs.
/// For example, aggregate functions like `COUNT` always return a non null value
/// but others like `MIN` will return `NULL` if there is nullable input.
/// Note that if the function is declared as *not* nullable, make sure the [`AggregateUDFImpl::default_value`] is `non-null`
Expand Down Expand Up @@ -635,6 +635,12 @@ pub trait AggregateUDFImpl: Debug + Send + Sync {
fn documentation(&self) -> Option<&Documentation> {
None
}

/// Indicates whether the aggregation function is monotonic as a set
/// function. See [`SetMonotonicity`] for details.
fn set_monotonicity(&self, _data_type: &DataType) -> SetMonotonicity {
SetMonotonicity::NotMonotonic
}
}

impl PartialEq for dyn AggregateUDFImpl {
Expand Down Expand Up @@ -818,6 +824,27 @@ pub mod aggregate_doc_sections {
};
}

/// Indicates whether an aggregation function is monotonic as a set
/// function. A set function is monotonically increasing if its value
/// increases as its argument grows (as a set). Formally, `f` is a
/// monotonically increasing set function if `f(S) >= f(T)` whenever `S`
/// is a superset of `T`.
///
/// For example `COUNT` and `MAX` are monotonically increasing as their
/// values always increase (or stay the same) as new values are seen. On
/// the other hand, `MIN` is monotonically decreasing as its value always
/// decreases or stays the same as new values are seen.
#[derive(Debug, Clone, PartialEq)]
pub enum SetMonotonicity {
/// Aggregate value increases or stays the same as the input set grows.
Increasing,
/// Aggregate value decreases or stays the same as the input set grows.
Decreasing,
/// Aggregate value may increase, decrease, or stay the same as the input
/// set grows.
NotMonotonic,
}

#[cfg(test)]
mod test {
use crate::{AggregateUDF, AggregateUDFImpl};
Expand Down
7 changes: 7 additions & 0 deletions datafusion/expr/src/window_frame.rs
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,13 @@ impl WindowFrame {
&& (self.end_bound.is_unbounded()
|| self.end_bound == WindowFrameBound::CurrentRow)
}

/// Is the window frame ever-expanding (it always grows in the superset sense).
/// Useful when understanding if set-monotonicity properties of functions can
/// be exploited.
pub fn is_ever_expanding(&self) -> bool {
self.start_bound.is_unbounded()
}
}

/// There are five ways to describe starting and ending frame boundaries:
Expand Down
Loading