Skip to content

sql: use type bytes for virtual inverted columns#58241

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
rytaft:vectorized2
Jan 6, 2021
Merged

sql: use type bytes for virtual inverted columns#58241
craig[bot] merged 1 commit intocockroachdb:masterfrom
rytaft:vectorized2

Conversation

@rytaft
Copy link
Copy Markdown
Collaborator

@rytaft rytaft commented Dec 23, 2020

This commit changes the column type for virtual inverted columns when
constructing scanNodes and TableReaderSpecs so that the type matches the data
type actually stored in the index. Prior to this commit, the type corresponded
to the column being indexed (e.g., geometry), rather than the actual type of
the key column (e.g., bytes). This is needed in order to enable vectorized
execution with the invertedFilterer processor.

Additionally, the column fetecher now treats virtual inverted columns
differently and dumps the data directly into a DBytes rather than attempting
to decode it.

Fixes #50695

Release note (performance improvement): Queries that use a geospatial inverted
index can now take advantage of vectorized execution for some parts of the
query plan, resulting in improved performance.

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@rytaft
Copy link
Copy Markdown
Collaborator Author

rytaft commented Dec 23, 2020

This PR is the result of rebasing #53202 and making a few tweaks:

  • Now the type of all inverted virtual columns is Bytes.
  • The cFetcher and index decoding functions are now aware of virtual inverted columns, and will dump the output directly into a DBytes instead of trying to decode it.

In order to get this working quickly, I added a hack for zig zag joins (it's called out in distsql_physical_planner.go). We'll need to do a bigger refactor of zig zag joins to get this to work without the hack, although I think we were going to need to do that anyway to support multi-column inverted indexes and integrate with the other work we've been doing in the optimizer to improve inverted index support. If this PR looks like the right approach, then I'll do the zig zag join refactor as a second commit in this PR.

@rytaft rytaft force-pushed the vectorized2 branch 6 times, most recently from 7a45d10 to 2592330 Compare December 23, 2020 16:24
@rytaft rytaft requested a review from a team as a code owner December 23, 2020 16:24
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.

Looks good to me, thanks for working on this!

Reviewed 26 of 26 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis, @RaduBerinde, and @rytaft)


pkg/sql/distsql_spec_exec_factory.go, line 240 at r1 (raw file):

		NeededColumns:    colCfg.wantedColumnsOrdinals,
	}
	if colCfg.virtualColumns != nil {

nit: this code is duplicated with distsql_physical_planner.


pkg/sql/catalog/colinfo/col_type_info.go, line 172 at r1 (raw file):

func GetColumnTypesFromColDescs(
	cols []descpb.ColumnDescriptor, columnIDs []descpb.ColumnID, outTypes []*types.T,
) ([]*types.T, error) {

nit: seems like err is always nil.


pkg/sql/rowexec/inverted_filterer.go, line 228 at r1 (raw file):

		// If the input is from the vectorized engine, the encoded bytes may be
		// empty.
		if row[ifr.invertedColIdx].Datum == nil {

Can we ever have a DNull value here?


pkg/sql/rowexec/rowfetcher.go, line 89 at r1 (raw file):

	}
	if virtualColumns != nil {
		tempCols := make([]descpb.ColumnDescriptor, len(cols), len(cols)+len(systemColumns))

nit: duplicated with colbatch_scan.go.

@rytaft rytaft force-pushed the vectorized2 branch 2 times, most recently from d1f5222 to 0df1ff8 Compare December 29, 2020 01:09
Copy link
Copy Markdown
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

TFTR, @yuzefovich! I'll work on adding the zig zag join commit now.

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


pkg/sql/distsql_spec_exec_factory.go, line 240 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: this code is duplicated with distsql_physical_planner.

Done.


pkg/sql/catalog/colinfo/col_type_info.go, line 172 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: seems like err is always nil.

Done.


pkg/sql/rowexec/inverted_filterer.go, line 228 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Can we ever have a DNull value here?

Shouldn't ever be DNull since nulls aren't stored in inverted indexes.


pkg/sql/rowexec/rowfetcher.go, line 89 at r1 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

nit: duplicated with colbatch_scan.go.

Done.

@rytaft rytaft force-pushed the vectorized2 branch 2 times, most recently from 15e1232 to 4fcf840 Compare December 30, 2020 18:36
@rytaft rytaft changed the title [DNM] sql: use type bytes for virtual inverted columns sql: use type bytes for virtual inverted columns Dec 30, 2020
@rytaft rytaft force-pushed the vectorized2 branch 2 times, most recently from 43131fe to 1bea5bd Compare December 30, 2020 19:00
Copy link
Copy Markdown
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

I've realized that the zigzag join refactor should be in a separate PR. I've found a way to remove the hack I added without needing to do the refactor. This is ready for review.

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

Copy link
Copy Markdown
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed 7 of 26 files at r1, 7 of 9 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis, @RaduBerinde, @rytaft, and @yuzefovich)


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

	// virtualColumns maps a subset of wantedColumns that are virtual to the
	// column type actually stored in the index. For example, the key column

nit: how about "inverted column" instead of "key column" since we now have multi-column inverted indexes?


pkg/sql/execinfrapb/processors_sql.proto, line 150 at r2 (raw file):

  // stored in the table descriptor. For example, the key column in an inverted
  // index has a different type than the column it indexes in the base table.
  repeated sqlbase.ColumnDescriptor virtual_columns = 16;

wondering why this is a repeated field instead of being optional.


pkg/sql/rowexec/inverted_filterer.go, line 228 at r1 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

Shouldn't ever be DNull since nulls aren't stored in inverted indexes.

Could you add a code comment that states this.


pkg/sql/rowexec/inverted_filterer.go, line 236 at r2 (raw file):

			return ifrStateUnknown, ifr.DrainHelper()
		}
		enc = []byte(*row[ifr.invertedColIdx].Datum.(*tree.DBytes))

This is a peculiar hack and I think deserves a longer comment above the if-block. Something like:
// NB: Inverted columns are custom encoded in a manner that does not correspond to Datum encoding, and in the code here we only want the encoded bytes. We have two possibilities with what the provider of this row has done:
// - not decoded the row: This is the len(enc) > 0 case.
// - decoded the row, but special-cased the inverted column by stuffing the encoded bytes into a "decoded" DBytes: This is the len(enc) == 0 case.


pkg/sql/rowexec/inverted_joiner.go, line 463 at r2 (raw file):

		}
		idx := ij.colIdxMap.GetDefault(ij.invertedColID)
		encInvertedVal := scannedRow[idx].EncodedBytes()

If the rowFetcher used by invertedJoiner had a columnar implementation, we would need to make a similar change here, yes?

Copy link
Copy Markdown
Contributor

@mgartner mgartner left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 20 of 26 files at r1, 9 of 9 files at r2.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @jordanlewis, @RaduBerinde, @rytaft, and @yuzefovich)

Copy link
Copy Markdown
Member

@jordanlewis jordanlewis left a comment

Choose a reason for hiding this comment

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

:lgtm: mod comments, thanks so much for taking care of this!

Reviewable status: :shipit: complete! 2 of 0 LGTMs obtained (waiting on @RaduBerinde, @rytaft, and @yuzefovich)


pkg/sql/catalog/tabledesc/structured.go, line 4019 at r2 (raw file):

// bool is true. If virtualCols is non-nil, substitutes the type of the virtual
// column instead of the table column with the same ID.
func (desc *Immutable) ColumnTypesWithMutationsAndVirtualCols(

cc @ajwerner @postamar do we need to be adding stuff like this to interfaces now too?

Copy link
Copy Markdown
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

TFTRs!

Based on @sumeerbhola's observation below, I think there is some unnecessary complexity in this implementation. I'm going to try a quick experiment to see if I can remove some of the complexity.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @mgartner, @RaduBerinde, @rytaft, @sumeerbhola, and @yuzefovich)


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

Previously, sumeerbhola wrote…

nit: how about "inverted column" instead of "key column" since we now have multi-column inverted indexes?

Done.


pkg/sql/execinfrapb/processors_sql.proto, line 150 at r2 (raw file):

Previously, sumeerbhola wrote…

wondering why this is a repeated field instead of being optional.

In theory there could be multiple virtual columns. We have other virtual column types (e.g., virtual computed), but I don't think there is a use case for them to be added here, so this is probably unnecessary.

I'm going to do a quick experiment to see if I can simplify this using the knowledge that (1) there will be at most one virtual inverted column, and (2) it always has type bytes.


pkg/sql/rowexec/inverted_filterer.go, line 228 at r1 (raw file):

Previously, sumeerbhola wrote…

Could you add a code comment that states this.

Done.


pkg/sql/rowexec/inverted_filterer.go, line 236 at r2 (raw file):

Previously, sumeerbhola wrote…

This is a peculiar hack and I think deserves a longer comment above the if-block. Something like:
// NB: Inverted columns are custom encoded in a manner that does not correspond to Datum encoding, and in the code here we only want the encoded bytes. We have two possibilities with what the provider of this row has done:
// - not decoded the row: This is the len(enc) > 0 case.
// - decoded the row, but special-cased the inverted column by stuffing the encoded bytes into a "decoded" DBytes: This is the len(enc) == 0 case.

Done.


pkg/sql/rowexec/inverted_joiner.go, line 463 at r2 (raw file):

Previously, sumeerbhola wrote…

If the rowFetcher used by invertedJoiner had a columnar implementation, we would need to make a similar change here, yes?

Yep, added a comment.

@postamar
Copy link
Copy Markdown

postamar commented Jan 5, 2021

func (desc *Immutable) ColumnTypesWithMutationsAndVirtualCols(

cc @ajwerner @postamar do we need to be adding stuff like this to interfaces now too?

I believe so.

This commit changes the column type for virtual inverted columns when
constructing scanNodes and TableReaderSpecs so that the type matches the data
type actually stored in the index. Prior to this commit, the type corresponded
to the column being indexed (e.g., geometry), rather than the actual type of
the key column (e.g., bytes). This is needed in order to enable vectorized
execution with the invertedFilterer processor.

Additionally, the column fetecher now treats virtual inverted columns
differently and dumps the data directly into a DBytes rather than attempting
to decode it.

Fixes cockroachdb#50695

Release note (performance improvement): Queries that use a geospatial inverted
index can now take advantage of vectorized execution for some parts of the
query plan, resulting in improved performance.
Copy link
Copy Markdown
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

I changed the virtual columns slice to be a pointer to a single column descriptor instead. I think it's improved things slightly. PTAL.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @ajwerner, @mgartner, @postamar, @RaduBerinde, @sumeerbhola, and @yuzefovich)


pkg/sql/catalog/tabledesc/structured.go, line 4019 at r2 (raw file):

Previously, jordanlewis (Jordan Lewis) wrote…

cc @ajwerner @postamar do we need to be adding stuff like this to interfaces now too?

I changed this to use wrapper instead of Immutable.


pkg/sql/execinfrapb/processors_sql.proto, line 150 at r2 (raw file):

Previously, rytaft (Rebecca Taft) wrote…

In theory there could be multiple virtual columns. We have other virtual column types (e.g., virtual computed), but I don't think there is a use case for them to be added here, so this is probably unnecessary.

I'm going to do a quick experiment to see if I can simplify this using the knowledge that (1) there will be at most one virtual inverted column, and (2) it always has type bytes.

Changed it to use optional.

Copy link
Copy Markdown
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

:lgtm:

I didn't carefully read the side-effects of this change on the sql package, since I am not very familiar with that code.

Reviewed 3 of 26 files at r1, 3 of 5 files at r3, 14 of 15 files at r4.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @ajwerner, @postamar, @RaduBerinde, @sumeerbhola, and @yuzefovich)

Copy link
Copy Markdown
Collaborator Author

@rytaft rytaft left a comment

Choose a reason for hiding this comment

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

TFTRs!

bors r+

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

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 6, 2021

Build succeeded:

@craig craig bot merged commit 43baeed into cockroachdb:master Jan 6, 2021
@RaduBerinde
Copy link
Copy Markdown
Member

Congrats on figuring this out!

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.

opt: change the key column of inverted indexes to use a new column id and type

8 participants