Skip to content

Commit

Permalink
feat(optimizer): Add scaffolding to create join graphs from logical p…
Browse files Browse the repository at this point in the history
…lans (#3501)

This PR adds the functionality to create join graphs from logical plans.
This is currently not used by Daft for now, as it's a building block for
join reordering.

More concretely, what this PR allows us to do is to take a query tree,
e.g.
```
               InnerJoin (a = d)
                /            \
    InnerJoin (a = b)    InnerJoin (c = d)
       /        \           /         \
     Agg      Scan(b)   Project       Scan(d)
     (Count(a))         (c_prime)
      |                    |
     Project               |
     (a <- a_prime)        |
      |                    |
     Scan(a_prime)      Scan(c_prime)
```
and create a join graph.

Join graphs contain sufficient information to reconstruct a query tree
that is equivalent to the query tree used to create join graph. In the
example above, this would entail storing information about how relations
are connected to each other. E.g.
```
l_a#Aggregate(a) <-> r_b#Source(b)
l_c#Source(c_prime) <-> r_d#Source(d)
l_a#Aggregate(a) <-> r_d#Source(d)
```

A relation here is determined by a "non-reorderable" node. The simplest
example would be a Source node that must be a leaf in the query tree.
But aside from inner joins, filters, and some projects (that do not
perform computation on join keys), most logical operators are
non-reorderable.

In addition to edges between relations, the query tree also maintains a
pre-order record of projections and filters that it encountered in the
query tree, which it would reapply at the top of the reconstructed query
tree. The very last projection to apply is the output schema of the root
logical plan that was used to construct the join graph.
  • Loading branch information
desmondcheongzx authored Dec 10, 2024
1 parent ba46d07 commit 4567601
Show file tree
Hide file tree
Showing 4 changed files with 800 additions and 0 deletions.
20 changes: 20 additions & 0 deletions src/daft-logical-plan/src/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -383,6 +383,26 @@ impl LogicalPlanBuilder {
Ok(self.with_new_plan(pivot_logical_plan))
}

// Helper function to create inner joins more ergonimically in tests.
#[cfg(test)]
pub(crate) fn inner_join<Right: Into<LogicalPlanRef>>(
&self,
right: Right,
left_on: Vec<ExprRef>,
right_on: Vec<ExprRef>,
) -> DaftResult<Self> {
self.join(
right,
left_on,
right_on,
JoinType::Inner,
None,
None,
None,
false,
)
}

#[allow(clippy::too_many_arguments)]
pub fn join<Right: Into<LogicalPlanRef>>(
&self,
Expand Down
1 change: 1 addition & 0 deletions src/daft-logical-plan/src/optimization/rules/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ mod materialize_scans;
mod push_down_filter;
mod push_down_limit;
mod push_down_projection;
mod reorder_joins;
mod rule;
mod simplify_expressions;
mod split_actor_pool_projects;
Expand Down
Loading

0 comments on commit 4567601

Please sign in to comment.