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(optimizer): Fix issues with join graph construction #3668

Merged

Conversation

desmondcheongzx
Copy link
Contributor

We make three fixes to join graph construction. With these changes, join ordering using the naive left deep join orderer produces correct results for TPCH queries.

Push up projections and filters only if they sit in between joins

Instead of pushing up all Projections and Filters until we hit an unreorderable node, we go down each linear chain of (reorderable) Projections and Filters until we hit a Join node, then push up the Projects and Filters we encountered along the way. If we hit an unreorderable node first, we simply treat the operator that sits at the top of the current linear chain as the relation to pass into the join.

For example, consider this query tree:

        //                InnerJoin (a = d)
        //                    /        \
        //                   /       Project
        //                  /        (d, quad <- double + double)
        //                 /               \
        //     InnerJoin (a = b)    InnerJoin (c = d)
        //        /        \           /         \
        // Scan(a)      Scan(b)   Filter        Scan(d)
        //                        (c < 5)
        //                           |
        //                        Project
        //                        (c <- c_prime, double <- c_prime + c_prime)
        //                           |
        //                        Filter
        //                        (c_prime > 0)
        //                           |
        //                        Scan(c_prime)

In between InnerJoin(c=d) and Scan(c_prime) there are Filter and Project nodes. Since there is no join below InnerJoin(c=d), we take the Filter(c<5) operator as the relation to pass into the join (as opposed to using Scan(c_prime) and pushing up the Projects and Filters above it).

If a relation needs to rename one column, make sure the other columns are also selected

Previously, a relation might need to have a column renamed. However when we did this, we didn't select the other columns in the relation, causing some columns to be dropped prematurely.

Rename columns for relations even if they are not involved in join conditions

Previously, if we encountered a projection (e.g. a_prime <- a), we would apply this projection directly above the source relation if a_prime was involved in a join, e.g. Join(left_on="a_prime", ..). Now, we uniformly apply these projections regardless if the projection affects a join condition.

@desmondcheongzx desmondcheongzx force-pushed the fix-join-graph-building branch from 822f145 to 62a604c Compare January 11, 2025 00:23
Copy link

codspeed-hq bot commented Jan 11, 2025

CodSpeed Performance Report

Merging #3668 will degrade performances by 30.82%

Comparing desmondcheongzx:fix-join-graph-building (7c578a8) with main (5702720)

Summary

❌ 1 regressions
✅ 26 untouched benchmarks

⚠️ Please fix the performance issues or acknowledge them on CodSpeed.

Benchmarks breakdown

Benchmark main desmondcheongzx:fix-join-graph-building Change
test_iter_rows_first_row[100 Small Files] 180.9 ms 261.5 ms -30.82%

Copy link

codecov bot commented Jan 11, 2025

Codecov Report

Attention: Patch coverage is 90.90909% with 15 lines in your changes missing coverage. Please review.

Project coverage is 77.80%. Comparing base (c932ec9) to head (7c578a8).
Report is 12 commits behind head on main.

Files with missing lines Patch % Lines
...src/optimization/rules/reorder_joins/join_graph.rs 90.85% 15 Missing ⚠️
Additional details and impacted files

Impacted file tree graph

@@            Coverage Diff             @@
##             main    #3668      +/-   ##
==========================================
- Coverage   78.06%   77.80%   -0.27%     
==========================================
  Files         728      729       +1     
  Lines       89967    90557     +590     
==========================================
+ Hits        70236    70458     +222     
- Misses      19731    20099     +368     
Files with missing lines Coverage Δ
.../rules/reorder_joins/naive_left_deep_join_order.rs 95.49% <100.00%> (ø)
...src/optimization/rules/reorder_joins/join_graph.rs 88.99% <90.85%> (+0.19%) ⬆️

... and 46 files with indirect coverage changes

break;
}
_ => {
self.process_leaf_relation(plan);
Copy link
Member

Choose a reason for hiding this comment

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

should this be plan or curr_node?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In this case it's intended to be plan because we just grab the node at the top of the linear chain. I guess process_leaf_relation is a misleading name. Changing the function name to add_relation and adding a comment explaining what's going on.

for (name, node, done) in &mut self.join_conds_to_resolve {
if !*done && schema.has_field(name) {
*node = plan.clone();
let mut cur_node = plan;
Copy link
Member

Choose a reason for hiding this comment

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

I think it would be less bug prone to have

let old_plan = plan;
and then you can modify plan freely instead of referring to cur_node

ending_node: &LogicalPlanRef,
) {
let mut cur_node = starting_node;
while cur_node != ending_node {
Copy link
Member

Choose a reason for hiding this comment

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

Does this do a full equality operation or just a Arc::ptr check?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah for some reason I keep forgetting that Arc dereferences the inner value when doing eq. Will change this to ptr_eq

// Continue to children.
cur_node = input;
}
_ => unreachable!(),
Copy link
Member

Choose a reason for hiding this comment

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

Let's have a better message here

@desmondcheongzx desmondcheongzx merged commit 4b67e5a into Eventual-Inc:main Jan 16, 2025
40 of 41 checks passed
@desmondcheongzx desmondcheongzx deleted the fix-join-graph-building branch January 16, 2025 03:23
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants