cmu-db / optd

CMU-DB's Cascades optimizer framework
https://cmu-db.github.io/optd/
MIT License
383 stars 22 forks source link

feat(core): support physical properties #238

Open skyzh opened 2 weeks ago

skyzh commented 2 weeks ago

copy-pasted from group discussion


I think an approach similar to Columbia would work fine in optd with some tweaks. We don’t need to store the child property along with each of the expr. The memo table stores everything in the same way as now except the winner change described below. From the paper, If an enforcer multi-expression becomes the winner for a physical property, it and the physical property are stored in the winner structure just like the normal multi-expression winners.

Based on this approach, the winner of a subgroup can be two things: either an expression that satisfies the required property of the subgroup, or a placeholder called enforcer which means that no expression in this group satisfies the required property and we need to add enforcer nodes when generating the binding / computing the cost. So, subgroup = (required_property, subgroup_winner), and subgroup_winner = enum { expr_satisfy_required_property | enforce }

When generating the binding and we encountered enforce is the winner, we get the subgroup winner without any required property and then enforce it by invoking enforcer rule, which is currently implemented as a function over the property struct. Note that we don’t invoke the enforcer rule during apply_rule and add the enforcer plan node directly into the memo table. It should not cause any group merges and should not create dup expressions. However, this approach doesn’t solve the problem of the sort node: consider the case that

group 1: logical sort group 2: logical join, sort merge join

If sort merge join already satisfies the sort order defined in the logical sort node, we do not need to generate the sort node in group 1 again, the output plan should simply be sort merge join. I don’t think Columbia handles this case or describes it in the paper clearly. So, I think we need another type of rule called enforcer’ rule (enforcer prime/alternative rule). The rule’ we need here is that: logical sort on is equivalent to with property enforced. The rule converts a plan node back into the physical property we want to search. On that, we should be able to eliminate the extra sort and simply not generating any operator for group 1.

On that, the winner struct needs to be refined as:

subgroup = (required_property, subgroup_winner), and subgroup_winner = enum { expr_satisfy_required_property | enforce | another group with required property } The enforcer’ rule can also be applied during apply_rule phase. When the search process successfully applied the enforcer rule, it should (1) invoke optimize input on the target group (2) update the winner of the current group to be “another group with required property” if this has a lower cost.

skyzh commented 2 weeks ago

obviously, a lot of things don't work for now...

enforce multiple properties

assume we want to enforce sort + single partition, we can have a MergeGather node for the partition. however, we cannot do that for now. either, we need to let user implement a single physical properties builder that takes all properties at the same time and return a set of operators, or, we should allow the current enforce to take the current input properties.

de-enforce multiple properties

if we are searching for a logical sort with required partition property, we should relay it to searching the child group with both properties, which is not implemented now. also, consider the case that the search goal is both sort prop A and partition prop B, and we are optimizing a logical sort C, we need to delegate this to target group with (A+C) B.

more efficient physical property

we have a lot of sequential-loop-and-equal checks in the codebase, this is not efficient, probably we want to impl Hash on the any type

and, we do a lot of derives + passthroughs, some of the result can indeed be cached

enforcer winner

now we assume the winner is either an expression that can passthrough all required properties, or an enforcer over the winner of the no required prop subgroup. is it possible that we have an expression that can pass through some of the properties, while we need to enforce the rest of the properties, and it has a lower cost?

when porting this code to main, we should resolve all these problems, and probably add the partition property

skyzh commented 2 weeks ago

and a quick explanation of the current code:

subgroup: we don't have a real "sub group" in the system. subgroup is a search goal (required property set) in a group. each group has multiple search goals, aka subgroups, and each of the subgroup has a winner. this is the main change of the memo table.

tasks: each task now comes with a subgroup, which is the search goal we are optimizing for within a group.

merge group: we also need to merge subgroups, and actually this might be wrong right now.

optimize group: now split into 3 stages; firstly, it explores all expressions within the group without any required properties; then, it explores all expressions that can always passthrough the required properties; and finally, it explores enforce the required properties on the winner of the no-required-properties subgroup (this is a new task called "optimize input finalize" now)

optimize input: after the normal optimize input flow, it will also try to call the de-enforcer rule to see if we can "propagate" the physical plan into a new search goal. for example, if we see physical sort C on child X + required sort property A at the end of the optimize input task, we will trigger a search of (A+C) on child X, and update the winner of the current group as "propagate X" if necessary

note that we do not store subgroups as child group IDs in any of the memo table expressions. this is because of the property that child required properties can be solely determined by required output property + node type. therefore, with a search goal (subgroup) and an expression, we can infer all the rest of the information.

also note that the expression in the group does not come with required output property by default. only the tuple (required_output_property, expr_id) can uniquely identify a plan node. so, if we have a filter in the group, it can be used to determine the winner of different required output properties, as filter can passthrough all physical properties. that's why we now have (expr_id, subgroup_id) in the optimizer mark_expr_explored set of functions -- a physical expression explored in different subgroups are different expressions in the end.

enforcing a property: this is actually done at generating bindings. we don't store the enforced operators within the memo table to avoid merging groups.

skyzh commented 2 weeks ago

and to emphasize, subgroup ID is simply a mapping (group_id, subgroup_id) <-> required physical properties. it does not mean an expression has to belong to a subgroup or not.

skyzh commented 2 weeks ago

and I can see benefits with using async (without async runtime) in this case: what if Rust can help us convert awaits into state machine, instead of us doing it, as the task graph is getting more complex...

yliang412 commented 2 weeks ago

I will read through it this weekend :)

skyzh commented 2 weeks ago

A quick idea: why do I choose to limit winners to be expressions in the current group? We can relax the winner definition and make it possible to have a winner that belongs to another logical group, or even a winner not in any logical group (for example, enforcer expressions)