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

feat(optimizer): Implement greedy join reordering #3538

Open
wants to merge 8 commits into
base: main
Choose a base branch
from

Conversation

desmondcheongzx
Copy link
Contributor

@desmondcheongzx desmondcheongzx commented Dec 10, 2024

Implements the Greedy Operator Ordering algorithm for join reordering.

With GreedyJoinOrderer, we can now transform join graphs back into logical plans with joins greedily reordered. Although we ultimately still want to apply DP-based join order algorithms, this greedy algorithm is still necessary when the number of join conditions is large and DP-based algorithms (which run in O(3^n), where n is the number of relations) are too expensive.

Copy link

codspeed-hq bot commented Dec 10, 2024

CodSpeed Performance Report

Merging #3538 will degrade performances by 49.57%

Comparing desmondcheongzx:goo-join-order (c77c3d1) with main (c3088d6)

Summary

❌ 2 regressions
✅ 15 untouched benchmarks

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

Benchmarks breakdown

Benchmark main desmondcheongzx:goo-join-order Change
test_iter_rows_first_row[100 Small Files] 105.9 ms 210 ms -49.57%
test_show[100 Small Files] 15.8 ms 23.4 ms -32.41%

Copy link

codecov bot commented Dec 11, 2024

Codecov Report

Attention: Patch coverage is 94.10029% with 20 lines in your changes missing coverage. Please review.

Project coverage is 77.72%. Comparing base (4567601) to head (c77c3d1).
Report is 7 commits behind head on main.

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

Impacted file tree graph

@@            Coverage Diff             @@
##             main    #3538      +/-   ##
==========================================
+ Coverage   77.69%   77.72%   +0.03%     
==========================================
  Files         710      714       +4     
  Lines       86896    87389     +493     
==========================================
+ Hits        67513    67923     +410     
- Misses      19383    19466      +83     
Files with missing lines Coverage Δ
src/common/scan-info/src/test/mod.rs 41.30% <100.00%> (+3.94%) ⬆️
src/daft-logical-plan/src/test/mod.rs 100.00% <100.00%> (ø)
src/daft-physical-plan/src/test/mod.rs 100.00% <100.00%> (ø)
...imization/rules/reorder_joins/greedy_join_order.rs 95.23% <95.23%> (ø)
...src/optimization/rules/reorder_joins/join_graph.rs 95.46% <93.27%> (-2.51%) ⬇️

... and 21 files with indirect coverage changes

@desmondcheongzx desmondcheongzx changed the title DRAFT: Implement Greedy Operator Ordering feat(optimizer): Implement greedy join reordering Dec 11, 2024
@github-actions github-actions bot added the feat label Dec 11, 2024
}
}

impl Display for JoinEdge {
pub(crate) struct JoinAdjList(
pub HashMap<LogicalPlanRef, HashMap<LogicalPlanRef, Vec<JoinCondition>>>,
Copy link
Member

Choose a reason for hiding this comment

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

I would strong avoid using the LogicalPlanRef as a hash key here. Whenever you do and insert or a look up the logical plan has to be hashed and then equity has to be computed. I would instead just assign each Node an incrementing ID or use the Arc::ptr as the ID

Copy link
Member

Choose a reason for hiding this comment

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

Alternatively you can also use something like a

struct GraphNodeKey {
  Id: usize,
  plan: LogicalPlanRef // ignored for hash and eq calcuations
}

#[derive(Debug)]
struct JoinEdge(JoinNode, JoinNode);
#[derive(Clone, Debug)]
pub(crate) struct JoinCondition {
Copy link
Member

Choose a reason for hiding this comment

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

do these need to pub(crate) or just local to the file pub(self) or nothing at all


impl GreedyJoinOrderer {
/// Consumes the join graph and transforms it into a logical plan with joins reordered.
pub(crate) fn compute_join_order(join_graph: &mut JoinGraph) -> DaftResult<LogicalPlanRef> {
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 this is doing too much (reordering and creating the logical plan) and isn't really that modular.

Instead what I would recommend is the following:

Trait JoinReorderer {
  fn reorder(&self, &mut graph: JoinGraph)
}

LogicalPlan -> JoinGraphBuilder -> JoinGraph -> JoinReorderer.reorder(JoinGraph) -> JoinGraph -> LogicalPlan

This way you get two things:

  1. must easier to test that the JoinGraph roundtrip is working correctly
  2. much easier to plug in a JoinReorderer (just implement that Trait)

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