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): runtime check for scalar subquery in batch queries #13880

Merged
merged 18 commits into from
Dec 20, 2023

Conversation

BugenZhao
Copy link
Member

@BugenZhao BugenZhao commented Dec 8, 2023

I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.

What's changed and what's your intention?

In #12908, a hack is introduced to allow a system query to be executed. We concluded that the support for unique keys may help to get rid of it. However, as discussed in #5019 and #1335, it appears that supporting runtime check for scalar subquery can be much easier to implement.

I reuse the plan nodes and executor for Limit to achieve that, not sure if it's a good practice.

Introduce a new plan node named MaxOneRow and its corresponding batch executor for doing this.

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • All checks passed in ./risedev check (or alias, ./risedev c)

Documentation

  • My PR needs documentation updates. (Please use the Release note section below to summarize the impact on users)

Release note

If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.

Signed-off-by: Bugen Zhao <[email protected]>
Signed-off-by: Bugen Zhao <[email protected]>
Signed-off-by: Bugen Zhao <[email protected]>
Signed-off-by: Bugen Zhao <[email protected]>
@BugenZhao BugenZhao marked this pull request as ready for review December 8, 2023 13:26
Signed-off-by: Bugen Zhao <[email protected]>
Copy link
Member

@xxchan xxchan left a comment

Choose a reason for hiding this comment

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

This sounds like an interesting idea. What's your motivation for doing this? (Did you meet some use cases, or just came up with this suddenly?!😄)

Copy link

codecov bot commented Dec 11, 2023

Codecov Report

Attention: 78 lines in your changes are missing coverage. Please review.

Comparison is base (1cacc07) 68.07% compared to head (46e2f6b) 68.00%.
Report is 13 commits behind head on main.

Files Patch % Lines
...ntend/src/optimizer/plan_node/batch_max_one_row.rs 0.00% 29 Missing ⚠️
...end/src/optimizer/plan_node/logical_max_one_row.rs 55.55% 24 Missing ⚠️
src/batch/src/executor/max_one_row.rs 65.00% 21 Missing ⚠️
...end/src/optimizer/plan_node/generic/max_one_row.rs 78.94% 4 Missing ⚠️
Additional details and impacted files
@@            Coverage Diff             @@
##             main   #13880      +/-   ##
==========================================
- Coverage   68.07%   68.00%   -0.07%     
==========================================
  Files        1548     1552       +4     
  Lines      267474   267653     +179     
==========================================
- Hits       182075   182018      -57     
- Misses      85399    85635     +236     
Flag Coverage Δ
rust 68.00% <65.93%> (-0.07%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@BugenZhao
Copy link
Member Author

What's your motivation for doing this?

It's #12908. 😄

@chenzl25
Copy link
Contributor

I am considering whether we should use a dedicated operator e.g. MaxOneRow to abstract this semantic instead of coupling with the Limit.
Pros: we can use Limit more easily without caring about the exceeding_limit restrictions.

@BugenZhao
Copy link
Member Author

I am considering whether we should use a dedicated operator e.g. MaxOneRow to abstract this semantic instead of coupling with the Limit. Pros: we can use Limit more easily without caring about the exceeding_limit restrictions.

Refactored the implementation. PTAL again. 🥰

@BugenZhao BugenZhao requested review from xxchan and stdrc December 19, 2023 06:49
Copy link
Member

@stdrc stdrc left a comment

Choose a reason for hiding this comment

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

So the subquery is executed during the optimization phase?

src/frontend/src/optimizer/plan_node/logical_join.rs Outdated Show resolved Hide resolved

impl ToBatch for LogicalMaxOneRow {
fn to_batch(&self) -> Result<PlanRef> {
todo!()
Copy link
Member

Choose a reason for hiding this comment

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

todo?

Copy link
Member Author

Choose a reason for hiding this comment

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

Oops. Forget to change this after reimplementation.

Signed-off-by: Bugen Zhao <[email protected]>
Signed-off-by: Bugen Zhao <[email protected]>
Copy link
Member

@xxchan xxchan left a comment

Choose a reason for hiding this comment

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

Generally LGTM.

proto/batch_plan.proto Show resolved Hide resolved
src/batch/src/executor/max_one_row.rs Outdated Show resolved Hide resolved

#[derive(Default)]
pub struct CheckApplyElimination {
result: CheckResult,
Copy link
Member

Choose a reason for hiding this comment

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

nit: Why not simply put a Result<(), RwError> here

Copy link
Member Author

Choose a reason for hiding this comment

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

We have weight on different error messages to provide better error reporting. 🥺

fn default_behavior() -> Self::DefaultBehavior {
Merge(std::cmp::max)
}

}

// Check if all `Apply`s are eliminated and the subquery is unnested.
plan.check_apply_elimination()?;
Copy link
Contributor

@chenzl25 chenzl25 Dec 19, 2023

Choose a reason for hiding this comment

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

After you removed the to_stream from logical join, we need to add a visitor (e.g. MaxOneRowFinder) to check whether dangling MaxOneRow operators exist for Streaming Query.

The example

create table t (a int);
explain create materialized view v as  select (select a from t) as c;

Copy link
Contributor

Choose a reason for hiding this comment

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

Otherwise, we will meet the error Not supported: streaming nested-loop join.

Copy link
Member Author

@BugenZhao BugenZhao Dec 19, 2023

Choose a reason for hiding this comment

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

You're right. So which approach do you think is better, call child.to_stream() first or check HasLogicalMaxOneRow before calling to_stream? In the latter approach, it seems LogicalMaxOneRow.to_stream() can be filled with unreachable.

Logical Rewrite For Stream:
 
 LogicalJoin { type: LeftOuter, on: true }
 ├─LogicalValues { rows: [[0:Int64]], schema: Schema { fields: [_row_id:Int64] } }
 └─LogicalMaxOneRow
   └─LogicalScan { table: t, columns: [a, _row_id] }
 
 ERROR: Not supported: streaming nested-loop join
 HINT: The non-equal join in the query requires a nested-loop join executor, which could be very expensive to run. Consider rewriting the query to use dynamic filter as a substitute if possible.
 See also: https://github.com/risingwavelabs/rfcs/blob/main/rfcs/0033-dynamic-filter.md

Copy link
Contributor

Choose a reason for hiding this comment

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

This approach looks better IMO: check HasLogicalMaxOneRow before calling to_stream.

@BugenZhao
Copy link
Member Author

So the subquery is executed during the optimization phase?

Not exactly. We now enable the checking of scalar subquery cardinality at runtime. Previously, only compile-time information was used, which resulted in some queries being unable to be planned.

@BugenZhao BugenZhao requested a review from chenzl25 December 20, 2023 02:11
Copy link
Contributor

@chenzl25 chenzl25 left a comment

Choose a reason for hiding this comment

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

LGTM. Thank you so much for this PR!

Copy link
Member

@stdrc stdrc left a comment

Choose a reason for hiding this comment

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

LGTM

@BugenZhao BugenZhao added this pull request to the merge queue Dec 20, 2023
Merged via the queue into main with commit 66f6bc0 Dec 20, 2023
7 of 9 checks passed
@BugenZhao BugenZhao deleted the bz/scalar-subquery-runtime-check branch December 20, 2023 04:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants