Skip to content

distsql: support tuples#25860

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
rjnn:distsql_tuple
Jun 4, 2018
Merged

distsql: support tuples#25860
craig[bot] merged 1 commit intocockroachdb:masterfrom
rjnn:distsql_tuple

Conversation

@rjnn
Copy link
Copy Markdown
Contributor

@rjnn rjnn commented May 23, 2018

Release note (feature): using tuples in a query no longer reverts you
to single node local SQL execution.

@rjnn rjnn requested review from a team May 23, 2018 17:35
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@rjnn rjnn requested a review from solongordon May 23, 2018 17:36
@rjnn rjnn force-pushed the distsql_tuple branch 4 times, most recently from 5ff1783 to 4d28ed0 Compare May 23, 2018 18:49
@RaduBerinde
Copy link
Copy Markdown
Member

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):

func getTupleKeyLength(buf []byte) (int, error) {
	return 0, errors.Errorf("TODO(donotmerge)")

WIP?


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch 2 times, most recently from 7258329 to cd05ed0 Compare May 23, 2018 20:00
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented May 23, 2018

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…

WIP?

Removed. Cruft code that wasn't used. Thanks.


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch from cd05ed0 to 219bc8b Compare May 23, 2018 20:01
rjnn pushed a commit to rjnn/cockroach that referenced this pull request May 23, 2018
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
@RaduBerinde
Copy link
Copy Markdown
Member

:lgtm:


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):

		return a.NewDOid(tree.MakeDOid(tree.DInt(i))), rkey, err
	default:
		if t, ok := valType.(types.TCollatedString); ok {

[nit] this block could be moved to a second switch t := valType.(type)


pkg/sql/sqlbase/testutils.go, line 119 at r2 (raw file):

	case ColumnType_TUPLE:
		// TODO(arjun)
		return tree.DNull

This violates nullOk, surprising it doesn't break something


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch 2 times, most recently from 88114e7 to 9cd2d05 Compare May 23, 2018 20:43
@jordanlewis
Copy link
Copy Markdown
Member

I think this needs some testing. I would recommend adding tests to encoding.go, alongside the tests like TestValueEncodeDecodeBytes and so forth, following the pattern of encoding random tuples, decoding them, and asserting that the output's equal to the input.


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):

		{`sqrt(123.0) + 456`, tree.FmtParsable,
			`sqrt(123.0:::DECIMAL) + 456:::DECIMAL`},
		{`row()`, tree.FmtParsable, `row()`},

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):

		panic(fmt.Sprintf("invalid datum type given: %s, expected %s",
			d.ResolvedType(), ptyp))
	}

Why did you remove this?


pkg/sql/sqlbase/encoded_datum.go, line 202 at r4 (raw file):

	}
	if err != nil {
		return errors.Wrapf(err, "error decoding % bytes", len(ed.encoded))

This format string probably should be %d :)


pkg/sql/sqlbase/table.go, line 1449 at r4 (raw file):

			for i, tupleType := range tupleTypes.Types {
				var err error
				dtuple.D[i], rkey, err = DecodeTableKey(a, tupleType, rkey, dir)

I'm confused about how this works. You encode with EncodeTableValue but decode with DecodeTableKey. How is this possible?

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):

			return decodeArray(a, typ.Typ, buf)
		case types.TTuple:
			return decodeTuple(a, typ, buf[1:])

why is this buf[1:]? what byte gets eaten? we don't seem to do that elsewhere.


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch 2 times, most recently from 1b15771 to 1919c1c Compare May 24, 2018 20:32
@petermattis
Copy link
Copy Markdown
Collaborator

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):

	case *tree.DTuple:
		scratch = scratch[0:0]
		// TODO(arjun): How do we eliminate this allocation?

Be kind to your future self and specify which allocation you're concerned about eliminating.

Seems like you're encoding a TupleValue as an encoded tuple that is then encoded again similarly to a bytes value. Did you consider the alternative of encoding a tuple value as a tag, a count of the number of items in the tuple and then the individual tuple values? I think with that encoding you'd be able to encode the tuple values directly into appendTo. Something like:

appendTo = encoding.EncodeValueTag(appendTo, uint32(colID), encoding.Tuple)
appendTo = encoding.EncodeNonsortingUvarint(appendTo, len(t.D))
for _, dd := range t.D {
  var err error
  appendTo, err = EncodeTableValue(appendTo, ColumnID(encoding.NoColumnID), dd, scratch)
  if err != nil {
    return nil, err
  }
}

Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch from 1919c1c to ff46161 Compare May 29, 2018 19:25
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented May 29, 2018

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…

this might need to be ROW() since we explicitly output that as the string.

Done.


pkg/sql/sqlbase/encoded_datum.go, line 142 at r4 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

Why did you remove this?

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…

This format string probably should be %d :)

Done.


pkg/sql/sqlbase/table.go, line 1436 at r2 (raw file):

Previously, RaduBerinde wrote…

[nit] this block could be moved to a second switch t := valType.(type)

Much cleaner! Thanks.


pkg/sql/sqlbase/table.go, line 1449 at r4 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

I'm confused about how this works. You encode with EncodeTableValue but decode with DecodeTableKey. How is this possible?

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

🙀 You are right! Will fix.


pkg/sql/sqlbase/table.go, line 1678 at r4 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

why is this buf[1:]? what byte gets eaten? we don't seem to do that elsewhere.

Removed, was bogus and wrong.


pkg/sql/sqlbase/table.go, line 731 at r6 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Be kind to your future self and specify which allocation you're concerned about eliminating.

Seems like you're encoding a TupleValue as an encoded tuple that is then encoded again similarly to a bytes value. Did you consider the alternative of encoding a tuple value as a tag, a count of the number of items in the tuple and then the individual tuple values? I think with that encoding you'd be able to encode the tuple values directly into appendTo. Something like:

appendTo = encoding.EncodeValueTag(appendTo, uint32(colID), encoding.Tuple)
appendTo = encoding.EncodeNonsortingUvarint(appendTo, len(t.D))
for _, dd := range t.D {
  var err error
  appendTo, err = EncodeTableValue(appendTo, ColumnID(encoding.NoColumnID), dd, scratch)
  if err != nil {
    return nil, err
  }
}

Wonderful. Done, thank you!


pkg/sql/sqlbase/testutils.go, line 119 at r2 (raw file):

Previously, RaduBerinde wrote…

This violates nullOk, surprising it doesn't break something

Fixed.


Comments from Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator

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):

	}

	result := tree.DTuple{

Do we need to add DatumAlloc.dtupleAlloc []tree.DTuple?


Comments from Reviewable

@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented May 29, 2018

pkg/sql/sqlbase/table.go, line 1559 at r7 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Do we need to add DatumAlloc.dtupleAlloc []tree.DTuple?

Sure, but the main allocation is the below line with make([]tree.Datum and that's hard to pre-allocate. Any thoughts on batching those allocations?


Comments from Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator

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…

Sure, but the main allocation is the below line with make([]tree.Datum and that's hard to pre-allocate. Any thoughts on batching those allocations?

DatumAlloc.datumAlloc []tree.Datum?


Comments from Reviewable

@rjnn rjnn requested a review from a team May 31, 2018 18:04
@petermattis
Copy link
Copy Markdown
Collaborator

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 needed to, empirically, as otherwise the value of Tuple was equal to the value of JSON.

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 iota with explicitly listing the values. That is:

const (
  Unknown Type = 0
  Null Type = 1
  NotNull Type = 2
...

Comments from Reviewable

@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented May 31, 2018

pkg/util/encoding/encoding.go, line 1060 at r9 (raw file):

Previously, petermattis (Peter Mattis) 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 iota with explicitly listing the values. That is:

const (
  Unknown Type = 0
  Null Type = 1
  NotNull Type = 2
...

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

@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented May 31, 2018

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…

mega nit: save the result of num*datumAllocMultipler so you don't have to put it twice

Done.


pkg/sql/sqlbase/table.go, line 1576 at r9 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

I don't understand what's going on here. What is the top level value you are referring to?

Also, when would this ever happen? This is saying that it's possible for the encoded number of elements (l) to differ from the number of expected type slots. I don't see how this has anything to do with NULL.

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…

Ah, so this is a special case for size-1 tuples that contain only null? Still, I don't understand why this would trigger a problem, and wouldn't for a size-n tuple that contains intermittent nulls, for example.

So if you are given a DNull, that is an acceptable value when you're expecting any Tuple. If you are given (1, NULL, 'foo'), that is only an acceptable value when you're expecting a tuple of type (int, any, string). In this case we are only checking if we are given a DNull. In this case, we won't see a tuple encoding header. If we are in the latter scenario, we'd see a tuple encoding header and the normal decoding path would work.


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch 2 times, most recently from c7bf7c5 to 1ae9772 Compare May 31, 2018 19:14
@jordanlewis
Copy link
Copy Markdown
Member

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…

So if you are given a DNull, that is an acceptable value when you're expecting any Tuple. If you are given (1, NULL, 'foo'), that is only an acceptable value when you're expecting a tuple of type (int, any, string). In this case we are only checking if we are given a DNull. In this case, we won't see a tuple encoding header. If we are in the latter scenario, we'd see a tuple encoding header and the normal decoding path would work.

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 DecodeTableValue, which peels off the type byte from an encoding and makes a decision. By the time control is passed to decodeTuple, we're 100% positive we're not decoding a NULL tuple.

The tests can be converted to use EncodeTableValue and DecodeTableValue to get around this.


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch 3 times, most recently from d875080 to 972a50c Compare June 4, 2018 18:23
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented Jun 4, 2018

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…

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 DecodeTableValue, which peels off the type byte from an encoding and makes a decision. By the time control is passed to decodeTuple, we're 100% positive we're not decoding a NULL tuple.

The tests can be converted to use EncodeTableValue and DecodeTableValue to get around this.

Done.


Comments from Reviewable

@jordanlewis
Copy link
Copy Markdown
Member

:lgtm: :shipit:


Review status: 12 of 19 files reviewed at latest revision, 2 unresolved discussions, some commit checks pending.


Comments from Reviewable

@rjnn rjnn force-pushed the distsql_tuple branch from 972a50c to 9b36fa7 Compare June 4, 2018 19:01
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented Jun 4, 2018

bors r+


Review status: 12 of 19 files reviewed at latest revision, 2 unresolved discussions, all commit checks successful.


Comments from Reviewable

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 4, 2018

Build failed (retrying...)

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 4, 2018

Build failed

@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented Jun 4, 2018

Build failures seem to be due to newly added logic tests that require rowsort. They weren't caught until the rebase. Nice work, bors, caught what would have turned into maddeningly flaky tests.

Will fix and retry.

@rjnn rjnn force-pushed the distsql_tuple branch from 9b36fa7 to c3c195c Compare June 4, 2018 20:41
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented Jun 4, 2018

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.
@rjnn rjnn force-pushed the distsql_tuple branch from c3c195c to 7751b46 Compare June 4, 2018 21:20
@rjnn
Copy link
Copy Markdown
Contributor Author

rjnn commented Jun 4, 2018

bors r+

craig bot pushed a commit that referenced this pull request Jun 4, 2018
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>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 4, 2018

Build succeeded

@craig craig bot merged commit 7751b46 into cockroachdb:master Jun 4, 2018
@rjnn rjnn deleted the distsql_tuple branch June 6, 2018 17:42
craig bot pushed a commit that referenced this pull request Jun 12, 2018
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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants