-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
sql: evaluate correlated subqueries as routines #95234
sql: evaluate correlated subqueries as routines #95234
Conversation
b788067
to
90dab90
Compare
90dab90
to
dd381d5
Compare
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.
This is so cool! It'll be really nice to see the end of these decorrelation issues. I just have a few questions + test requests.
Reviewed 1 of 1 files at r1, 1 of 3 files at r2, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @mgartner and @msirek)
pkg/sql/opt/exec/execbuilder/scalar.go
line 767 at r2 (raw file):
// Create a tree.RoutinePlanFn that can plan the statements in the UDF body. // TODO(mgartner): Add support for WITH expressions inside UDF bodies.
Does this mean there can't be any usage of WITH
in UDFs, or just that a UDF definition can't refer to an external WITH
clause?
pkg/sql/opt/exec/execbuilder/testdata/subquery
line 514 at r2 (raw file):
Scan /Table/110/1/2/0 Scan /Table/110/1/3/0 Scan /Table/110/1/4/0
Could you add some no-op cases that we still can't handle, with ANY
and EXISTS
? It also might be nice to have a regression test for one of the issues this is closing, unless doing that ends up being too much trouble.
Also, what happens if a correlated subquery involves a statement that can't (currently) be executed as a UDF, like a mutation? Can we have a test for that case? I guess other cases that might be interesting would be a correlated subquery with placeholders, with a WITH
statement inside it, and/or with a *
expression.
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.
Very nice change!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball and @mgartner)
pkg/sql/logictest/testdata/logic_test/subquery
line 506 at r2 (raw file):
The following NOT MATERIALIZED
case shows a discrepancy between CASE and non-CASE predicates:
statement ok
CREATE TABLE corr (
k INT PRIMARY KEY,
i INT
);
statement ok
CREATE TABLE corr2 (
k INT,
i INT
);
statement ok
INSERT INTO corr VALUES (1, 10), (2, 22), (3, 30), (4, 40), (5, 50);
-- This inserts 1 row into corr2 as the WITH clause is folded (inlined) into the main query
query II
WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k)
SELECT * FROM corr
WHERE k+1 = (SELECT DISTINCT k FROM insVals tmp WHERE k = corr.k) ;
----
-- If instead of the above query, the following with a CASE expression is run, 4 rows are inserted into corr2
query II
WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k)
SELECT * FROM corr
WHERE CASE WHEN k < 5 THEN k+1 = (SELECT DISTINCT k FROM insVals tmp WHERE k = corr.k) END;
----
Note, postgres inserts 1 row in both cases. From https://www.postgresql.org/docs/current/queries-with.html#id-1.5.6.12.7:
Data-modifying statements in WITH are executed exactly once, and always to completion, independently of whether the primary query reads all (or indeed any) of their output.
So, when executing the subquery, not by directly folding the WITH expression into the query, but as per-row UDF evaluation, we may now get different behavior depending on the form of the predicate.
We currently have the restriction:
You can use a data-modifying statement (INSERT, DELETE, etc.) as a common table expression, as long as the WITH clause containing the data-modifying statement is at the top level of the query.
This might be there to protect the user from unintentionally evaluating a data modification more than once. We might want to similarly protect the user from unexpected query behavior.
Maybe the answer is just to mimic the postgres behavior for data-modifying statements. Though that would require a documentation change.
dd381d5
to
1b59e3d
Compare
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.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball and @msirek)
pkg/sql/opt/exec/execbuilder/scalar.go
line 767 at r2 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
Does this mean there can't be any usage of
WITH
in UDFs, or just that a UDF definition can't refer to an externalWITH
clause?
CTE's defined within a UDF body are not currently allowed, see #92961. This is a limitation I'm hoping to lift soon.
A UDF should never be able to refer to a CTE outside it. The CTE wouldn't exist at the time of CREATE FUNCTION
, so it should be impossible to create such a UDF.
pkg/sql/logictest/testdata/logic_test/subquery
line 506 at r2 (raw file):
Previously, msirek (Mark Sirek) wrote…
The following
NOT MATERIALIZED
case shows a discrepancy between CASE and non-CASE predicates:statement ok CREATE TABLE corr ( k INT PRIMARY KEY, i INT ); statement ok CREATE TABLE corr2 ( k INT, i INT ); statement ok INSERT INTO corr VALUES (1, 10), (2, 22), (3, 30), (4, 40), (5, 50); -- This inserts 1 row into corr2 as the WITH clause is folded (inlined) into the main query query II WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k) SELECT * FROM corr WHERE k+1 = (SELECT DISTINCT k FROM insVals tmp WHERE k = corr.k) ; ---- -- If instead of the above query, the following with a CASE expression is run, 4 rows are inserted into corr2 query II WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k) SELECT * FROM corr WHERE CASE WHEN k < 5 THEN k+1 = (SELECT DISTINCT k FROM insVals tmp WHERE k = corr.k) END; ----Note, postgres inserts 1 row in both cases. From https://www.postgresql.org/docs/current/queries-with.html#id-1.5.6.12.7:
Data-modifying statements in WITH are executed exactly once, and always to completion, independently of whether the primary query reads all (or indeed any) of their output.
So, when executing the subquery, not by directly folding the WITH expression into the query, but as per-row UDF evaluation, we may now get different behavior depending on the form of the predicate.
We currently have the restriction:
You can use a data-modifying statement (INSERT, DELETE, etc.) as a common table expression, as long as the WITH clause containing the data-modifying statement is at the top level of the query.
This might be there to protect the user from unintentionally evaluating a data modification more than once. We might want to similarly protect the user from unexpected query behavior.
Maybe the answer is just to mimic the postgres behavior for data-modifying statements. Though that would require a documentation change.
Great catch! It looks like this is an existing problem with the InlineWith
rule, that causes this behavior even without the changes in this PR. For example, on v22.2.2:
defaultdb> CREATE TABLE corr ( k INT PRIMARY KEY, i INT);
CREATE TABLE
defaultdb> CREATE TABLE corr2 (k INT, i INT);
CREATE TABLE
defaultdb> INSERT INTO corr VALUES (1, 10), (2, 22), (3, 30), (4, 40), (5, 50);
INSERT 0 5
defaultdb> WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k)
SELECT * FROM corr
WHERE EXISTS (SELECT * FROM insVals WHERE k = corr.k UNION ALL SELECT k FROM corr c3 WHERE k = corr.k);
k | i
----+-----
1 | 10
2 | 22
3 | 30
4 | 40
5 | 50
(5 rows)
defaultdb> SELECT * FROM corr2;
k | i
----+-----
1 | 10
1 | 10
1 | 10
1 | 10
1 | 10
(5 rows)
I've created #95360 to track this.
This PR should not allow CTEs with mutations to be executed with routines because routines do not yet fully support mutations. I've added a check for this and a corresponding test.
pkg/sql/opt/exec/execbuilder/testdata/subquery
line 514 at r2 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
Could you add some no-op cases that we still can't handle, with
ANY
andEXISTS
? It also might be nice to have a regression test for one of the issues this is closing, unless doing that ends up being too much trouble.Also, what happens if a correlated subquery involves a statement that can't (currently) be executed as a UDF, like a mutation? Can we have a test for that case? I guess other cases that might be interesting would be a correlated subquery with placeholders, with a
WITH
statement inside it, and/or with a*
expression.
These are great ideas. Thanks for the suggestions!
- I added
<op> ANY
andEXISTS
tests to thesubquery
logic test file with TODOs. - I added a new logic test file with a regression test for the asyncpg issue.
- I've disallowed this change when the subquery has a mutation within it and added a test for that. Great catch!
- I've added a test where the subquery references placeholders and a
WITH
expression inside a subquery. - I've also added a test with star-expansion in the subquery. Star-expansion is currently disallowed in UDFs because the resolution of
*
to columns changes as columns in the underlying table change. But it should cause no issue in a subquery because anALTER TABLE ...
statement cannot be interleaved between evaluations of the subquery.
The logic for creating a planning closure for a `tree.Routine` has been moved to a helper function so that it can be reused in future commits. Release note: None
Previously, the optimizer would error in rare cases when it was unable to hoist correlated subqueries into apply-joins. Now, scalar, correlated subqueries that aren't hoisted are executed successfully. There is remaining work to apply the same method in this commit to `EXISTS` and `<op> ANY` subqueries. Hoisting correlated subqueries is not possible when a conditional expression, like a `CASE`, wraps a subquery that is not leak-proof. One of the effects of hoisting a subquery is that the subquery will be unconditionally evaluated. For leak-proof subqueries, the worst case is that unnecessary computation is performed. For non-leak-proof subqueries, errors could originate from the subquery when it should have never been evaluated because the corresponding conditional expression was never true. So, in order to support these cases, we must be able to execute a correlated subquery. A correlated subquery can be thought of as a relational expression with parameters that need to be filled in with constant value arguments for each invocation. It is essentially a user-defined function with a single statement in the function body. So, the `tree.RoutineExpr` machinery that powers UDFs is easily repurposed to facilitate evaluation of correlated subqueries. Fixes cockroachdb#71908 Fixes cockroachdb#73573 Fixes cockroachdb#80169 Release note (sql change): Some queries which previously resulted in the error "could not decorrelate subquery" now succeed.
1b59e3d
to
61233f0
Compare
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.
Reviewed 3 of 3 files at r3, 11 of 11 files at r4, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @DrewKimball and @mgartner)
pkg/sql/logictest/testdata/logic_test/subquery
line 506 at r2 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
Great catch! It looks like this is an existing problem with the
InlineWith
rule, that causes this behavior even without the changes in this PR. For example, on v22.2.2:defaultdb> CREATE TABLE corr ( k INT PRIMARY KEY, i INT); CREATE TABLE defaultdb> CREATE TABLE corr2 (k INT, i INT); CREATE TABLE defaultdb> INSERT INTO corr VALUES (1, 10), (2, 22), (3, 30), (4, 40), (5, 50); INSERT 0 5 defaultdb> WITH insVals AS NOT MATERIALIZED (INSERT INTO corr2 VALUES (1, 10) RETURNING k) SELECT * FROM corr WHERE EXISTS (SELECT * FROM insVals WHERE k = corr.k UNION ALL SELECT k FROM corr c3 WHERE k = corr.k); k | i ----+----- 1 | 10 2 | 22 3 | 30 4 | 40 5 | 50 (5 rows) defaultdb> SELECT * FROM corr2; k | i ----+----- 1 | 10 1 | 10 1 | 10 1 | 10 1 | 10 (5 rows)
I've created #95360 to track this.
This PR should not allow CTEs with mutations to be executed with routines because routines do not yet fully support mutations. I've added a check for this and a corresponding test.
Thanks!
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.
Reviewed 1 of 3 files at r2, 3 of 3 files at r3, 11 of 11 files at r4, all commit messages.
Reviewable status:complete! 2 of 0 LGTMs obtained (waiting on @mgartner)
pkg/sql/opt/exec/execbuilder/scalar.go
line 767 at r2 (raw file):
Previously, mgartner (Marcus Gartner) wrote…
CTE's defined within a UDF body are not currently allowed, see #92961. This is a limitation I'm hoping to lift soon.
A UDF should never be able to refer to a CTE outside it. The CTE wouldn't exist at the time of
CREATE FUNCTION
, so it should be impossible to create such a UDF.
Thanks!
TFTRs! bors r+ |
Build failed (retrying...): |
Build succeeded: |
Hello sir, I would like to ask if our optimization is necessary. In theory, we are not allowed to use exists and any associated subqueries in join. If you see it, I would very much like you to answer my question. thank you so much |
Sorry, I don't understand what you are asking. Are you running into issues due to this change? |
I apologize for the previous misunderstanding. Sir, I now think that this optimization should be done only when we need to use this correlated subquery. There are no true uncorrelated subqueries. is it sir. |
Are you concerned about the cost of this conversion in cases when the subquery is never evaluated? This conversion should be relatively cheap, and in most cases it's impossible to know at optimization-time whether or not the correlated subquery will need to be evaluated. There could be some ways to optimize queries with these correlated subqueries further, but I can see no downsides to this change. Any query plans it affects would have resulted in an error before. If you're having a problem with a particular query, please provide more details and we can take a look. |
ok thank you sir. Your answer is right and helped me understand the question |
opt: create tree.Routine planning closure in helper function
The logic for creating a planning closure for a
tree.Routine
has beenmoved to a helper function so that it can be reused in future commits.
Release note: None
sql: evaluate correlated subqueries as routines
Previously, the optimizer would error in rare cases when it was unable
to hoist correlated subqueries into apply-joins. Now, scalar, correlated
subqueries that aren't hoisted are executed successfully. There is
remaining work to apply the same method in this commit to
EXISTS
and<op> ANY
subqueries.Hoisting correlated subqueries is not possible when a conditional
expression, like a
CASE
, wraps a subquery that is not leak-proof. Oneof the effects of hoisting a subquery is that the subquery will be
unconditionally evaluated. For leak-proof subqueries, the worst case is
that unnecessary computation is performed. For non-leak-proof
subqueries, errors could originate from the subquery when it should have
never been evaluated because the corresponding conditional expression
was never true. So, in order to support these cases, we must be able to
execute a correlated subquery.
A correlated subquery can be thought of as a relational expression with
parameters that need to be filled in with constant value arguments for
each invocation. It is essentially a user-defined function with a single
statement in the function body. So, the
tree.RoutineExpr
machinerythat powers UDFs is easily repurposed to facilitate evaluation of
correlated subqueries.
Fixes #71908
Fixes #73573
Fixes #80169
Release note (sql change): Some queries which previously resulted in the
error "could not decorrelate subquery" now succeed.