Skip to content

Make scan filter push-down idempotent#20003

Open
askalt wants to merge 2 commits intoapache:mainfrom
askalt:askalt/filter-push-down-improve
Open

Make scan filter push-down idempotent#20003
askalt wants to merge 2 commits intoapache:mainfrom
askalt:askalt/filter-push-down-improve

Conversation

@askalt
Copy link
Copy Markdown
Contributor

@askalt askalt commented Jan 26, 2026

Which issue does this PR close?

Rationale for this change

Non-idempotent rule logic leads to the artifacts described in the issue above if the rule is applied several times.

What changes are included in this PR?

  • Pass filters pushed earlier to supports_filter_pushdown(...) call.
  • Extend a scan projection if some pushed filters become unsupported.

Are these changes tested?

There are unit tests.

@github-actions github-actions bot added optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt) labels Jan 26, 2026
@askalt askalt force-pushed the askalt/filter-push-down-improve branch from 236ef40 to 6697447 Compare January 26, 2026 08:07
@askalt
Copy link
Copy Markdown
Contributor Author

askalt commented Jan 26, 2026

Will fix CI soon.

@askalt askalt force-pushed the askalt/filter-push-down-improve branch 2 times, most recently from b3c756b to 52cb896 Compare January 27, 2026 07:02
Comment on lines +1222 to +1223
// Extend a projection.
projection.extend(additional_projection);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

There should be a tradeoff between fetching an extra column from storage vs the selectivity of the related filter

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

It seems for me that the change only touches correctness of the rule (not performance), here we must extend projection (because filter is explicitly not supported by the provider).

Comment on lines +1206 to +1207

if additional_projection.is_empty() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

The branch of scan is expanding, any thoughts to split the branch

07)------------Projection: customer.c_phone, customer.c_acctbal
08)--------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey
09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])
09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]) AND Boolean(true)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why does this now have "AND true"? That seems unecessary 🤔

Copy link
Copy Markdown
Contributor Author

@askalt askalt Jan 28, 2026

Choose a reason for hiding this comment

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

It is an incorrect behavior of filter-push-down to omit this true in Filter (just a coincidence that it is trivial filter).

Currently, in main branch we have the following:

09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])
10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]), Boolean(true)]

We can notice that here , Boolean(true) in the partial_filters section. As this filter is not supported fully, it should be re-checked in filter node, so if it would non-trivial filter (push-down does not know if is it) it would lead to incorrect selection result.

@askalt askalt force-pushed the askalt/filter-push-down-improve branch from 52cb896 to 5336253 Compare January 28, 2026 07:15
@askalt askalt requested a review from alamb January 28, 2026 07:16
@askalt askalt force-pushed the askalt/filter-push-down-improve branch from 5336253 to 12baee1 Compare January 28, 2026 16:45
askalt added 2 commits March 11, 2026 12:40
Part of apache#19929

Let "t" be a table provider that supports exactly any single filter
but not a conjunction. Consider the following optimizer pipeline:

1. Try to push `a = 1, b = 1`.
   `supports_filters_pushdown` returns [Exact, Inexact]
   OK: the optimizer records that a = 1 is pushed and creates a filter node for b = 1.

...
Another optimization iteration.

2. Try to push `b = 1`.
   `supports_filters_pushdown` returns [Exact]. Of course, the table provider can't
   remember all previously pushed filters, so it has no choice but to answer `Exact`.
   Now, the optimizer thinks the conjunction a = 1 AND b = 1 is supported exactly, but
   it is not.

To prevent this problem, this patch passes filters that were already pushed into the scan
earlier to `supports_filters_pushdown`.
Consider the following optimizer-run scenario:

1. `supports_filters_pushdown` returns `Exact` on some filter, e.g. "a = 1",
   where the column "a" is not required by the query projection.

2. "a" is removed from the table provider projection by "optimize projection"
   rule.

3. `supports_filters_pushdown` changes a decision and returns `Inexact` on
   this filter the next time. e.g., input filters are changed and it prefers
   to use a new one.

4. "a" is not returned to the table provider projection which leads to filter
   that references a column which is not a part of the input schema.

This patch fixes issue introducing the following logic within a filter push-down rule:

1. Collect columns that are not used in the current table provider scan projection,
   but required for filter expressions. Call it `additional_projection`.

2. If `additional_projection` is empty -- leave logic as is prior the patch.

3. Otherwise extend a table provider projection and wrap a plan with
   an additional projection node to preserve schema used prior to the rule.
@askalt askalt force-pushed the askalt/filter-push-down-improve branch from 12baee1 to a59d2fd Compare March 11, 2026 09:41
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Improve filter push-down

3 participants