-
Notifications
You must be signed in to change notification settings - Fork 598
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(frontend): prune order key using functional dependencies #16204
Merged
Merged
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
cdc7fe8
use minimize_order_key
kwannoel ac95ac4
handle case where input is not a key
kwannoel 986df89
dapt
kwannoel c7112c9
dont prune index
kwannoel b297bd9
use a different algorithm for order key
kwannoel e7fa42d
dapt
kwannoel 9c2ea29
add more planner test
kwannoel bfb385b
improve code
kwannoel be95574
doc
kwannoel 0127d54
add note on time complexity
kwannoel ac853a4
revert index workaround
kwannoel 5a46f37
handle [] as a fun dep
kwannoel aa11d51
handle dist key
kwannoel 73221fe
fix top n
kwannoel 1ab4f59
minimize order key while making sure dist key is not pruned
kwannoel 4ee529c
minimize order key for topn
kwannoel File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
49 changes: 49 additions & 0 deletions
49
src/frontend/planner_test/tests/testdata/input/functional_dependency.yaml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
- name: test functional dependency for order key pruning (order by - prefix fd) | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
select id, i from t1 order by id, i limit 2; | ||
expected_outputs: | ||
- logical_plan | ||
- stream_plan | ||
- batch_plan | ||
|
||
# Order key should not be pruned for suffix functional dependency. | ||
- name: test functional dependency for order key pruning (order by - suffix fd) | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
select id, i from t1 order by i, id limit 2; | ||
expected_outputs: | ||
- logical_plan | ||
- stream_plan | ||
- batch_plan | ||
# Order key should not be pruned for suffix functional dependency. | ||
- name: test functional dependency for order key pruning on singleton | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
create materialized view v as select count(*) cnt from t1; | ||
select cnt from v order by cnt; | ||
expected_outputs: | ||
- logical_plan | ||
- stream_plan | ||
- batch_plan | ||
# Order key should not be pruned for index, | ||
# since it uses it as distribution key as well. | ||
- name: test functional dependency for order key pruning (index) | ||
sql: | | ||
create table t1 (v1 int, v2 int); | ||
create materialized view v as select count(*) cnt from t1; | ||
explain (verbose) create index mv_idx on v(cnt); | ||
expected_outputs: | ||
- explain_output | ||
- name: test functional dependency for order key pruning (index 2) | ||
sql: | | ||
create table t1 (v1 int primary key, v2 int); | ||
explain (verbose) create index mv_idx on t1(v1, v2); | ||
expected_outputs: | ||
- explain_output | ||
- name: test functional dependency for order key pruning (index 3) | ||
sql: | | ||
create table t1 (v1 int primary key, v2 int); | ||
explain (verbose) create index mv_idx on t1(v2, v1); | ||
expected_outputs: | ||
- explain_output |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
84 changes: 84 additions & 0 deletions
84
src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,84 @@ | ||
# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. | ||
- name: test functional dependency for order key pruning (order by - prefix fd) | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
select id, i from t1 order by id, i limit 2; | ||
logical_plan: |- | ||
LogicalTopN { order: [t1.id ASC], limit: 2, offset: 0 } | ||
└─LogicalProject { exprs: [t1.id, t1.i] } | ||
└─LogicalScan { table: t1, columns: [t1.id, t1.i] } | ||
batch_plan: |- | ||
BatchTopN { order: [t1.id ASC], limit: 2, offset: 0 } | ||
└─BatchExchange { order: [], dist: Single } | ||
└─BatchLimit { limit: 2, offset: 0 } | ||
└─BatchScan { table: t1, columns: [t1.id, t1.i], limit: 2, distribution: UpstreamHashShard(t1.id) } | ||
stream_plan: |- | ||
StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } | ||
└─StreamProject { exprs: [t1.id, t1.i] } | ||
└─StreamTopN { order: [t1.id ASC], limit: 2, offset: 0 } | ||
└─StreamExchange { dist: Single } | ||
└─StreamGroupTopN { order: [t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } | ||
└─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } | ||
└─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } | ||
- name: test functional dependency for order key pruning (order by - suffix fd) | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
select id, i from t1 order by i, id limit 2; | ||
logical_plan: |- | ||
LogicalTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } | ||
└─LogicalProject { exprs: [t1.id, t1.i] } | ||
└─LogicalScan { table: t1, columns: [t1.id, t1.i] } | ||
batch_plan: |- | ||
BatchTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } | ||
└─BatchExchange { order: [], dist: Single } | ||
└─BatchTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } | ||
└─BatchScan { table: t1, columns: [t1.id, t1.i], distribution: UpstreamHashShard(t1.id) } | ||
stream_plan: |- | ||
StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [i, id], pk_conflict: NoCheck } | ||
└─StreamProject { exprs: [t1.id, t1.i] } | ||
└─StreamTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } | ||
└─StreamExchange { dist: Single } | ||
└─StreamGroupTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } | ||
└─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } | ||
└─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } | ||
- name: test functional dependency for order key pruning on singleton | ||
sql: | | ||
create table t1 (id int primary key, i int); | ||
create materialized view v as select count(*) cnt from t1; | ||
select cnt from v order by cnt; | ||
logical_plan: |- | ||
LogicalProject { exprs: [v.cnt] } | ||
└─LogicalScan { table: v, columns: [v.cnt], cardinality: 0..=1 } | ||
batch_plan: |- | ||
BatchExchange { order: [v.cnt ASC], dist: Single } | ||
└─BatchSort { order: [v.cnt ASC] } | ||
└─BatchScan { table: v, columns: [v.cnt], distribution: Single } | ||
stream_plan: |- | ||
StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } | ||
└─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } | ||
- name: test functional dependency for order key pruning (index) | ||
sql: | | ||
create table t1 (v1 int, v2 int); | ||
create materialized view v as select count(*) cnt from t1; | ||
explain (verbose) create index mv_idx on v(cnt); | ||
explain_output: | | ||
StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [cnt], pk_conflict: NoCheck } | ||
└─StreamExchange { dist: HashShard(v.cnt) } | ||
└─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } | ||
- name: test functional dependency for order key pruning (index 2) | ||
sql: | | ||
create table t1 (v1 int primary key, v2 int); | ||
explain (verbose) create index mv_idx on t1(v1, v2); | ||
explain_output: | | ||
StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck } | ||
└─StreamExchange { dist: HashShard(t1.v1) } | ||
└─StreamTableScan { table: t1, columns: [t1.v1, t1.v2], stream_scan_type: ArrangementBackfill, stream_key: [t1.v1], pk: [v1], dist: UpstreamHashShard(t1.v1) } | ||
- name: test functional dependency for order key pruning (index 3) | ||
sql: | | ||
create table t1 (v1 int primary key, v2 int); | ||
explain (verbose) create index mv_idx on t1(v2, v1); | ||
explain_output: | | ||
StreamMaterialize { columns: [v2, v1], stream_key: [v1], pk_columns: [v2, v1], pk_conflict: NoCheck } | ||
└─StreamExchange { dist: HashShard(t1.v2) } | ||
└─StreamProject { exprs: [t1.v2, t1.v1] } | ||
└─StreamTableScan { table: t1, columns: [t1.v1, t1.v2], stream_scan_type: ArrangementBackfill, stream_key: [t1.v1], pk: [v1], dist: UpstreamHashShard(t1.v1) } |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not sure why prune TopN columns can also rewrite the BatchTopN 🤔 . Regardless, here it could be rewritten to
BatchLimit
instead I think.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed