distsql: support tuples#25860
Conversation
5ff1783 to
4d28ed0
Compare
|
Nice to close on this. What is the versioning story? Seems like we should at least bump the distsql version no? Review status: 0 of 11 files reviewed at latest revision, all discussions resolved, some commit checks failed. pkg/util/encoding/encoding.go, line 2189 at r1 (raw file):
WIP? Comments from Reviewable |
7258329 to
cd05ed0
Compare
|
Good point, this requires a version bump. Added. Review status: 0 of 13 files reviewed at latest revision, 1 unresolved discussion, some commit checks failed. pkg/util/encoding/encoding.go, line 2189 at r1 (raw file): Previously, RaduBerinde wrote…
Removed. Cruft code that wasn't used. Thanks. Comments from Reviewable |
Use the distsql distinct processor for processing local sql queries. This is a work-in-progress to demonstrate and begin a discussion around the shims used. This commit is rebased on top of cockroachdb#25860, so only evaluate this commit. The main interfaces on which I'd like discussion are RowSourceToPlanNode and PlanNodeToRowSource. The rest of this commit is some planning plumbing, and code removal in local distinct. Release note: none
|
Review status: 0 of 12 files reviewed at latest revision, all discussions resolved, some commit checks pending. pkg/sql/sqlbase/table.go, line 1436 at r2 (raw file):
[nit] this block could be moved to a second pkg/sql/sqlbase/testutils.go, line 119 at r2 (raw file):
This violates Comments from Reviewable |
88114e7 to
9cd2d05
Compare
|
I think this needs some testing. I would recommend adding tests to Review status: 0 of 14 files reviewed at latest revision, 2 unresolved discussions, some commit checks failed. pkg/sql/sem/tree/format_test.go, line 197 at r4 (raw file):
this might need to be ROW() since we explicitly output that as the string. pkg/sql/sqlbase/encoded_datum.go, line 142 at r4 (raw file):
Why did you remove this? pkg/sql/sqlbase/encoded_datum.go, line 202 at r4 (raw file):
This format string probably should be %d :) pkg/sql/sqlbase/table.go, line 1449 at r4 (raw file):
I'm confused about how this works. You encode with Now that I think about it, is this code even used? We never encode tuple types with key encoding. I have a feeling this stanza is dead code. If not, I'd love to know why it isn't pkg/sql/sqlbase/table.go, line 1678 at r4 (raw file):
why is this Comments from Reviewable |
1b15771 to
1919c1c
Compare
|
Review status: 0 of 14 files reviewed at latest revision, 7 unresolved discussions, some commit checks failed. pkg/sql/sqlbase/table.go, line 731 at r6 (raw file):
Be kind to your future self and specify which allocation you're concerned about eliminating. Seems like you're encoding a Comments from Reviewable |
|
Added some randomized testing, great idea @jordanlewis! Finding test failures, and debugging them now. Thank you. Review status: 0 of 14 files reviewed at latest revision, 8 unresolved discussions, some commit checks failed. pkg/sql/sem/tree/format_test.go, line 197 at r4 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
Done. pkg/sql/sqlbase/encoded_datum.go, line 142 at r4 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
It was making debugging difficult, so I temporarily removed it. Put back in. pkg/sql/sqlbase/encoded_datum.go, line 202 at r4 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
Done. pkg/sql/sqlbase/table.go, line 1436 at r2 (raw file): Previously, RaduBerinde wrote…
Much cleaner! Thanks. pkg/sql/sqlbase/table.go, line 1449 at r4 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
🙀 You are right! Will fix. pkg/sql/sqlbase/table.go, line 1678 at r4 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
Removed, was bogus and wrong. pkg/sql/sqlbase/table.go, line 731 at r6 (raw file): Previously, petermattis (Peter Mattis) wrote…
Wonderful. Done, thank you! pkg/sql/sqlbase/testutils.go, line 119 at r2 (raw file): Previously, RaduBerinde wrote…
Fixed. Comments from Reviewable |
|
The new encoding LGTM. Review status: 0 of 15 files reviewed at latest revision, 7 unresolved discussions, some commit checks failed. pkg/sql/sqlbase/table.go, line 1559 at r7 (raw file):
Do we need to add Comments from Reviewable |
|
pkg/sql/sqlbase/table.go, line 1559 at r7 (raw file): Previously, petermattis (Peter Mattis) wrote…
Sure, but the main allocation is the below line with Comments from Reviewable |
|
Review status: 0 of 15 files reviewed at latest revision, 8 unresolved discussions, some commit checks failed. pkg/sql/sqlbase/table.go, line 1559 at r7 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Comments from Reviewable |
|
Review status: 14 of 17 files reviewed at latest revision, 5 unresolved discussions, some commit checks pending. pkg/util/encoding/encoding.go, line 1060 at r9 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
I don't think the comments here does justice to the requirement: we can't change any of these values. I think it would be worthwhile to replace the use of Comments from Reviewable |
|
pkg/util/encoding/encoding.go, line 1060 at r9 (raw file): Previously, petermattis (Peter Mattis) wrote…
I agree; I will do so in this PR. And it needs to be moved to a proto in a subsequent PR. I've already lost hours. Comments from Reviewable |
|
Review status: 14 of 17 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed. pkg/sql/sqlbase/table.go, line 1099 at r9 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
Done. pkg/sql/sqlbase/table.go, line 1576 at r9 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
I've removed that comment since it seems to only cause confusion. pkg/sql/sqlbase/table.go, line 1582 at r9 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
So if you are given a Comments from Reviewable |
c7bf7c5 to
1ae9772
Compare
|
Review status: 12 of 17 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed. pkg/sql/sqlbase/table.go, line 1582 at r9 (raw file): Previously, arjunravinarayan (Arjun Narayan) wrote…
Recapping offline conversation: This stanza exists to prevent null tuples from being incorrectly decoded. However, this isn't a problem in practice - null values are already decoded separately up in The tests can be converted to use Comments from Reviewable |
d875080 to
972a50c
Compare
|
PTAL @jordanlewis Review status: 12 of 19 files reviewed at latest revision, 5 unresolved discussions, some commit checks pending. pkg/sql/sqlbase/table.go, line 1582 at r9 (raw file): Previously, jordanlewis (Jordan Lewis) wrote…
Done. Comments from Reviewable |
|
Review status: 12 of 19 files reviewed at latest revision, 2 unresolved discussions, some commit checks pending. Comments from Reviewable |
|
bors r+ Review status: 12 of 19 files reviewed at latest revision, 2 unresolved discussions, all commit checks successful. Comments from Reviewable |
Build failed (retrying...) |
Build failed |
|
Build failures seem to be due to newly added logic tests that require Will fix and retry. |
|
The bors is really paying its rent today, found #26402. Will rebase on top of that and retry once that merges. |
Closes cockroachdb#15938. Release note (performance improvement): using tuples in a query no longer reverts you to single node local SQL execution.
|
bors r+ |
25395: sql: show allocated memory in `SHOW SESSIONS` r=jordanlewis a=jordanlewis Release note (sql change): `SHOW SESSIONS` now includes the number of allocated bytes by the session. 25860: distsql: support tuples r=arjunravinarayan a=arjunravinarayan Release note (feature): using tuples in a query no longer reverts you to single node local SQL execution. Co-authored-by: Jordan Lewis <jordanthelewis@gmail.com> Co-authored-by: Arjun Narayan <arjun@cockroachlabs.com>
Build succeeded |
25873: sql: use distsql for local distinct r=jordanlewis a=arjunravinarayan Use the distsql distinct processor for processing local sql queries. This is a work-in-progress to demonstrate and begin a discussion around the shims used. This PR is rebased on top of #25860, so only evaluate the last commit. The main interfaces on which I'd like discussion are RowSourceToPlanNode and PlanNodeToRowSource. The rest of this PR is exporting changes (2nd commit), some planning plumbing, and code removal in local distinct. Co-authored-by: Arjun Narayan <arjun@cockroachlabs.com>
Release note (feature): using tuples in a query no longer reverts you
to single node local SQL execution.