Join GitHub today
GitHub is home to over 28 million developers working together to host and review code, manage projects, and build software together.
Sign upsql: add EXPLAIN(DISTSQL) support for subqueries #30857
Conversation
arjunravinarayan
requested review from
cockroachdb/distsql-prs
as
code owners
Oct 1, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
arjunravinarayan
requested a review
from cockroachdb/sql-opt-prs
as a
code owner
Oct 3, 2018
arjunravinarayan
changed the title from
WIP sql: support subqueries in EXPLAIN (DISTSQL)
to
sql: support subqueries in EXPLAIN (DISTSQL)
Oct 3, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
|
Ready for review. |
arjunravinarayan
requested a review
from
jordanlewis
Oct 3, 2018
RaduBerinde
reviewed
Oct 3, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/opt/exec/execbuilder/testdata/distsql_subquery, line 16 at r2 (raw file):
query T SELECT url FROM [EXPLAIN (DISTSQL) SELECT * FROM (VALUES (1, 2))]
This query doesn't have a subquery. Some of the others don't have one either (after optimization). I'd add the corresponding EXPLAIN (VERBOSE) in here so it's clear what plan we're looking at.
jordanlewis
requested changes
Oct 3, 2018
nit: please update the PR description to be like the commit message
Reviewed 2 of 2 files at r1.
Reviewable status:complete! 0 of 0 LGTMs obtained
pkg/sql/distsql_running.go, line 598 at r1 (raw file):
recv *DistSQLReceiver, maybeDistribute bool, ) (bool, []PhysicalPlan) {
this is counter to the Go idiom of having the bool in the second argument - I'd switch 'em. "contraidiomatic"
pkg/sql/explain_distsql.go, line 75 at r2 (raw file):
if len(n.subqueryPlans) > 0 { params.p.curPlan.subqueryPlans = n.subqueryPlans
Do we need to reset this after we finish running the subqueries?
pkg/sql/explain_distsql.go, line 96 at r2 (raw file):
) ok, _ := distSQLPlanner.PlanAndRunSubqueries(
wait, where do we use the second return value then, if not here?
pkg/sql/explain_distsql.go, line 105 at r2 (raw file):
) if !ok { return errors.New("error in planning subquery")
Can't you propagate the error here, like we do in the other place we do PlanAndRunSubqueries, by looking at recv.commErr?
pkg/sql/opt/exec/execbuilder/testdata/distsql_subquery, line 26 at r2 (raw file):
query T SELECT url FROM [EXPLAIN (DISTSQL) INSERT INTO abc (a, b, c) (SELECT a+1,b,c FROM abc)]
What about explain analyze(distsql) on a query with subqueries, does that work properly? I have a feeling it might crash, but I'm not sure, especially if you use a feature like placeholders.
arjunravinarayan
changed the title from
sql: support subqueries in EXPLAIN (DISTSQL)
to
sql: add EXPLAIN(DISTSQL) support for subqueries
Oct 3, 2018
arjunravinarayan
reviewed
Oct 3, 2018
PR description updated.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 75 at r2 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
Do we need to reset this after we finish running the subqueries?
No. in fact it is needed as this is how the outer plan looks up the values from the eval-ed subqueries.
pkg/sql/explain_distsql.go, line 96 at r2 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
wait, where do we use the second return value then, if not here?
Good catch. I suppose commit 1 is unnecessary. I was trying something with threading those plans through and printing all of them, but dropped that without dropping this refactor.
pkg/sql/explain_distsql.go, line 105 at r2 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
Can't you propagate the error here, like we do in the other place we do PlanAndRunSubqueries, by looking at
recv.commErr?
Done.
pkg/sql/opt/exec/execbuilder/testdata/distsql_subquery, line 16 at r2 (raw file):
Previously, RaduBerinde wrote…
This query doesn't have a subquery. Some of the others don't have one either (after optimization). I'd add the corresponding
EXPLAIN (VERBOSE)in here so it's clear what plan we're looking at.
I'll flesh out with more tests, and add EXPLAIN (VERBOSE) outputs as well.
pkg/sql/distsql_running.go, line 598 at r1 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
this is counter to the Go idiom of having the bool in the second argument - I'd switch 'em. "contraidiomatic"
Elided due to dropping the second argument as per your other suggestion. Thanks for the tip though. Noted!
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
jordanlewis
Oct 3, 2018
Member
Also, that lint failure about the logic test header is important - it is telling you (I think) that the logic tests aren't testing the distsql configuration, just the default local one... I think explain(distsql) still works in that circumstance but you should double check.
|
Also, that lint failure about the logic test header is important - it is telling you (I think) that the logic tests aren't testing the distsql configuration, just the default local one... I think explain(distsql) still works in that circumstance but you should double check. |
arjunravinarayan
requested a review
from cockroachdb/sql-rest-prs
as a
code owner
Oct 8, 2018
jordanlewis
requested review from
asubiotto and
solongordon
Oct 9, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
jordanlewis
Oct 9, 2018
Member
PTAL - fixed up the remaining issues with this. Subqueries are now displayed in EXPLAIN(DISTSQL) in their original form, pre-substitution - you'll see the actual query text. In EXPLAIN ANALYZE, the results of the subqueries will be displayed.
|
PTAL - fixed up the remaining issues with this. Subqueries are now displayed in EXPLAIN(DISTSQL) in their original form, pre-substitution - you'll see the actual query text. In EXPLAIN ANALYZE, the results of the subqueries will be displayed. |
asubiotto
approved these changes
Oct 10, 2018
Reviewed 1 of 3 files at r2, 6 of 9 files at r3.
Reviewable status:complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 87 at r3 (raw file):
if n.analyze && len(n.subqueryPlans) > 0 { // Discard rows that are returned. rw := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error {
Do you think we could reuse the result writer and receiver? We seem to do so in execWithDistSQLEngine
pkg/sql/explain_distsql.go, line 94 at r3 (raw file):
planCtx.ctx, rw, tree.Rows,
I wonder why we don't use n.statementType here and below.
pkg/sql/explain_distsql.go, line 107 at r3 (raw file):
params.p, func() *extendedEvalContext { ret := *params.extendedEvalCtx
Does this evalCtx factory behave differently than the one created in execWithDistSQLEngine? If so, is that an issue?
pkg/sql/logictest/testdata/planner_test/distsql_subquery, line 14 at r3 (raw file):
https://cockroachdb.github.io/distsqlplan/decode.html#eJyMj0FLxDAUhO_-ijKnXQmYXgNelIgFdaVdVJAcYvJYizUpeQkIS_-7bHMQD4LH-SaZmXdEiJ4e7Ccx1CtaGIE5RUfMMZ1QfdD5LygpMIa55BM2Ai4mgjoij3kiKDzZqRBfSAh4ynac1kTZXDUb2bj3Ej54C4GegqekGv3SDfthM-g7fb0_v-l39_bNPd_qXtvLdguzCMSSf-o42wNByUX8f1JPPMfA9GvS38lGgPyB6tkcS3L0mKJba6rcrf9W4IlzddsqulCtxSxn3wEAAP__zL5u_w==
Seems that the formatting is a bit messed up (no spaces), know what's up with that?
pkg/sql/logictest/testdata/planner_test/distsql_subquery, line 24 at r3 (raw file):
SELECT url FROM [EXPLAIN ANALYZE (DISTSQL) SELECT EXISTS (SELECT max(a) FROM abc)] ---- https://cockroachdb.github.io/distsqlplan/decode.html#eJyMj0FLAzEUhO_-imVOLQRMr7m1ngriSiteJIclGerimix5LyCU_e_SzUE8CB7nm2Rm3hUpRz4NnxS4N-zgDeaSA0VyuaH24Bi_4KzBmOaqN-wNQi6Eu0JHnQiH12GqlHsLg0gdxmlNtN2h29guvNf0IVsYnJgii-se9ueXjZbK_fnQ949b-MUgV_0pEB0uhLOL-f-IE2XOSfhrxN_J3oDxwnao5FoCn0sOa02T_fpvBZGizd01cUzNWvxy9x0AAP__guVrPA==
This one too
jordanlewis
reviewed
Oct 10, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 87 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Do you think we could reuse the result writer and receiver? We seem to do so in
execWithDistSQLEngine
Maybe, but unlike that code this code doesn't need to be super performant. I think recreating this stuff is more clear.
pkg/sql/explain_distsql.go, line 94 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I wonder why we don't use
n.statementTypehere and below.
It's because the statement type of a subquery is always tree.Rows. n.statementType isn't relevant for each of the subqueries.
pkg/sql/explain_distsql.go, line 107 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Does this evalCtx factory behave differently than the one created in execWithDistSQLEngine? If so, is that an issue?
It does. We don't have a handle on that one in here, unfortunately, and it would be hard to get one. This shallow copy seems to be fine in this case, but truthfully I'm not sure if things might break in more complex situations. Can you think of stuff that could be broken by it?
pkg/sql/logictest/testdata/planner_test/distsql_subquery, line 14 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
https://cockroachdb.github.io/distsqlplan/decode.html#eJyMj0FLxDAUhO_-ijKnXQmYXgNelIgFdaVdVJAcYvJYizUpeQkIS_-7bHMQD4LH-SaZmXdEiJ4e7Ccx1CtaGIE5RUfMMZ1QfdD5LygpMIa55BM2Ai4mgjoij3kiKDzZqRBfSAh4ynac1kTZXDUb2bj3Ej54C4GegqekGv3SDfthM-g7fb0_v-l39_bNPd_qXtvLdguzCMSSf-o42wNByUX8f1JPPMfA9GvS38lGgPyB6tkcS3L0mKJba6rcrf9W4IlzddsqulCtxSxn3wEAAP__zL5u_w==Seems that the formatting is a bit messed up (no spaces), know what's up with that?
That's done on purpose by the distsql plan printer in some cases. I agree it's weird but we should fix it in a different PR.
asubiotto
reviewed
Oct 10, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 87 at r3 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
Maybe, but unlike that code this code doesn't need to be super performant. I think recreating this stuff is more clear.
I was more concerned about cleanliness than performance
pkg/sql/explain_distsql.go, line 107 at r3 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
It does. We don't have a handle on that one in here, unfortunately, and it would be hard to get one. This shallow copy seems to be fine in this case, but truthfully I'm not sure if things might break in more complex situations. Can you think of stuff that could be broken by it?
It seems to do some extra placeholder stuff. Maybe try running a query with placeholders?
jordanlewis
reviewed
Oct 11, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 87 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I was more concerned about cleanliness than performance
What do you mean re-use? I'd like to take your suggestion, but I'm not sure how.
pkg/sql/explain_distsql.go, line 107 at r3 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
It seems to do some extra placeholder stuff. Maybe try running a query with placeholders?
It's doing that because the default evalCtx factory doesn't set the placeholders. Here, we're taking a shallow copy of the extendedEvalCtx that does already have placeholders set.
That being said, I was able to construct a query (PREPARE x AS EXPLAIN ANALYZE (DISTSQL) ...) that fails when adding a placeholder. Good call. Fixed and added tests.
asubiotto
approved these changes
Oct 11, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/explain_distsql.go, line 87 at r3 (raw file):
Previously, jordanlewis (Jordan Lewis) wrote…
What do you mean re-use? I'd like to take your suggestion, but I'm not sure how.
Hmm I don't know. It would be nice if you could put both the subquery and main query execution in one if statement but it seems like that wouldn't work. You might be able to use a closure to at least reduce the amount of code. Or have a nil recv and rw in the outer scope that are created when n.analyze but that seems dirty.
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
|
TFTR! Going to merge as-is. bors r+ |
bot
pushed a commit
that referenced
this pull request
Oct 11, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
craig
bot
commented
Oct 11, 2018
Build succeeded |
arjunravinarayan commentedOct 1, 2018
•
edited by jordanlewis
This currently does not provide the plans for the subqueries
themselves, just the outer query. But this no longer pops up the old
error message of "subqueries not supported", preventing a UX problem
where some queries are explainable and some are not.
Fixes #28729.
Release note: None