Skip to content

Conversation

@avantgardnerio
Copy link
Contributor

Which issue does this PR close?

Rationale for this change

We would like plans with window functions to go faster.

What changes are included in this PR?

A proof-of-concept level optimizer rule push limits past some window functions.

Are these changes tested?

So far only by the existing sqllogictest suite.

Are there any user-facing changes?

Queries with window functions may go faster, particularly if the TableProvider supports TopK pushdown.

@github-actions github-actions bot added documentation Improvements or additions to documentation optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt) common Related to common crate labels Aug 28, 2025
@avantgardnerio avantgardnerio marked this pull request as ready for review August 29, 2025 14:46
Copy link
Contributor

@akurmustafa akurmustafa left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @avantgardnerio for this PR

if let Some(window) = node.as_any().downcast_ref::<BoundedWindowAggExec>() {
for expr in window.window_expr().iter() {
let frame = expr.get_window_frame();
if frame.units != WindowFrameUnits::Rows {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As far as I can see, we update latest_max only for WindowFrameUnits::Rows cases. If I am not mistaken, it is also valid to update latest_max for WindowFrameUnits::Range and WindowFrameUnits::Groups as longs as end_bound is WindowFrameBound::Preceding(_). I think, we can extend checks to include this use case also.
This should change the plan for following kind of queries

SELECT
    c9,
    SUM(c9) OVER(ORDER BY c9 ASC RANGE BETWEEN 5 PRECEDING AND 1 PRECEDING) as sum1
    FROM aggregate_test_100
    LIMIT 5

However, we can do this change in subsequent PRs. I think in current form, this PR is correct and we can merge as is. Thanks @avantgardnerio for this PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point! I will file a subsequent PR 😄

Copy link
Contributor Author

@avantgardnerio avantgardnerio Oct 14, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried this and I think it's unsafe:

1. query result mismatch:
[SQL] SELECT
 SUM(c1) OVER (ORDER BY c2 DESC) as summation1
 FROM null_cases
 LIMIT 5;
[Diff] (-expected|+actual)
-   962
-   962
-   962
-   962
-   962
+   263
+   263
+   263
+   263
+   263

and

[Diff] (-expected|+actual)
    logical_plan
    01)Projection: sum(null_cases.c1) ORDER BY [null_cases.c2 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS summation1
    02)--Limit: skip=0, fetch=5
    03)----WindowAggr: windowExpr=[[sum(null_cases.c1) ORDER BY [null_cases.c2 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]]
    04)------TableScan: null_cases projection=[c1, c2]
    physical_plan
    01)ProjectionExec: expr=[sum(null_cases.c1) ORDER BY [null_cases.c2 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as summation1]
    02)--GlobalLimitExec: skip=0, fetch=5
    03)----BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c2 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c2 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted]
-   04)------SortExec: expr=[c2@1 DESC], preserve_partitioning=[false]
+   04)------SortExec: TopK(fetch=5), expr=[c2@1 DESC], preserve_partitioning=[false]
    05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2], file_type=csv, has_header=true

I think this is failing because there are multiple entries with the same value, so it needs more than the limit.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

However, for this case end bound is Current Row. I think, when end bound is WindowFrameBound::Preceding(_) we can still do this optimization for WindowFrameUnits::Range and WindowFrameUnits::Groups

@avantgardnerio avantgardnerio merged commit 25acb64 into apache:main Aug 29, 2025
29 checks passed
@avantgardnerio avantgardnerio deleted the bg_push_limits branch August 29, 2025 17:32
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Sep 2, 2025
Add physical optimizer rule to push limits past certain window functions (part 1).

(cherry picked from commit 25acb64)
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Oct 2, 2025
avantgardnerio added a commit to coralogix/arrow-datafusion that referenced this pull request Oct 15, 2025
Dandandan added a commit to coralogix/arrow-datafusion that referenced this pull request Nov 17, 2025
* Get working build

* Add pool_size method to MemoryPool (#218) (#230)

* Add pool_size method to MemoryPool

* Fix

* Fmt

Co-authored-by: Daniël Heres <danielheres@gmail.com>

* Respect `IGNORE NULLS` flag in `ARRAY_AGG` (#260) (apache#15544) v48

* Hook for doing distributed `CollectLeft` joins (#269)

* Ignore writer shutdown error (#271)

* ignore writer shutdown error

* cargo check

* Fix  bug in `swap_hash_join` (#278)

* Try and fix swap_hash_join

* Only swap projections when join does not have projections

* just backport upstream fix

* remove println

* Support Duration in min/max agg functions (#283) (apache#15310) v47

* Support Duration in min/max agg functions

* Attempt to fix build

* Attempt to fix build - Fix chrono version

* Revert "Attempt to fix build - Fix chrono version"

This reverts commit fd76fe6.

* Revert "Attempt to fix build"

This reverts commit 9114b86.

---------

Co-authored-by: svranesevic <svranesevic@users.noreply.github.com>

* Fix panics in array_union (#287) (apache#15149) v48

* Drop rust-toolchain

* Fix panics in array_union

* Fix the chrono

* Backport `GroupsAccumulator` for Duration min/max agg (#288) (apache#15322) v47

* Fix array_sort for empty record batch (#290) (apache#15149) v48

* fix: rewrite fetch, skip of the Limit node in correct order (apache#14496) v46

* fix: rewrite fetch, skip of the Limit node in correct order

* style: fix clippy

* Support aliases in ConstEvaluator (apache#14734) (#281) v46

* Support aliases in ConstEvaluator (apache#14734)

Not sure why they are not supported. It seems that if we're not careful,
some transformations can introduce aliases nested inside other expressions.

* Format Cargo.toml

* Preserve the name of grouping sets in SimplifyExpressions (#282) (apache#14888) v46

Whenever we use `recompute_schema` or `with_exprs_and_inputs`,
this ensures that we obtain the same schema.

* Support Duration in min/max agg functions (#284) (apache#15310) v47

Co-authored-by: svranesevic <svranesevic@users.noreply.github.com>

* fix case_column_or_null with nullable when conditions (apache#13886) v45

* fix case_column_or_null with nullable when conditions

* improve sqllogictests for case_column_or_null

---------

Co-authored-by: zhangli20 <zhangli20@kuaishou.com>

* Fix casewhen (apache#14156) v45

* Cherry-pick topk limit pushdown fix (apache#14192) v45

* fix: FULL OUTER JOIN and LIMIT produces wrong results (apache#14338) v45

* fix: FULL OUTER JOIN and LIMIT produces wrong results

* Fix minor slt testing

* fix test

(cherry picked from commit ecc5694)

* Cherry-pick global limit fix (apache#14245) v45

* fix: Limits are not applied correctly (apache#14418) v46

* fix: Limits are not applied correctly

* Add easy fix

* Add fix

* Add slt testing

* Address comments

* Disable grouping set in CSE

* Fix spm + limit (apache#14569) v46

* prost 0.13 / fix parquet dep

* Delete unreliable checks

* Segfault in ByteGroupValueBuilder (#294) (apache#15968) v50

* test to demonstrate segfault in ByteGroupValueBuilder

* check for offset overflow

* clippy

(cherry picked from commit 5bdaeaf)

* Update arrow dependency to include rowid (#295)

* Update arrow version

* Feat: Add fetch to CoalescePartitionsExec (apache#14499) (#298) v46

* add fetch info to CoalescePartitionsExec

* use Statistics with_fetch API on CoalescePartitionsExec

* check limit_reached only if fetch is assigned

Co-authored-by: mertak-synnada <mertak67+synaada@gmail.com>

* Fix `CoalescePartitionsExec` proto serialization (apache#15824) (#299) v48

* add fetch to CoalescePartitionsExecNode

* gen proto code

* Add test

* fix

* fix build

* Fix test build

* remove comments

Co-authored-by: 张林伟 <lewiszlw520@gmail.com>

* Add JoinContext with JoinLeftData to TaskContext in HashJoinExec (#300)

* Add JoinContext with JoinLeftData to TaskContext in HashJoinExec

* Expose random state as const

* re-export ahash::RandomState

* JoinContext default impl

* Add debug log when setting join left data

* Update arrow version for not preserving dict_id (#303)

* Use partial aggregation schema for spilling to avoid column mismatch in GroupedHashAggregateStream (apache#13995) (#302) v45

* Refactor spill handling in GroupedHashAggregateStream to use partial aggregate schema

* Implement aggregate functions with spill handling in tests

* Add tests for aggregate functions with and without spill handling

* Move test related imports into mod test

* Rename spill pool test functions for clarity and consistency

* Refactor aggregate function imports to use fully qualified paths

* Remove outdated comments regarding input batch schema for spilling in GroupedHashAggregateStream

* Update aggregate test to use AVG instead of MAX

* assert spill count

* Refactor partial aggregate schema creation to use create_schema function

* Refactor partial aggregation schema creation and remove redundant function

* Remove unused import of Schema from arrow::datatypes in row_hash.rs

* move spill pool testing for aggregate functions to physical-plan/src/aggregates

* Use Arc::clone for schema references in aggregate functions

(cherry picked from commit 81b50c4)

Co-authored-by: kosiew <kosiew@gmail.com>

* Update tag

* Push limits past windows (#337) (apache#17347) v50

* Restore old method for DQE

* feat(optimizer): Enable filter pushdown on window functions (apache#14026) v45

* Avoid Aliased Window Expr Enter Unreachable Code (apache#14109) v45

(cherry picked from commit fda500a)

* Use `Expr::qualified_name()` and `Column::new()` to extract partition keys from window and aggregate operators (#355) (apache#17757) v51

* Update PR template to be relevant to our fork

* Make limit pushdown work for SortPreservingMergeExec (apache#17893) (#361)

* re-publicise functions DQE relies on

* Handle columns in with_new_exprs with a Join (apache#15055) (#384)

apache#15055

* handle columns in with_new_exprs with Join

* test doesn't return result

* take join from result

* clippy

* make test fallible

* accept any pair of expression for new_on in with_new_exprs for Join

* use with_capacity

Co-authored-by: delamarch3 <68732277+delamarch3@users.noreply.github.com>

---------

Co-authored-by: Georgi Krastev <georgi.krastev@coralogix.com>
Co-authored-by: Daniël Heres <danielheres@gmail.com>
Co-authored-by: Dan Harris <1327726+thinkharderdev@users.noreply.github.com>
Co-authored-by: Faiaz Sanaulla <105630300+fsdvh@users.noreply.github.com>
Co-authored-by: Sava Vranešević <20240220+svranesevic@users.noreply.github.com>
Co-authored-by: svranesevic <svranesevic@users.noreply.github.com>
Co-authored-by: Yingwen <realevenyag@gmail.com>
Co-authored-by: Zhang Li <richselian@gmail.com>
Co-authored-by: zhangli20 <zhangli20@kuaishou.com>
Co-authored-by: Aleksey Kirilishin <54231417+avkirilishin@users.noreply.github.com>
Co-authored-by: xudong.w <wxd963996380@gmail.com>
Co-authored-by: Qi Zhu <821684824@qq.com>
Co-authored-by: Martins Purins <martins.purins@coralogix.com>
Co-authored-by: mertak-synnada <mertak67+synaada@gmail.com>
Co-authored-by: 张林伟 <lewiszlw520@gmail.com>
Co-authored-by: kosiew <kosiew@gmail.com>
Co-authored-by: nuno-faria <nunofpfaria@gmail.com>
Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com>
Co-authored-by: Mason Hall <mason.hall@coralogix.com>
Co-authored-by: delamarch3 <68732277+delamarch3@users.noreply.github.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

common Related to common crate documentation Improvements or additions to documentation optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Optimizer should push limits past window functions

4 participants