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

Fix cte filter pushdown wrong results by splitting SpecialFormExpressions #22700

Merged
merged 1 commit into from
May 9, 2024

Conversation

jaystarshot
Copy link
Member

@jaystarshot jaystarshot commented May 8, 2024

Description

Fixes #22147. This is needed due to a bug in SpecialFormExpressions #22698 if > 2 conditions are used.
Also simplified the extra filters so that predicatePushdown optimizer could push down the conditions further

Motivation and Context

Impact

Test Plan

Contributor checklist

  • Please make sure your submission complies with our development, formatting, commit message, and attribution guidelines.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== RELEASE NOTES ==

General Changes
* Fix wrong results when queries using materialized CTEs have multiple common filters pushed into the CTE :pr:`22700 `

@jaystarshot jaystarshot force-pushed the oss-cte-filter-fix branch 2 times, most recently from fbeb8cc to 07e1101 Compare May 8, 2024 21:21
@jaystarshot jaystarshot marked this pull request as ready for review May 8, 2024 21:22
@jaystarshot jaystarshot requested review from feilong-liu and a team as code owners May 8, 2024 21:22
}
return new FilterNode(node.getSourceLocation(), idAllocator.getNextId(), node, predicate);
resultPredicate = SimplifyRowExpressions.rewrite(resultPredicate, metadata, session.toConnectorSession());
Copy link
Member Author

@jaystarshot jaystarshot May 9, 2024

Choose a reason for hiding this comment

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

Simplifying the or conditions is essential and I observed a case in prod where it helped push these filters further down in predicatePushdown rule

Copy link
Contributor

Choose a reason for hiding this comment

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

Curious what does this simplify do? Is it to rebalance multiple OR condition?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yeah looks like it breaks down the multiple ors into efficient cnf and dnf forms. In the predicate pushdown optimizer, the current visitJoin method appears to only decompose projects when they are pre broken by this optimizer.
I've noticed instances where 'or' conditions were not pushed down beneath joins.

@steveburnett
Copy link
Contributor

Just a nit, suggest release note entry revision.

== RELEASE NOTES ==

General Changes
* Fix a bug in common filter pushdown when using CTE materialization :pr:`22700 `

@rschlussel
Copy link
Contributor

The release notes should mention that it's a wrong results fix and a bit more user facing. e.g. maybe something like

  • Fix wrong results when queries using materialized CTEs have multiple common filters pushed into the CTE

@rschlussel
Copy link
Contributor

Can be separate but we should add argument checks when creating special form expressions that they have the right number of arguments.

@jaystarshot jaystarshot merged commit 8e34250 into prestodb:master May 9, 2024
56 checks passed
@wanglinsong wanglinsong mentioned this pull request Jun 25, 2024
36 tasks
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Incorrect results after CteCommonFilterPushdown due to trimmed filters
4 participants