Skip to content

sql: introduce proto with metadata needed by fetchers, use it in row.Fetcher#75633

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
RaduBerinde:index-fetch-spec
Feb 1, 2022
Merged

sql: introduce proto with metadata needed by fetchers, use it in row.Fetcher#75633
craig[bot] merged 2 commits intocockroachdb:masterfrom
RaduBerinde:index-fetch-spec

Conversation

@RaduBerinde
Copy link
Copy Markdown
Member

descpb: introduce IndexFetchSpec

This commit introduces a new IndexFetchSpec proto which efficiently
encodes the necessary metadata to fetch a set of columns from an
index. The goal is to use this instead of serializing the entire
TableDescriptor when we execute a distributed scan.

Release note: None

sql: use IndexFetchSpec in row.Fetcher

In this commit, most of the row.Fetcher code is modified to internally
use an IndexFetchSpec instead of the table and index descriptor.

The remaining use is around decoding keys when converting errors.

One interesting thing to note here is that the row fetcher no longer
relies on the caller to "fix up" inverted column types; we do this
automatically now when creating the IndexFetchSpec.

Release note: None

@RaduBerinde RaduBerinde requested review from a team as code owners January 27, 2022 22:36
@RaduBerinde RaduBerinde requested review from a team and samiskin and removed request for a team January 27, 2022 22:36
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

The goal is to use this instead of serializing the entire
TableDescriptor when we execute a distributed scan.

Wouldn't it be even better to just serialize a descriptor ID, version, timestamp triple? #74352 That doesn't feel very far off.

Do you have bigger dreams for this thing? If it's just about optimizing distsql messages, I'm far from sold. I've heard you suggest crazier push down ideas. Is this part of that?

// inverted and we fetch the inverted key, the corresponding Column contains the
// inverted column type.
func InitIndexFetchSpec(
s *descpb.IndexFetchSpec,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I get why you're doing this, but would it be better to make this a method on catalog.TableDescriptor and cache them on the leased descriptor wrapper objects? Seems like this thing is immutable and avoiding this re-computation and allocations seems good. As of #74722 we generally get a handle to the leased descriptor

Copy link
Copy Markdown
Member Author

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

The bigger dream is to revive Jordan's prototype where KV builds the columnar batches (#52863). There we are talking about many more, smaller KV operations (max ~10MB of data scanned at a time); serializing and deserializing the table descriptor would be a much bigger problem than in DistSQL today (where it essentially happens once per query per node). As for having the descriptor available by ID/version on the KV side, that seems very problematic in multi-tenant (as well as architecturally).

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @samiskin, and @yuzefovich)


pkg/sql/rowenc/index_fetch.go, line 31 at r2 (raw file):

Previously, ajwerner wrote…

I get why you're doing this, but would it be better to make this a method on catalog.TableDescriptor and cache them on the leased descriptor wrapper objects? Seems like this thing is immutable and avoiding this re-computation and allocations seems good. As of #74722 we generally get a handle to the leased descriptor

Most of the spec (other than the fetched columns) is indeed immutable and we could have it cached in the descriptor. On the other hand, we're only talking about populating some fields and allocating one slice (KeyColumns) so it's not very pressing.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

As for having the descriptor available by ID/version on the KV side, that seems very problematic in multi-tenant (as well as architecturally).

We're aligned that the ID/Version in the spec thing is just for sql pods. I just wanted to confirm that you're not doing this work to diminish the number of allocations on the sql-pod side in distsql (as is mentioned in the commit messages).

There we are talking about many more, smaller KV operations (max ~10MB of data scanned at a time); serializing and deserializing the table descriptor would be a much bigger problem than in DistSQL today (where it essentially happens once per query per node).

No doubt. I think what I'd do is:

  1. box this widdled down spec in bytes in roachpb protos to sidestep import sadness
  2. cache the fully serialized spec (and deserialized) spec in the catalog.TableDescriptor object we cache in individual sql pods (as suggested).
  3. on the kvserver-side inject interface that takes those bytes and gives you back some column batch builder thingy that you can pass to MVCC. That interface internally can have a cache of serialized spec to deserialized things.
  4. (optional) do the (ID/Version) thing in distsql specs

That way, in the common case, nobody is ever serializing or deserializing a spec or table descriptor ~anywhere. Just shuttling some bytes as a cache key. And, as you noted, your widdled down IndexFetchSpec is going to be pretty small.

Seems like all of this can be done without too too much in the way of hideous boundary violation or cost.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @ajwerner, @samiskin, and @yuzefovich)

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Nice! I only have nits, but it'd probably be good to get an approval from someone from Schema too. :lgtm:

Reviewed 7 of 7 files at r1, 9 of 9 files at r2, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @ajwerner, @RaduBerinde, and @samiskin)


pkg/sql/catalog/descpb/index_fetch.go, line 1 at r1 (raw file):

// Copyright 2017 The Cockroach Authors.

nit: s/2017/2022/.


pkg/sql/catalog/descpb/index_fetch.proto, line 26 at r1 (raw file):

  message Column {
    optional string name = 1 [(gogoproto.nullable) = false];
    optional uint32 column_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "ColumnID", (gogoproto.casttype) = "ColumnID"];

nit: long line.


pkg/sql/catalog/descpb/index_fetch.proto, line 33 at r1 (raw file):

    optional sql.sem.types.T type = 3;

    // IsNonNullable indicates that it would be an corruption error if we ever

nit: s/an/a/.


pkg/sql/catalog/descpb/index_fetch.proto, line 78 at r1 (raw file):

                                     (gogoproto.casttype) = "IndexDescriptorEncodingType"];

  // NumKeySuffixColumns returns the number of suffix columns (corresponding to

nit: "returns" sounds odd.


pkg/sql/row/fetcher.go, line 82 at r2 (raw file):

	// One value per column that is part of the key; each value is a column
	// index (into FetchedColumns); -1 if we don't need the value for that column.

nit: "cols" is also mentioned in the comments on neededValueColsByIdx and colIdxMap.


pkg/sql/row/fetcher.go, line 868 at r2 (raw file):

		}

		if DebugRowFetch {

nit: do we not want to keep it?


pkg/sql/row/fetcher.go, line 1122 at r2 (raw file):

				for _, idx := range table.indexColIdx {
					if idx != -1 {
						indexColValues = append(indexColValues, table.row[idx].String(table.spec.FetchedColumns[idx].Type))

nit: we probably can use col.Type here.


pkg/sql/rowenc/index_encoding.go, line 511 at r2 (raw file):

			return nil, false, err
		}
		if vals[j].IsNull() {

nit: it should be slightly faster with

foundNull = foundNull || vals[j].IsNull()

pkg/sql/rowenc/testdata/index-fetch, line 71 at r1 (raw file):

# Primary index scan, not all columns.
index-fetch
table: t 

nit: trailing spaces.

This commit introduces a new IndexFetchSpec proto which efficiently
encodes the necessary metadata to fetch a set of columns from an
index. The goal is to use this instead of serializing the entire
TableDescriptor when we execute a distributed scan.

Release note: None
In this commit, most of the row.Fetcher code is modified to internally
use an IndexFetchSpec instead of the table and index descriptor.

The remaining use is around decoding keys when converting errors.

One interesting thing to note here is that the row fetcher no longer
relies on the caller to "fix up" inverted column types; we do this
automatically now when creating the `IndexFetchSpec`.

Release note: None
Copy link
Copy Markdown
Member Author

@RaduBerinde RaduBerinde left a comment

Choose a reason for hiding this comment

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

TFTR!

it'd probably be good to get an approval from someone from Schema too

@ajwerner sounds like you are ok with this?

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @samiskin, and @yuzefovich)


pkg/sql/row/fetcher.go, line 868 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: do we not want to keep it?

I'm not convinced there's much value to these (KV traces have more info).


pkg/sql/row/fetcher.go, line 1122 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: we probably can use col.Type here.

It's a different index

Copy link
Copy Markdown
Member

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewed 3 of 12 files at r3, 9 of 9 files at r4, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @RaduBerinde, and @samiskin)


pkg/sql/row/fetcher.go, line 868 at r2 (raw file):

Previously, RaduBerinde wrote…

I'm not convinced there's much value to these (KV traces have more info).

I think I agree - let's just remove it altogether then.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

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

@ajwerner sounds like you are ok with this?

Indeed

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @RaduBerinde, and @samiskin)

@RaduBerinde
Copy link
Copy Markdown
Member Author

CI failure is a test flake.

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 31, 2022

Build failed (retrying...):

@craig craig bot merged commit 73b6819 into cockroachdb:master Feb 1, 2022
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Feb 1, 2022

Build succeeded:

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.

4 participants