Skip to content

changefeedccl: Scope CDC expression using optimizer#93979

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
miretskiy:optexpr
Jan 10, 2023
Merged

changefeedccl: Scope CDC expression using optimizer#93979
craig[bot] merged 2 commits intocockroachdb:masterfrom
miretskiy:optexpr

Conversation

@miretskiy
Copy link
Copy Markdown
Contributor

@miretskiy miretskiy commented Dec 20, 2022

Changefeed expressions are different from regular
SELECT statements in that they only target one column
family at a time. That means, for example, that *
expands differently for CDC expression, and that expression
itself cannot reference columns outside of the target
column family.

Prior to this change, this logic was enforced by CDC
itself via FROM clause mangling, where FROM tbl would
be replaced with FROM (SELECT c1, c2, ... FROM tbl) AS tbl.

This approach worked okay; however, it is better
to teach optimizer about column family targetting needed by CDC.
The reason why it is better to have optimizer be responsible for
this is that name resolution and star expansion are very tricky
to get right.
In particular, prior to this PR, CDC expressions with table-typed
tuples (CREATE CHANGEFEED ... AS SELECT rides FROM rides)
did not work, since CDC was not smart enough to know
that the rides in the SELECT clause is a table-typed tuple.

This PR teaches optimizer how to handle CDC expressions, targeted
to a single column family. This is accomplished by extending
SQL grammar for index flags so that target column family can be
specified:
CREATE CHANGEFEED ... AS SELECT * FROM rides@{FAMILY=[0]}
Then, the catalog.TableDescriptor presented to the optimizer
is wrapped so that only to the columns available in
the target column family are visible to the optimizer.

Informs #90442
Fixes #82461
Informs #90260
Epic: CRDB-17161

--
Second commit:

Support system columns in CDC expressions.
System columns, such as crdb_internal_mvcc_timestamp are exposed
to CDC expressions. These columns are normally hidden,
but can be explicitly accessed:

CREATE CHANGEFEED ... AS
SELECT *, crdb_internal_mvcc_timestamp AS mvcc FROM rides

These system columns are also available in the cdc_prev tuple.
This makes it possible to e.g. determine the age of the event:

CREATE CHANGEFEED ... AS
SELECT
 crdb_internal_mvcc_timestamp - cdc_prev.crdb_internal.mvcc_timestamp AS age
FROM rides

Fixes #90442
Epic: CRDB-17161

Release note (enterprise change): Changefeed expressions support
system columns.

Release note (enterprise change): Improve changefeed expressions
code to rely on optimizer to evaluate star expansion.

@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@miretskiy miretskiy changed the title Optexpr changefeedccl: Scope CDC expression using optimizer] Dec 20, 2022
@miretskiy miretskiy changed the title changefeedccl: Scope CDC expression using optimizer] changefeedccl: Scope CDC expression using optimizer Dec 20, 2022
@miretskiy miretskiy marked this pull request as ready for review December 20, 2022 16:18
@miretskiy miretskiy requested a review from a team as a code owner December 20, 2022 16:18
@miretskiy miretskiy requested a review from a team December 20, 2022 16:18
@miretskiy miretskiy requested a review from a team as a code owner December 20, 2022 16:18
@miretskiy miretskiy force-pushed the optexpr branch 3 times, most recently from 8a7db18 to ed7bf36 Compare December 20, 2022 22:06
Copy link
Copy Markdown
Contributor

@HonoreDB HonoreDB left a comment

Choose a reason for hiding this comment

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

Reviewed 14 of 14 files at r1, 10 of 10 files at r3, 11 of 11 files at r4, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @mgartner and @miretskiy)


pkg/ccl/changefeedccl/event_processing.go line 322 at r4 (raw file):

	// Get prev value, if necessary.
	prevRow, err := func() (cdcevent.Row, error) {
		if keyOnly || !c.details.Opts.GetFilters().WithDiff {

What about keyOnly feeds with references to cdc_prev in the WHERE clause?


pkg/ccl/changefeedccl/cdcevent/event.go line 560 at r4 (raw file):

// to make the order of system columns clear and explicit.
// In particular, when decoding previous row, we strip table OID column
// since it makes little sense to include it in the previous row value.

Can you say more about why this is worth doing? I get that it'll always be the same in prev and current, but what's the harm?


pkg/sql/distsql_plan_changefeed.go line 389 at r1 (raw file):

// CDC currently supports primary index only.
// TODO(yevgeniy): We should be able to use secondary indexes provided
// the CDC expression access only the columns available in that secondary index.

nonblocking: Occurs to me just now that we'll need to be opt-in about this to avoid customers shooting themselves in the foot by altering/dropping a secondary index they didn't realize a changefeed was running on, which would probably need to trigger a replan and partial backfill.


pkg/sql/parser/sql.y line 12054 at r1 (raw file):

     $$.val = &tree.IndexFlags{ZigzagIndexIDs: []tree.IndexID{tree.IndexID($4.int64())}}
  }
| FAMILY '=' '[' iconst64 ']'

Would it cause ambiguities if we made this FAMILY <familyname> to be consistent with other places we specify a family (column definitions and regular changefeeds)? This could be a mildly useful bit of syntax to actually surface and support (at point) for people who want to ensure certain statements are only touching one family.

@miretskiy miretskiy requested a review from HonoreDB December 20, 2022 23:37
Copy link
Copy Markdown
Contributor Author

@miretskiy miretskiy left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @HonoreDB and @mgartner)


pkg/ccl/changefeedccl/event_processing.go line 322 at r4 (raw file):

Previously, HonoreDB (Aaron Zinger) wrote…

What about keyOnly feeds with references to cdc_prev in the WHERE clause?

Good point!


pkg/ccl/changefeedccl/cdcevent/event.go line 560 at r4 (raw file):

Previously, HonoreDB (Aaron Zinger) wrote…

Can you say more about why this is worth doing? I get that it'll always be the same in prev and current, but what's the harm?

It just looked so weird. There is really no other reason except that it makes little sense to me.
I of course, used to put it unconditionally, but felt it was annoying enough that I wanted it gone.
In reality, I wanted to have the same rules applied to cdc_prev as to the regular star expansion. Alas, that's not that simple, and so I settled on this to remove at least part of the thing that's annoying to me.
If you feel strongly, I can bring back old behavior.


pkg/sql/distsql_plan_changefeed.go line 389 at r1 (raw file):

Previously, HonoreDB (Aaron Zinger) wrote…

nonblocking: Occurs to me just now that we'll need to be opt-in about this to avoid customers shooting themselves in the foot by altering/dropping a secondary index they didn't realize a changefeed was running on, which would probably need to trigger a replan and partial backfill.

I don't think drop of a secondary index would need to trigger backfill. Once plan is made (and we only use primary index for this), sec index does not matter..


pkg/sql/parser/sql.y line 12054 at r1 (raw file):

Previously, HonoreDB (Aaron Zinger) wrote…

Would it cause ambiguities if we made this FAMILY <familyname> to be consistent with other places we specify a family (column definitions and regular changefeeds)? This could be a mildly useful bit of syntax to actually surface and support (at point) for people who want to ensure certain statements are only touching one family.

I don't think it would; I was thinking of keeping the name, but realized that it's not necessary since I never serialize this representation anyway -- family index flag added dynamically once we know which family we have the event for.

@miretskiy
Copy link
Copy Markdown
Contributor Author

@HonoreDB can I have another pass on this? @mgartner let me know if you have any q's re this pr.

@mgartner
Copy link
Copy Markdown
Contributor

Apologies for the delay. Please wait until I have a chance to take a look before merging.

@miretskiy
Copy link
Copy Markdown
Contributor Author

Ack. No worries

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.

This approach seems to work well, but I do have a concern about VIRTUAL computed columns.

How does CDC currently work with VIRTUAL computed columns? Those don't belong to any families, so with this approach we'd never be able to access them.

Reviewed 5 of 14 files at r1, 1 of 10 files at r3, 2 of 13 files at r6, 13 of 13 files at r8, 10 of 11 files at r9, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @HonoreDB and @miretskiy)


pkg/ccl/changefeedccl/cdceval/expr_eval.go line 234 at r9 (raw file):

) ([]*types.T, catalog.TableColMap, error) {
	numCols := len(ed.ResultColumns()) + len(colinfo.AllSystemColumnDescs)
	inputTypes := make([]*types.T, numCols)

You need the length to be len(ed.ResultColumns()) and the capacity to be numCols, right?


pkg/ccl/changefeedccl/cdceval/expr_eval.go line 242 at r9 (raw file):

		}
		inputCols.Set(col.GetID(), i)
		inputTypes[i] = c.Typ

nit: consider using append here so that you can make the length 0 above and the capacity numCols - feels simpler and less error-prone.


pkg/ccl/changefeedccl/cdceval/validation.go line 52 at r8 (raw file):

	// Configure index flags to restrict access to specific column family.
	sc := *n.SelectClause
	sc.From.Tables = append(tree.TableExprs(nil), n.SelectClause.From.Tables...)

nit: Why use append here rather than sc.From.Tables = tree.TableExprs{&tree.AliasedTableExpr{ ... }}?


pkg/ccl/changefeedccl/cdceval/validation.go line 55 at r8 (raw file):

	sc.From.Tables[0] = &tree.AliasedTableExpr{
		Expr:       n.SelectClause.From.Tables[0],
		IndexFlags: &tree.IndexFlags{FamilyID: &n.desc.FamilyID},

These should really be renamed to something like Hints instead of IndexFlags. No need to change that in this PR though. I created #94647 to track this.


pkg/sql/opt/cat/catalog.go line 79 at r8 (raw file):

	// IndexFlag contain index flags specified when accessing table (if any).
	IndexFlags *tree.IndexFlags

The only "index flag" that is needed is a family, so it seems like overkill to bring in all of these flags.


pkg/sql/opt/optbuilder/util.go line 638 at r8 (raw file):

// the fully qualified name.
func (b *Builder) resolveDataSource(
	tn *tree.TableName, flags cat.Flags,

Do we need all of cat.Flags here? Can we just pass the privilege and family, if there is one?

Copy link
Copy Markdown
Contributor Author

@miretskiy miretskiy left a comment

Choose a reason for hiding this comment

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

Your concerns about VIRTUAL computed columns is valid. As it stands right now, virtual columns
are not supported by CDC, and we have issues/todo's to address that.
So, as far as this PR is concerned, status quo is maintained.

We would have to do something along the lines of walking virtual computed column expressions
and determining if it's valid in the context of the target family. Or perhaps, we just let optimizer
error out and figure out what to do in this case (figure out from the UX perspective).

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @HonoreDB and @mgartner)


pkg/ccl/changefeedccl/cdceval/expr_eval.go line 234 at r9 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

You need the length to be len(ed.ResultColumns()) and the capacity to be numCols, right?

Done.


pkg/ccl/changefeedccl/cdceval/expr_eval.go line 242 at r9 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

nit: consider using append here so that you can make the length 0 above and the capacity numCols - feels simpler and less error-prone.

Done.


pkg/ccl/changefeedccl/cdceval/validation.go line 52 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

nit: Why use append here rather than sc.From.Tables = tree.TableExprs{&tree.AliasedTableExpr{ ... }}?

Thanks for calling this out. Arguably, this should have been done in the other PR (#94459) I have for review; but I think it makes sense to do it here. Basically, I don't want to mutate underlying selectClause in any way.
I left a comment to explain why.


pkg/sql/opt/cat/catalog.go line 79 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

The only "index flag" that is needed is a family, so it seems like overkill to bring in all of these flags.

Perhaps; but I think I do need to be able to tell if this option was set ; I would need to specify *FamilyID as a index flag -- which is fine, but also seems like a bit more work to set; i.e.
instead of:

ds, depName := b.resolveDataSourceRef(source,
			cat.Flags{RequiredPrivilege: privilege.SELECT, IndexFlags: indexFlags})

It would have to be something like:

familyIDFromIndexFlags := func() *FamilyID {
   if indexFlags != nil {
      return indexFlags.FamilyID
  }
  return nil
}
ds, depName := b.resolveDataSourceRef(source,
			cat.Flags{RequiredPrivilege: privilege.SELECT, FamilyID: familyIDFromIndexFlags()})

So..... I guess, how strongly do you feel about this.


pkg/sql/opt/optbuilder/util.go line 638 at r8 (raw file):

Previously, mgartner (Marcus Gartner) wrote…

Do we need all of cat.Flags here? Can we just pass the privilege and family, if there is one?

Your question/request is a perfectly valid one -- normally, that'd be my preference too.
There are two reasons why I chose to pass cat.Flags:

  1. This method makes a call to ds, resName, err := b.catalog.ResolveDataSource(b.ctx, flags, tn) which takes flags anyway (catalog.Catalog interface)
  2. I felt that passing in *FamilyID would be pretty strange considering that (at least right now) CDC is the only user.

(I lied: it's more of a 3 reasons, but 3rd one is more of a pet peeve -- I don't like adding nil /* familyID */ nil arguments with comment at call sites -- I think cat.Flags{} is explicit, clearer, and doesn't require silly trailing comments).

If I haven't convinced you and you feel strongly about this, please let me know.

miretskiy pushed a commit to miretskiy/cockroach that referenced this pull request Jan 4, 2023
Previous PRs (cockroachdb#93979) moved
the logic to scope changefeed expressions to specific column family
to the optimizer.

This had a side effect, causing CDC expression evaluator
to re-initialize its plan every time event for a different family
was received.  This PR fixes this problem by having evaluator
cache per-family state.

Epic: CRDB-17161

Release note: None
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 9, 2023

This PR was included in a batch that successfully built, but then failed to merge into master. It will not be retried.

Additional information:

@miretskiy
Copy link
Copy Markdown
Contributor Author

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 9, 2023

Build failed (retrying...):

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 9, 2023

Build failed:

@miretskiy
Copy link
Copy Markdown
Contributor Author

Bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 9, 2023

Build failed (retrying...):

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 9, 2023

Build failed (retrying...):

Yevgeniy Miretskiy added 2 commits January 9, 2023 19:22
Changefeed expressions are different from regular
`SELECT` statements in that they only target one column
family at a time.  That means, for example, that `*`
expands differently for CDC expression, and that expression
itself cannot reference columns outside of the target
column family.

Prior to this change, this logic was enforced by CDC
itself via `FROM` clause mangling, where `FROM tbl` would
be replaced with `FROM (SELECT c1, c2, ... FROM tbl) AS tbl`.

This approach worked okay; however, it is better
to teach optimizer about column family targetting needed by CDC.
The reason why it is better to have optimizer be responsible for
this is that name resolution and star expansion are very tricky
to get right.
In particular, prior to this PR, CDC expressions with table-typed
tuples (`CREATE CHANGEFEED ... AS SELECT rides FROM rides`)
did not work, since CDC was not smart enough to know
that the `rides` in the `SELECT` clause is a table-typed tuple.

This PR extends sql grammar, by adding a new index flag, so that CDC can cpecify
expressions targeted to a single column family:

```
CREATE CHANGEFEED ... AS SELECT * FROM rides@{FAMILY=[0]}
```

Then, the `catalog.TableDescriptor` presented to the optimizer
is wrapped so that only to the columns available in
the target column family are visible to the optimizer.

Informs cockroachdb#90442
Fixes cockroachdb#82461
Informs cockroachdb#90260
Epic: CRDB-17161

Release note (enterprise change): Improve changefeed expressions
code to rely on optimizer to evaluate star expansion.
Support system columns in CDC expressions.
System columns, such as `crdb_internal_mvcc_timestamp` are exposed
to CDC expressions.  These columns are normally hidden,
but can be explicitly accessed:

```
CREATE CHANGEFEED ... AS
SELECT *, crdb_internal_mvcc_timestamp AS mvcc FROM rides
```

These system columns are also available in the `cdc_prev` tuple.
This makes it possible to e.g. determine the age of the event:

```
CREATE CHANGEFEED ... AS
SELECT
 crdb_internal_mvcc_timestamp - cdc_prev.crdb_internal.mvcc_timestamp AS age
FROM rides
```

Fixes cockroachdb#90442
Fixes cockroachdb#83189

Epic: CRDB-17161

Release note (enterprise change): Changefeed expressions support
system columns.
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 10, 2023

Canceled.

@miretskiy
Copy link
Copy Markdown
Contributor Author

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 10, 2023

This PR was included in a batch that successfully built, but then failed to merge into master. It will not be retried.

Additional information:

@miretskiy
Copy link
Copy Markdown
Contributor Author

Bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 10, 2023

Build failed:

@miretskiy
Copy link
Copy Markdown
Contributor Author

Bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 10, 2023

Build succeeded:

@craig craig bot merged commit 0cf6cc8 into cockroachdb:master Jan 10, 2023
miretskiy pushed a commit to miretskiy/cockroach that referenced this pull request Jan 11, 2023
Previous PRs (cockroachdb#93979) moved
the logic to scope changefeed expressions to specific column family
to the optimizer.

This had a side effect, causing CDC expression evaluator
to re-initialize its plan every time event for a different family
was received.  This PR fixes this problem by having evaluator
cache per-family state.

Epic: CRDB-17161

Release note: None
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.

cdc: crdb_internal_mvcc_timestamp unusable in cdc expressions Changefeedccl: meaning the of *

4 participants