From b5cb99073a0dee584d7a4ca126f8a060cc5f4dd1 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Wed, 27 Jul 2022 19:16:39 -0400 Subject: [PATCH] changefeedccl: Rely on optimizer and distSQL for expressions evaluation Previous PRs (#82562) introduced CDC expressions. This PR builds on that and replaces majority of hand written evaluation logic in favor of tighter integration with optimizer and dist SQL processors. CDC expression, which is really a simplified `SELECT` statement, is now planned by the optimizer `sql.PlanCDCExpression`. The resulting plan is then fed to the distSQL, to produce a specialized CDC execution plan (`sql.RunCDCEvaluation`). The execution plan is special in that it is guaranteed to be a local execution plan, and changefeed is expected to "feed" the data (encoded row) directly into the execution pipeline, with change aggregators consuming resulting projection. The benefit of this approach is that expression optimization, and evaluation is now handled by optimizer and distSQL. The responsibility of CDC evaluation package is to make sure that CDC expression is "sane" and to setup CDC specific functions. Since the whole expression is not yet fully planned by distSQL (i.e. we don't have changefeed operator implemented yet), the integration between CDC expressions and optimizer/execInfra is not yet complete. In particular, this PR does not replace current distSQL execution for CDC -- namely, we still keep the same execution model using hand planned `ChangeFrontierProcessor` and `ChangeAggretagorProcessor`. It augments existing model, while tightening the integration. Still, this is an improvement over previous state. The follow on work will complete integration. Some changes enabled by this implementation include the replacement of `cdc_prev()` function which returned JSONb representation of the previous row, with a `cdc_prev` tuple. This makes changefeed expressions more natural since tuples are strongly typed: ``` SELECT * FROM tbl WHERE col != cdc_prev.col` ``` In addition, by using tuple to represent the state of the previous row, we can now leverage existing SQL functions. For example, to emit previus row as JSONb we can do: ``` SELECT *, row_to_json(cdc_prev) AS prevJson FROM tbl ``` Fixes #90416 Fixes #90714 Fixes #90455 Informs #90442 Informs CRDB-18978 Informs CRDB-17161 Release note (enterprise change): CDC expressions are now planned and evaluated using SQL optimizer and distSQL execution. The state of the previous row is now exposed as `cdc_prev` tuple. Release note (backward-incompatible change): The replacement of cdc_prev() function in favor a cdc_prev tuple is an incompatible change that may break changefeeds that use old cdc_prev() function. --- pkg/ccl/changefeedccl/BUILD.bazel | 1 - pkg/ccl/changefeedccl/bench_test.go | 9 +- pkg/ccl/changefeedccl/cdceval/BUILD.bazel | 22 +- pkg/ccl/changefeedccl/cdceval/constraint.go | 106 -- pkg/ccl/changefeedccl/cdceval/doc.go | 93 +- pkg/ccl/changefeedccl/cdceval/expr_eval.go | 930 +++++------------- .../changefeedccl/cdceval/expr_eval_test.go | 426 ++++---- .../changefeedccl/cdceval/func_resolver.go | 77 +- .../cdceval/func_resolver_test.go | 4 +- pkg/ccl/changefeedccl/cdceval/functions.go | 312 ++++-- .../changefeedccl/cdceval/functions_test.go | 265 ++--- pkg/ccl/changefeedccl/cdceval/parse.go | 28 +- pkg/ccl/changefeedccl/cdceval/plan.go | 145 +++ pkg/ccl/changefeedccl/cdceval/plan_test.go | 280 ++++++ pkg/ccl/changefeedccl/cdceval/validation.go | 493 ++++++---- .../changefeedccl/cdceval/validation_test.go | 224 +++-- pkg/ccl/changefeedccl/cdcevent/BUILD.bazel | 4 +- pkg/ccl/changefeedccl/cdcevent/event.go | 75 +- pkg/ccl/changefeedccl/cdcevent/event_test.go | 10 +- pkg/ccl/changefeedccl/cdcevent/projection.go | 39 +- .../changefeedccl/cdcevent/projection_test.go | 12 +- pkg/ccl/changefeedccl/changefeed_dist.go | 37 +- .../changefeedccl/changefeed_processors.go | 8 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 47 +- pkg/ccl/changefeedccl/changefeed_test.go | 129 +-- pkg/ccl/changefeedccl/event_processing.go | 109 +- pkg/jobs/registry.go | 6 - pkg/server/server_sql.go | 1 + pkg/sql/BUILD.bazel | 5 - pkg/sql/constraint.go | 206 ---- pkg/sql/constraint_test.go | 276 ------ pkg/sql/distsql_plan_changefeed.go | 44 +- pkg/sql/distsql_plan_changefeed_test.go | 21 + pkg/sql/execinfra/server_config.go | 3 + pkg/sql/job_exec_context.go | 19 - pkg/sql/opt/exec/execbuilder/builder.go | 34 +- pkg/sql/opt/norm/fold_constants_funcs.go | 21 +- pkg/sql/planhook.go | 1 - pkg/sql/sem/tree/function_name.go | 17 - 39 files changed, 2133 insertions(+), 2406 deletions(-) delete mode 100644 pkg/ccl/changefeedccl/cdceval/constraint.go create mode 100644 pkg/ccl/changefeedccl/cdceval/plan.go create mode 100644 pkg/ccl/changefeedccl/cdceval/plan_test.go delete mode 100644 pkg/sql/constraint.go delete mode 100644 pkg/sql/constraint_test.go diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 5effd956679a..203b806a3757 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -246,7 +246,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/sem/volatility", "//pkg/sql/sessiondata", - "//pkg/sql/sessiondatapb", "//pkg/sql/sqlliveness", "//pkg/sql/sqlliveness/sqllivenesstestutils", "//pkg/sql/tests", diff --git a/pkg/ccl/changefeedccl/bench_test.go b/pkg/ccl/changefeedccl/bench_test.go index 7e17fd45a32a..f09193ca05eb 100644 --- a/pkg/ccl/changefeedccl/bench_test.go +++ b/pkg/ccl/changefeedccl/bench_test.go @@ -27,8 +27,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" - "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -247,10 +248,10 @@ func createBenchmarkChangefeed( if err != nil { return nil, nil, err } - serverCfg := s.DistSQLServer().(*distsql.ServerImpl).ServerConfig - eventConsumer, err := newKVEventToRowConsumer(ctx, &serverCfg, nil, sf, initialHighWater, + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + eventConsumer, err := newKVEventToRowConsumer(ctx, &execCfg, sf, initialHighWater, sink, encoder, makeChangefeedConfigFromJobDetails(details), - execinfrapb.Expression{}, TestingKnobs{}, nil, nil) + execinfrapb.Expression{}, username.PublicRoleName(), TestingKnobs{}, nil, nil) if err != nil { return nil, nil, err diff --git a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel index af9c2910fd9c..7f8c7fad9023 100644 --- a/pkg/ccl/changefeedccl/cdceval/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdceval/BUILD.bazel @@ -4,12 +4,12 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "cdceval", srcs = [ - "constraint.go", "doc.go", "expr_eval.go", "func_resolver.go", "functions.go", "parse.go", + "plan.go", "validation.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval", @@ -19,27 +19,26 @@ go_library( "//pkg/ccl/changefeedccl/changefeedbase", "//pkg/clusterversion", "//pkg/jobs/jobspb", - "//pkg/keys", + "//pkg/kv", "//pkg/roachpb", + "//pkg/security/username", "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/schemaexpr", + "//pkg/sql/catalog/descs", + "//pkg/sql/execinfra", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/eval", - "//pkg/sql/sem/normalize", "//pkg/sql/sem/tree", "//pkg/sql/sem/volatility", - "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", "//pkg/sql/types", + "//pkg/util/ctxgroup", "//pkg/util/hlc", - "//pkg/util/json", - "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", "@com_github_lib_pq//oid", ], @@ -52,6 +51,7 @@ go_test( "func_resolver_test.go", "functions_test.go", "main_test.go", + "plan_test.go", "validation_test.go", ], args = ["-test.timeout=295s"], @@ -62,26 +62,28 @@ go_test( "//pkg/ccl/changefeedccl/cdctest", "//pkg/ccl/changefeedccl/changefeedbase", "//pkg/jobs/jobspb", + "//pkg/keys", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", "//pkg/server", - "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", - "//pkg/sql/distsql", - "//pkg/sql/parser", "//pkg/sql/randgen", "//pkg/sql/rowenc", + "//pkg/sql/rowenc/keyside", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", + "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/json", "//pkg/util/leaktest", diff --git a/pkg/ccl/changefeedccl/cdceval/constraint.go b/pkg/ccl/changefeedccl/cdceval/constraint.go deleted file mode 100644 index d019a1e16648..000000000000 --- a/pkg/ccl/changefeedccl/cdceval/constraint.go +++ /dev/null @@ -1,106 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package cdceval - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" -) - -// ConstrainPrimaryIndexSpanByFilter attempts to constrain table primary -// index span if changefeed expression (select clause) is specified. -// Returns possibly constrained spans, and a possibly modified (optimized) -// select clause. -func ConstrainPrimaryIndexSpanByFilter( - ctx context.Context, - execCtx sql.JobExecContext, - selectClause string, - descr catalog.TableDescriptor, - target jobspb.ChangefeedTargetSpecification, - includeVirtual bool, - keyOnly bool, -) (_ []roachpb.Span, updatedSelect string, _ error) { - if selectClause == "" { - return nil, "", errors.AssertionFailedf("unexpected empty filter") - } - sc, err := ParseChangefeedExpression(selectClause) - if err != nil { - return nil, "", pgerror.Wrap(err, pgcode.InvalidParameterValue, - "could not parse changefeed expression") - } - - ed, err := newEventDescriptorForTarget(descr, target, schemaTS(execCtx), includeVirtual, keyOnly) - if err != nil { - return nil, "", err - } - - evalCtx := &execCtx.ExtendedEvalContext().Context - spans, remainingFilter, err := constrainSpansBySelectClause( - ctx, execCtx, evalCtx, execCtx.ExecCfg().Codec, sc, ed) - if err != nil { - return nil, "", err - } - - if remainingFilter != nil { - // non-nil remainingFilter implies we had sc.Where clause. - if remainingFilter == tree.DBoolTrue { - sc.Where = nil - } else { - sc.Where.Expr = remainingFilter - } - } - - return spans, AsStringUnredacted(sc), nil -} - -// constrainSpansBySelectClause is a helper that attempts to constrain primary -// index spans by the filter in the select clause. Returns constrained spans, -// along with the remaining filter. -func constrainSpansBySelectClause( - ctx context.Context, - sc sql.SpanConstrainer, - evalCtx *eval.Context, - codec keys.SQLCodec, - selectClause *tree.SelectClause, - ed *cdcevent.EventDescriptor, -) ([]roachpb.Span, tree.Expr, error) { - // Predicate changefeed currently works on a single table only. - // Verify this assumption. - if len(selectClause.From.Tables) != 1 { - return nil, nil, errors.AssertionFailedf( - "expected 1 table expression, found %d", len(selectClause.From.Tables)) - } - - if selectClause.Where == nil { - // Nothing to constrain. - return []roachpb.Span{ed.TableDescriptor().PrimaryIndexSpan(codec)}, nil, nil - } - - tableName := tableNameOrAlias(ed.TableName, selectClause.From.Tables[0]) - semaCtx := newSemaCtxWithTypeResolver(ed) - return sc.ConstrainPrimaryIndexSpanByExpr( - ctx, sql.BestEffortConstrain, tableName, ed.TableDescriptor(), - evalCtx, semaCtx, selectClause.Where.Expr) -} - -func schemaTS(execCtx sql.JobExecContext) hlc.Timestamp { - return execCtx.ExecCfg().Clock.Now() -} diff --git a/pkg/ccl/changefeedccl/cdceval/doc.go b/pkg/ccl/changefeedccl/cdceval/doc.go index a909ae81c8c9..4bbb5a403738 100644 --- a/pkg/ccl/changefeedccl/cdceval/doc.go +++ b/pkg/ccl/changefeedccl/cdceval/doc.go @@ -11,47 +11,66 @@ package cdceval /*** cdceval package is a library for evaluating various expressions in CDC. -Namely, this package concerns itself with 3 things: - * Filter evaluation -- aka predicates: does the event match boolean expression. - * Projection evaluation: given the set of projection expressions, evaluate them. - * (Soon to be added) Evaluation of computed virtual columns. - -Evaluator is the gateway into the evaluation logic; it has 3 methods matching -the above use cases. Before filtering and projection can be used, Evaluator must -be configured with appropriate predicate and filtering expressions via configureProjection. - -If the Evaluator is not configured with configureProjection, then each event is assumed -to match filter by default, and projection operation is an identity operation returning input -row. - -Evaluator constructs a helper structure (exprEval) to perform actual evaluation. -One exprEval exists per cdcevent.EventDescriptor (currently, a new exprEval created -whenever event descriptor changes; we might have to add some sort of caching if needed). - -Evaluation of projections and filter expressions are identical. - -First, we have "compilation" phase: - 1. Expression is "walked" to resolve the names and replace those names with tree.IndexedVar expressions. - The "index" part of the indexed var refers to the "index" of the datum in the row. - (note however: the row is abstracted under cdcevent package). IndexedVar allows the value of that - variable to be bound later once it is known; it also associates the type information - with that variable. - 2. Expression is typed check to ensure that it is of the appropriate type. - * Projection expressions can be of tree.Any type, while filters must be tree.DBool. - 3. Expression is then normalized (constants folded, etc). - -It is an error to have a filter expression which evaluates to "false" -- in this case, Evaluator -will return a "contradiction" error. - -After expressions "compiled", they can be evaluated; and again, both projections and filters use the same -logic (evalExpr() function); basically, all IndexedVars are bound to the Datums in the updated row, and the -expression is evaluated to the appropriate target type. + +The expression evaluation and execution is integrated with planner/distSQL. + +First, when starting changefeed, CDC expression is planned and normalized +(NormalizeAndPlan). The normalization step involves various sanity checks +(ensuring for example that expression does not target multiple column familes). +The expression is planned as well because this step, by leveraging optimizer, +may simplify expressions, and, more importantly, can detect if the expression +will not match any rows (this results in an error being returned). + +The normalized expression is persisted into job record. +When changefeed job starts running, it once again plans expression execution. +Part of the planning stage figures out which spans need to be scanned. If the +predicate restricted primary index span, then we will scan only portion +of the table. + +Then, once aggregators start up, they will once again plan the expression +(sql.PlanCDCExpression), but this time each incoming KV event will be evaluated +by DistSQL to produce final result (projection). +PlanCDCExpression fully integrates with optimizer, and produces a plan +that can then be used to execute the "flow" via normal +distSQL mechanism (PlanAndRun). + +What makes CDC expression execution different is that CDC is responsible for +pushing the data into the execution pipeline. This is accomplished via +execinfra.RowReceiver which is returned as part of the plan. +CDC will receive rows (encoded datums) from rangefeed, and then "pushes" those +rows into execution pipeline. + +CDC then reads the resulting projection via distSQL result writer. + +Evaluator is the gateway into the evaluation logic; it takes care of running +execution flow, caching (to reuse the same plan as long as the descriptor version +does not change), etc. Expressions can contain functions. We restrict the set of functions that can be used by CDC. Volatile functions, window functions, aggregate functions are disallowed. Certain stable functions (s.a. now(), current_timestamp(), etc) are allowed -- they will always return the MVCC timestamp of the event. -We also provide custom, CDC specific functions, such as cdc_prev() which returns prevoius row as -a JSONB record. See functions.go for more details. + +Access to the previous state of the row is accomplished via (typed) cdc_prev tuple. +This tuple can be used to build complex expressions around the previous state of the row: + SELECT * FROM foo WHERE status='active' AND cdc_prev.status='inactive' + +During normalization stage, we determine if the expression has cdc_prev access. +If so, the expression is rewritten as: + SELECT ... FROM tbl, (SELECT ((crdb_internal.cdc_prev_row()).*)) AS cdc_prev +The crdb_internal.cdc_prev_row function is created to return a tuple based on +the previous table descriptor. Access to this function is arranged via custom +function resolver. + +In addition, prior to evaluating CDC expression, the WHERE clause is rewritten as: + SELECT where, ... FROM tbl, ... +That is, WHERE clause is turned into a boolean datum. When projection results are +consumed, we can determine if the row ought to be filtered. This step is done to +ensure that we correctly release resources for each event -- even the ones that +are filtered out. + +Virtual computed columns can be easily supported but currently are not. +To support virtual computed columns we must ensure that the expression in that +column references only the target changefeed column family. ***/ diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval.go b/pkg/ccl/changefeedccl/cdceval/expr_eval.go index 2f79c992c98b..53a6d207e230 100644 --- a/pkg/ccl/changefeedccl/cdceval/expr_eval.go +++ b/pkg/ccl/changefeedccl/cdceval/expr_eval.go @@ -14,704 +14,330 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/normalize" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/errors" - "github.com/lib/pq/oid" ) // Evaluator is a responsible for evaluating expressions in CDC. type Evaluator struct { - selectors []tree.SelectExpr - from tree.TableExpr - where tree.Expr - - evalCtx *eval.Context - // Current evaluator. Re-initialized whenever event descriptor - // version changes. - evaluator *exprEval -} + norm *NormalizedSelectClause -// NewEvaluator returns evaluator configured to process specified -// select expression. -func NewEvaluator( - ctx context.Context, evalCtx *eval.Context, sc *tree.SelectClause, -) (*Evaluator, error) { - e := &Evaluator{evalCtx: evalCtx.Copy()} + // Plan related state. + input execinfra.RowReceiver + planGroup ctxgroup.Group + errCh chan error + currDesc *cdcevent.EventDescriptor + prevDesc *cdcevent.EventDescriptor - if len(sc.From.Tables) > 0 { // 0 tables used only in tests. - if len(sc.From.Tables) != 1 { - return nil, errors.AssertionFailedf("expected 1 table, found %d", len(sc.From.Tables)) - } - e.from = sc.From.Tables[0] - } + // Execution context. + execCfg *sql.ExecutorConfig + user username.SQLUsername + fnResolver CDCFunctionResolver - if err := e.initSelectClause(ctx, sc); err != nil { - return nil, err - } + // rowCh receives projection datums. + rowCh chan tree.Datums + projection cdcevent.Projection - return e, nil -} - -// ComputeVirtualColumns updates row with computed values for all virtual columns. -func (e *Evaluator) ComputeVirtualColumns(ctx context.Context, row *cdcevent.Row) error { - return errors.AssertionFailedf("unimplemented yet") + // rowEvalCtx contains state necessary to evaluate expressions. + // updated for each row. + rowEvalCtx rowEvalContext } -// MatchesFilter returns true if row matches evaluator filter expression. -func (e *Evaluator) MatchesFilter( - ctx context.Context, updatedRow cdcevent.Row, mvccTS hlc.Timestamp, prevRow cdcevent.Row, -) (_ bool, err error) { - defer func() { - if pan := recover(); pan != nil { - err = changefeedbase.WithTerminalError( - errors.Newf("error while evaluating WHERE clause: %s", pan)) - } - }() - if e.where == nil { - return true, nil +// NewEvaluator constructs new evaluator for changefeed expression. +func NewEvaluator( + sc *tree.SelectClause, execCfg *sql.ExecutorConfig, user username.SQLUsername, +) (*Evaluator, error) { + e := Evaluator{ + execCfg: execCfg, + user: user, + norm: &NormalizedSelectClause{ + SelectClause: sc, + }, + rowCh: make(chan tree.Datums, 1), } - if err := e.initEval(ctx, updatedRow.EventDescriptor); err != nil { - return false, err - } + // Arrange to be notified when event does not match predicate. + predicateAsProjection(e.norm) - return e.evaluator.matchesFilter(ctx, updatedRow, mvccTS, prevRow) + return &e, nil } -// Projection performs evalProjection operation on the updated row. -// mvccTS is an mvcc timestamp of updated row, and prevRow may optionally contain -// the value of the previous row. -// Returns cdcevent.Row representing evalProjection. -func (e *Evaluator) Projection( - ctx context.Context, updatedRow cdcevent.Row, mvccTS hlc.Timestamp, prevRow cdcevent.Row, -) (_ cdcevent.Row, err error) { +// Eval evaluates projection for the specified updated and (optional) previous row. +// Returns projection result. If the filter does not match the event, returns +// "zero" Row. +func (e *Evaluator) Eval( + ctx context.Context, updatedRow cdcevent.Row, prevRow cdcevent.Row, +) (projection cdcevent.Row, evalErr error) { defer func() { - if pan := recover(); pan != nil { - err = changefeedbase.WithTerminalError( - errors.Newf("error while evaluating SELECT clause: %s", pan)) + if evalErr != nil { + // If we can't evaluate a row, we are bound to keep failing. + // So mark error permanent. + evalErr = changefeedbase.WithTerminalError(evalErr) } }() - if len(e.selectors) == 0 { - return updatedRow, nil - } - if err := e.initEval(ctx, updatedRow.EventDescriptor); err != nil { - return cdcevent.Row{}, err - } + if !e.sameVersion(updatedRow.EventDescriptor, prevRow.EventDescriptor) { + // Descriptor versions changed; re-initialize. + if err := e.closeErr(); err != nil { + return cdcevent.Row{}, err + } - return e.evaluator.evalProjection(ctx, updatedRow, mvccTS, prevRow) -} + e.errCh = make(chan error, 1) + e.currDesc, e.prevDesc = updatedRow.EventDescriptor, prevRow.EventDescriptor -// initSelectClause configures this evaluator to evaluate specified select clause. -func (e *Evaluator) initSelectClause(ctx context.Context, sc *tree.SelectClause) (err error) { - defer func() { - if pan := recover(); pan != nil { - err = changefeedbase.WithTerminalError( - errors.Newf("error while validating CHANGEFEED expression: %s", pan)) + if err := e.planAndRun(ctx); err != nil { + return cdcevent.Row{}, err } - }() - if len(sc.Exprs) == 0 { // Shouldn't happen, but be defensive. - return pgerror.New(pgcode.InvalidParameterValue, - "expected at least 1 projection") } - semaCtx := newSemaCtx() - e.selectors = sc.Exprs - for _, se := range e.selectors { - expr, err := validateExpressionForCDC(ctx, se.Expr, semaCtx) - if err != nil { - return err - } - se.Expr = expr + // Setup context. + if err := e.setupContextForRow(ctx, updatedRow, prevRow); err != nil { + return cdcevent.Row{}, err } - if sc.Where != nil { - expr, err := validateExpressionForCDC(ctx, sc.Where.Expr, semaCtx) - if err != nil { - return err - } - e.where = expr + // Push data into DistSQL. + if st := e.input.Push(updatedRow.EncDatums(), nil); st != execinfra.NeedMoreRows { + return cdcevent.Row{}, errors.Newf("evaluator shutting down due to status %s", st) } - return nil -} - -// initEval initializes evaluator for the specified event descriptor. -func (e *Evaluator) initEval(ctx context.Context, d *cdcevent.EventDescriptor) error { - if e.evaluator != nil { - sameVersion, sameTypes := d.EqualsWithUDTCheck(e.evaluator.EventDescriptor) - if sameVersion && sameTypes { - // Event descriptor and UDT types are the same -- re-use the same evaluator. - return nil + // Read the evaluation result. + select { + case <-ctx.Done(): + return cdcevent.Row{}, ctx.Err() + case err := <-e.errCh: + return cdcevent.Row{}, err + case row := <-e.rowCh: + if !tree.MustBeDBool(row[0]) { + // Filter did not match. + return cdcevent.Row{}, nil } + // Strip out temporary boolean value (result of the WHERE clause) + // since this information is not sent to the consumer. + row = row[1:] - if sameVersion { - // Here, we know that even though descriptor versions are the same, the - // check for equality with UDT type check failed. Thus, we know some user - // defined types have changed. - // The previously parsed select & filter expressions have type annotations, - // and those may now be incorrect. So, parse and re-initialize evaluator - // expressions. - var where *tree.Where - if e.where != nil { - where = tree.NewWhere(tree.AstWhere, e.where) + for i, d := range row { + if err := e.projection.SetValueDatumAt(i, d); err != nil { + return cdcevent.Row{}, err } - sc, err := ParseChangefeedExpression(AsStringUnredacted(&tree.SelectClause{ - From: tree.From{Tables: tree.TableExprs{e.from}}, - Exprs: e.selectors, - Where: where, - })) - if err != nil { - return err - } - if err := e.initSelectClause(ctx, sc); err != nil { - return err - } - // Fall through to re-create e.evaluator. } - } - - evaluator := newExprEval(e.evalCtx, d, tableNameOrAlias(d.TableName, e.from)) - for _, selector := range e.selectors { - if err := evaluator.addSelector(ctx, selector, len(e.selectors)); err != nil { - return err - } - } - - if err := evaluator.addFilter(ctx, e.where); err != nil { - return err - } - - e.evaluator = evaluator - return nil -} - -type exprEval struct { - *cdcevent.EventDescriptor - semaCtx *tree.SemaContext - evalCtx *eval.Context - - evalHelper *rowContainer // evalHelper is a container tree.IndexedVarContainer. - iVarHelper tree.IndexedVarHelper // iVarHelper helps create indexed variables bound to evalHelper. - resolver cdcNameResolver // resolver responsible for performing function name resolution. - - starProjection bool // Set to true if we have a single '*' projection. - selectors []tree.TypedExpr // set of expressions to evaluate when performing evalProjection. - projection cdcevent.Projection // cdcevent.Projects helps construct projection results. - filter tree.TypedExpr // where clause filter - - // keep track of number of times particular column name was used - // in selectors. Since the data produced by CDC gets converted - // to the formats (JSON, avro, etc.) that may not like having multiple - // fields named the same way, this map helps us unique-ify those columns. - nameUseCount map[string]int - - // rowEvalCtx contains state necessary to evaluate expressions. - // updated for each row. - rowEvalCtx rowEvalContext -} - -func newExprEval( - evalCtx *eval.Context, ed *cdcevent.EventDescriptor, tableName *tree.TableName, -) *exprEval { - cols := ed.ResultColumns() - e := &exprEval{ - EventDescriptor: ed, - semaCtx: newSemaCtxWithTypeResolver(ed), - evalCtx: evalCtx.Copy(), - evalHelper: &rowContainer{cols: cols}, - projection: cdcevent.MakeProjection(ed), - nameUseCount: make(map[string]int), - } - - evalCtx = nil // From this point, only e.evalCtx should be used. - - // Configure semantic context. - e.semaCtx.SearchPath = &sessiondata.DefaultSearchPath - e.semaCtx.FunctionResolver = &CDCFunctionResolver{} - e.semaCtx.Properties.Require("cdc", - tree.RejectAggregates|tree.RejectGenerators|tree.RejectWindowApplications|tree.RejectNestedGenerators, - ) - e.semaCtx.Annotations = tree.MakeAnnotations(cdcAnnotationAddr) - e.semaCtx.IVarContainer = e.evalHelper - - // Configure evaluation context. - e.evalCtx.Annotations = &e.semaCtx.Annotations - e.evalCtx.Annotations.Set(cdcAnnotationAddr, &e.rowEvalCtx) - e.evalCtx.IVarContainer = e.evalHelper - - // Extract colinfo.ResultColumn from cdcevent.ResultColumn - nakedResultColumns := func() (rc []colinfo.ResultColumn) { - rc = make([]colinfo.ResultColumn, len(cols)) - for i := 0; i < len(cols); i++ { - rc[i] = cols[i].ResultColumn - } - return rc - } - - e.iVarHelper = tree.MakeIndexedVarHelper(e.evalHelper, len(cols)) - e.resolver = cdcNameResolver{ - EventDescriptor: ed, - NameResolutionVisitor: schemaexpr.MakeNameResolutionVisitor( - colinfo.NewSourceInfoForSingleTable(*tableName, nakedResultColumns()), - e.iVarHelper, - ), - } - - return e -} - -// rowEvalContext represents the context needed to evaluate row expressions. -type rowEvalContext struct { - mvccTS hlc.Timestamp - updatedRow cdcevent.Row - prevRow cdcevent.Row - memo struct { - prevJSON tree.Datum - } -} - -// setupContext configures evaluation context with the provided row information. -func (e *exprEval) setupContext( - updatedRow cdcevent.Row, mvccTS hlc.Timestamp, prevRow cdcevent.Row, -) { - e.rowEvalCtx.updatedRow = updatedRow - e.rowEvalCtx.prevRow = prevRow - e.rowEvalCtx.mvccTS = mvccTS - e.evalCtx.TxnTimestamp = mvccTS.GoTime() - e.evalCtx.StmtTimestamp = mvccTS.GoTime() - - // Clear out all memo records - e.rowEvalCtx.memo.prevJSON = nil -} - -// evalProjection responsible for evaluating projection expression. -// Returns new projection Row. -func (e *exprEval) evalProjection( - ctx context.Context, updatedRow cdcevent.Row, mvccTS hlc.Timestamp, prevRow cdcevent.Row, -) (cdcevent.Row, error) { - if e.starProjection { - return updatedRow, nil - } - - e.setupContext(updatedRow, mvccTS, prevRow) - - for i, expr := range e.selectors { - d, err := e.evalExpr(ctx, expr, types.Any) + projection, err := e.projection.Project(updatedRow) if err != nil { return cdcevent.Row{}, err } - if err := e.projection.SetValueDatumAt(ctx, e.evalCtx, i, d); err != nil { - return cdcevent.Row{}, err - } + return projection, nil } - - return e.projection.Project(updatedRow) } -// matchesFilter returns true if row matches configured filter. -func (e *exprEval) matchesFilter( - ctx context.Context, updatedRow cdcevent.Row, mvccTS hlc.Timestamp, prevRow cdcevent.Row, -) (bool, error) { - if e.filter == nil { - return true, nil +// sameVersion returns true if row descriptor versions match this evaluator +// versions. Note: current, and previous maybe at different versions, but we +// don't really care about that. +func (e *Evaluator) sameVersion(curr, prev *cdcevent.EventDescriptor) bool { + if e.currDesc == nil { + return false } - - e.setupContext(updatedRow, mvccTS, prevRow) - d, err := e.evalExpr(ctx, e.filter, types.Bool) - if err != nil { - return false, err - } - return d == tree.DBoolTrue, nil -} - -// computeRenderColumnName returns render name for a selector, adjusted for CDC use case. -func (e *exprEval) computeRenderColumnName( - ctx context.Context, selector tree.SelectExpr, -) (string, error) { - as, err := func() (string, error) { - if selector.As != "" { - return string(selector.As), nil - } - // We use ComputeColNameInternal instead of GetRenderName because the latter, if it can't - // figure out the name, returns "?column?" as the name; but we want to name things slightly - // different in that case. - _, s, err := tree.ComputeColNameInternal(ctx, e.semaCtx.SearchPath, selector.Expr, e.semaCtx.FunctionResolver) - return s, err - }() - if err != nil { - return "", err + if sameVersion, sameTypes := e.currDesc.EqualsWithUDTCheck(curr); !(sameVersion && sameTypes) { + return false } - if as == "" { - as = fmt.Sprintf("column_%d", 1+len(e.selectors)) + if !e.norm.RequiresPrev() { + return true } - return e.makeUniqueName(as), nil + sameVersion, sameTypes := e.prevDesc.EqualsWithUDTCheck(prev) + return sameVersion && sameTypes } -// makeUniqueName returns a unique name for the specified name. -// We do this because seeing same named fields in JSON might be confusing. -func (e *exprEval) makeUniqueName(as string) string { - useCount := e.nameUseCount[as] - e.nameUseCount[as]++ - if useCount > 0 { - // Unique-ify evalProjection name. - as = fmt.Sprintf("%s_%d", as, useCount) - } - return as -} - -// addSelector adds specified select expression to evalProjection set. -func (e *exprEval) addSelector( - ctx context.Context, selector tree.SelectExpr, numSelectors int, -) error { - as, err := e.computeRenderColumnName(ctx, selector) - if err != nil { - return err - } - - typedExpr, err := e.typeCheck(ctx, selector.Expr, types.Any) - if err != nil { +// planAndRun plans CDC expression and starts execution pipeline. +func (e *Evaluator) planAndRun(ctx context.Context) (err error) { + var plan sql.CDCExpressionPlan + if err := e.preparePlan(ctx, &plan); err != nil { return err } - - // Expand "*". We walked expression during type check above, so we only expect to - // see UnqualifiedStar. - if _, isStar := typedExpr.(tree.UnqualifiedStar); isStar { - if numSelectors == 1 { - // Single star gets special treatment. - e.starProjection = true - } else { - for ord, col := range e.ResultColumns() { - e.addProjection(e.iVarHelper.IndexedVar(ord), e.makeUniqueName(col.Name)) + e.setupProjection(plan.Presentation) + e.input, err = e.executePlan(ctx, plan) + return err +} + +func (e *Evaluator) preparePlan(ctx context.Context, plan *sql.CDCExpressionPlan) error { + return withPlanner( + ctx, e.execCfg, e.user, e.currDesc.SchemaTS, sessiondatapb.SessionData{}, + func(ctx context.Context, execCtx sql.JobExecContext) error { + semaCtx := execCtx.SemaCtx() + semaCtx.FunctionResolver = &e.fnResolver + semaCtx.Properties.Require("cdc", rejectInvalidCDCExprs) + semaCtx.Annotations = tree.MakeAnnotations(cdcAnnotationAddr) + + evalCtx := execCtx.ExtendedEvalContext().Context + evalCtx.Annotations = &semaCtx.Annotations + evalCtx.Annotations.Set(cdcAnnotationAddr, &e.rowEvalCtx) + + e.norm.desc = e.currDesc + if e.norm.RequiresPrev() { + e.rowEvalCtx.prevRowTuple = e.fnResolver.setPrevFuncForEventDescriptor(e.prevDesc) + } else { + e.rowEvalCtx.prevRowTuple = nil } - } - } else { - e.addProjection(typedExpr, as) - } - return nil + p, err := sql.PlanCDCExpression(ctx, execCtx, e.norm.SelectStatement()) + if err != nil { + return err + } + *plan = p + return nil + }, + ) } -// addFilter adds where clause filter. -func (e *exprEval) addFilter(ctx context.Context, where tree.Expr) error { - if where == nil { - return nil - } - typedExpr, err := e.typeCheck(ctx, where, types.Bool) - if err != nil { - return err - } +// setupProjection configures evaluator projection. +func (e *Evaluator) setupProjection(presentation colinfo.ResultColumns) { + e.projection = cdcevent.MakeProjection(e.currDesc) - if typedExpr == tree.DBoolTrue { - if log.V(1) { - log.Infof(ctx, "ignoring tautological filter %q", where) + // makeUniqueName returns a unique name for the specified name. We do this + // because seeing same named fields in JSON output might be confusing (though + // allowed). + nameUseCount := make(map[string]int, len(presentation)) + makeUniqueName := func(as string) string { + useCount := nameUseCount[as] + nameUseCount[as]++ + if useCount > 0 { + as = fmt.Sprintf("%s_%d", as, useCount) } - return nil + return as } - if typedExpr == tree.DBoolFalse { - return errors.Newf("filter %q is a contradiction", where) + // Add presentation columns to the final project, skipping the first + // column which contains the result of the filter evaluation. + for i := 1; i < len(presentation); i++ { + c := presentation[i] + e.projection.AddValueColumn(makeUniqueName(c.Name), c.Typ) } - - e.filter = typedExpr - return nil } -// addProjection adds expression to be returned by evalProjection. -func (e *exprEval) addProjection(expr tree.TypedExpr, as string) { - e.selectors = append(e.selectors, expr) - e.projection.AddValueColumn(as, expr.ResolvedType()) -} - -// typeCheck converts expression to the expression of specified target type. -func (e *exprEval) typeCheck( - ctx context.Context, expr tree.Expr, targetType *types.T, -) (tree.TypedExpr, error) { - // If we have variable free immutable expressions, then we can just evaluate it right away. - typedExpr, err := schemaexpr.SanitizeVarFreeExpr( - ctx, expr, targetType, "cdc", e.semaCtx, - volatility.Immutable, true) - if err == nil { - d, err := eval.Expr(ctx, e.evalCtx, typedExpr) +// inputSpecForEventDescriptor returns input specification for the +// event descriptor. +func inputSpecForEventDescriptor( + ed *cdcevent.EventDescriptor, +) (inputTypes []*types.T, inputCols catalog.TableColMap, _ error) { + for i, c := range ed.ResultColumns() { + col, err := ed.TableDescriptor().FindColumnWithName(tree.Name(c.Name)) if err != nil { - return nil, err + return inputTypes, inputCols, err } - return d, nil - } - - // We must work harder. Bind variables and resolve names. - expr, _ = tree.WalkExpr(&e.resolver, expr) - if e.resolver.err != nil { - return nil, e.resolver.err - } - - if star, isStar := expr.(tree.UnqualifiedStar); isStar { - // Can't type check star -- we'll handle it later during eval. - return star, nil + inputCols.Set(col.GetID(), i) + inputTypes = append(inputTypes, c.Typ) } + return inputTypes, inputCols, nil +} - // Run type check & normalize. - typedExpr, err = expr.TypeCheck(ctx, e.semaCtx, targetType) +// executePlan starts execution of the plan and returns input which receives +// rows that need to be evaluated. +func (e *Evaluator) executePlan( + ctx context.Context, plan sql.CDCExpressionPlan, +) (inputReceiver execinfra.RowReceiver, err error) { + // Configure input. + inputTypes, inputCols, err := inputSpecForEventDescriptor(e.currDesc) if err != nil { return nil, err } - return normalize.Expr(ctx, e.evalCtx, typedExpr) -} - -// evalExpr evaluates typed expression and returns resulting datum. -// must be called after setupContext has been called. -func (e *exprEval) evalExpr( - ctx context.Context, expr tree.TypedExpr, targetType *types.T, -) (tree.Datum, error) { - switch t := expr.(type) { - case tree.Datum: - return t, nil - case *tree.IndexedVar: - d, err := e.rowEvalCtx.updatedRow.DatumAt(t.Idx) - if err != nil { - return nil, err - } - return d, nil - default: - v := replaceIndexVarVisitor{row: e.rowEvalCtx.updatedRow} - newExpr, _ := tree.WalkExpr(&v, expr) - if v.err != nil { - return nil, v.err - } - typedExpr, err := tree.TypeCheck(ctx, newExpr, e.semaCtx, targetType) - if err != nil { - return nil, err - } - d, err := eval.Expr(ctx, e.evalCtx, typedExpr) - if err != nil { - return nil, err + // The row channel created below will have exactly 1 sender (this evaluator). + // The buffer size parameter doesn't matter much, as long as it is greater + // than 0 to make sure that if the main context is cancelled and the flow + // exits, that we can still push data into the row channel without blocking, + // so that we notice cancellation request when we try to read the result of + // the evaluation. + const numSenders = 1 + const bufSize = 16 + var input execinfra.RowChannel + input.InitWithBufSizeAndNumSenders(inputTypes, bufSize, numSenders) + + // writer sends result of the evaluation into row channel. + writer := sql.NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + select { + case <-ctx.Done(): + return ctx.Err() + case e.rowCh <- row: + return nil } - return d, nil - } -} - -// cdcExprVisitor is a visitor responsible for analyzing expression to determine -// if it consists of expressions supported by CDC. -// This visitor is used early to sanity check expression. -type cdcExprVisitor struct { - semaCtx *tree.SemaContext - ctx context.Context - err error -} - -var _ tree.Visitor = (*cdcExprVisitor)(nil) - -// validateExpressionForCDC runs quick checks to make sure that expr is valid for -// CDC use case. This doesn't catch all the invalid cases, but is a place to pick up -// obviously wrong expressions. -func validateExpressionForCDC( - ctx context.Context, expr tree.Expr, semaCtx *tree.SemaContext, -) (tree.Expr, error) { - v := cdcExprVisitor{semaCtx: semaCtx, ctx: ctx} - expr, _ = tree.WalkExpr(&v, expr) - if v.err != nil { - return nil, v.err - } - return expr, nil -} + }) + + // receiver writes the results to the writer. + receiver := sql.MakeDistSQLReceiver( + ctx, + writer, + tree.Rows, + e.execCfg.RangeDescriptorCache, + nil, + nil, /* clockUpdater */ + &sql.SessionTracing{}, + e.execCfg.ContentionRegistry, + ) -// VisitPre implements tree.Visitor interface. -func (v *cdcExprVisitor) VisitPre(expr tree.Expr) (bool, tree.Expr) { - return v.err == nil, expr -} + // Start execution. + e.planGroup = ctxgroup.WithContext(ctx) + e.planGroup.GoCtx(func(ctx context.Context) (err error) { + defer func() { + if r := recover(); r != nil { + err = errors.Newf("error evaluating CDC expression %q: %s", + tree.AsString(e.norm.SelectClause), r) + } + e.errCh <- err + }() -// VisitPost implements tree.Visitor interface. -func (v *cdcExprVisitor) VisitPost(expr tree.Expr) tree.Expr { - switch t := expr.(type) { - case *tree.FuncExpr: - fn, err := checkFunctionSupported(v.ctx, t, v.semaCtx) - if err != nil { - v.err = err - return expr + defer receiver.Release() + if err := sql.RunCDCEvaluation(ctx, plan, &input, inputCols, receiver); err != nil { + return err } - return fn - case *tree.Subquery: - v.err = pgerror.New(pgcode.FeatureNotSupported, "subquery expressions not supported by CDC") - return expr - default: - return expr - } -} - -// cdcNameResolver is a visitor that resolves names in the expression -// and associates them with the EventDescriptor columns. -type cdcNameResolver struct { - schemaexpr.NameResolutionVisitor - *cdcevent.EventDescriptor - err error -} + return writer.Err() + }) -// tag errors generated by cdcNameResolver. -type cdcResolverError struct { - error + return &input, nil } -func (v *cdcNameResolver) wrapError() func() { - // NameResolutionVisitor returns "column X does not exist" error if expression references - // column that was not configured. This is a bit confusing for CDC since a column - // may exist in the table, but not be available for a particular family. So, annotate - // the error to make it more obvious. - // We only want to do this for errors returned by NameResolutionVisitor, and not errors - // that we generate ourselves. - if v.err == nil { - return func() { - if v.NameResolutionVisitor.Err() != nil && v.err == nil { - v.err = errors.WithHintf(v.Err(), - "object does not exist in table %q, family %q", v.TableName, v.FamilyName) - } +// setupContextForRow configures evaluation context with the provided row +// information. +func (e *Evaluator) setupContextForRow(ctx context.Context, updated, prev cdcevent.Row) error { + e.rowEvalCtx.ctx = ctx + e.rowEvalCtx.updatedRow = updated + if e.norm.RequiresPrev() { + if err := prev.CopyInto(e.rowEvalCtx.prevRowTuple); err != nil { + return err } } - return func() {} -} - -// VisitPre implements tree.Visitor interface. -func (v *cdcNameResolver) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { - defer v.wrapError()() - recurse, newExpr = v.NameResolutionVisitor.VisitPre(expr) - return v.err == nil, newExpr + return nil } -// VisitPost implements tree.Visitor interface. -func (v *cdcNameResolver) VisitPost(expr tree.Expr) tree.Expr { - defer v.wrapError()() - expr = v.NameResolutionVisitor.VisitPost(expr) - - switch t := expr.(type) { - case *tree.AllColumnsSelector: - // AllColumnsSelector occurs when "x.*" is used. We have a simple 1 table support, - // so make sure table names match. - if t.TableName.String() != v.TableName { - v.err = &cdcResolverError{ - error: pgerror.Newf(pgcode.UndefinedTable, "no data source matches pattern: %s", t.String()), - } - return t - } - // Now that we know table names match, turn this into unqualified star. - return tree.UnqualifiedStar{} - default: - return expr - } +// Close closes currently running execution. +func (e *Evaluator) Close() { + _ = e.closeErr() // We expect to see an error, such as context cancelled. } -func checkFunctionSupported( - ctx context.Context, fnCall *tree.FuncExpr, semaCtx *tree.SemaContext, -) (*tree.FuncExpr, error) { - var fnName string - var fnClass tree.FunctionClass - var fnVolatility volatility.V - - unsupportedFunctionErr := func() error { - if fnName == "" { - fnName = fnCall.Func.String() - } - return &cdcResolverError{ - error: pgerror.Newf(pgcode.UndefinedFunction, "function %q unsupported by CDC", fnName), - } - } - - funcDef, err := fnCall.Func.Resolve(ctx, semaCtx.SearchPath, semaCtx.FunctionResolver) - if err != nil { - return nil, unsupportedFunctionErr() - } - - if _, isCDCFn := cdcFunctions[funcDef.Name]; isCDCFn { - return fnCall, nil +func (e *Evaluator) closeErr() error { + if e.errCh != nil { + defer close(e.errCh) // Must be deferred since planGroup go routine might write. } - fnClass, err = funcDef.GetClass() - if err != nil { - return nil, err - } - fnName = funcDef.Name - if fnCall.ResolvedOverload() != nil { - fnVolatility = fnCall.ResolvedOverload().Volatility - } else { - // Pick highest volatility overload. - for i := range funcDef.Overloads { - overload := funcDef.Overloads[i].Overload - if overload.Volatility > fnVolatility { - fnVolatility = overload.Volatility - } - } - } - - // Aggregates, generators and window functions are not supported. - switch fnClass { - case tree.AggregateClass, tree.GeneratorClass, tree.WindowClass: - return nil, unsupportedFunctionErr() + if e.input != nil { + e.input.ProducerDone() + e.input = nil + return e.planGroup.Wait() } - - if fnVolatility <= volatility.Immutable { - // Remaining immutable functions are safe. - return fnCall, nil - } - - // We have a non-immutable function -- make sure it is supported. - _, isSafe := supportedVolatileBuiltinFunctions[fnName] - if !isSafe { - return nil, unsupportedFunctionErr() - } - return fnCall, nil -} - -// rowContainer is a structure to assist with evaluation of CDC expressions. -type rowContainer struct { - cols []cdcevent.ResultColumn -} - -var _ tree.IndexedVarContainer = (*rowContainer)(nil) - -// IndexedVarResolvedType implements tree.IndexedVarContainer -func (c *rowContainer) IndexedVarResolvedType(idx int) *types.T { - return c.cols[idx].Typ -} - -// IndexedVarNodeFormatter implements tree.IndexedVarContainer -func (c *rowContainer) IndexedVarNodeFormatter(idx int) tree.NodeFormatter { return nil } -type replaceIndexVarVisitor struct { - row cdcevent.Row - err error -} - -var _ tree.Visitor = (*replaceIndexVarVisitor)(nil) - -// VisitPre implements tree.Visitor interface. -func (v *replaceIndexVarVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) { - if iVar, ok := expr.(*tree.IndexedVar); ok { - datum, err := v.row.DatumAt(iVar.Idx) - if err != nil { - v.err = pgerror.Wrapf(err, pgcode.NumericValueOutOfRange, "variable @%d out of bounds", iVar.Idx) - return false, expr - } - return true, datum - } - return true, expr -} - -// VisitPost implements tree.Visitor interface. -func (v *replaceIndexVarVisitor) VisitPost(expr tree.Expr) (newNode tree.Expr) { - return expr +// rowEvalContext represents the context needed to evaluate row expressions. +type rowEvalContext struct { + ctx context.Context + updatedRow cdcevent.Row + prevRowTuple *tree.DTuple } // cdcAnnotationAddr is the address used to store relevant information @@ -724,72 +350,40 @@ func rowEvalContextFromEvalContext(evalCtx *eval.Context) *rowEvalContext { return evalCtx.Annotations.Get(cdcAnnotationAddr).(*rowEvalContext) } -const rejectInvalidCDCExprs = (tree.RejectAggregates | tree.RejectGenerators | - tree.RejectWindowApplications | tree.RejectNestedGenerators) +const rejectInvalidCDCExprs = tree.RejectAggregates | tree.RejectGenerators | + tree.RejectWindowApplications | tree.RejectNestedGenerators -// newSemaCtx returns new tree.SemaCtx configured for cdc without type resolver. -func newSemaCtx() *tree.SemaContext { - sema := tree.MakeSemaContext() - sema.SearchPath = &sessiondata.DefaultSearchPath - sema.FunctionResolver = &CDCFunctionResolver{} - sema.Properties.Require("cdc", rejectInvalidCDCExprs) - return &sema -} +// configSemaForCDC configures existing semaCtx to be used for CDC expression +// evaluation; returns cleanup function which restores previous configuration. +func configSemaForCDC(semaCtx *tree.SemaContext, d *cdcevent.EventDescriptor) func() { + origProps, origResolver := semaCtx.Properties, semaCtx.FunctionResolver + var r CDCFunctionResolver + r.setPrevFuncForEventDescriptor(d) + semaCtx.FunctionResolver = &r + semaCtx.Properties.Require("cdc", rejectInvalidCDCExprs) -// newSemaCtxWithTypeResolver returns new tree.SemaCtx configured for cdc. -func newSemaCtxWithTypeResolver(d *cdcevent.EventDescriptor) *tree.SemaContext { - sema := newSemaCtx() - if d.HasUserDefinedTypes() { - sema.TypeResolver = newTypeReferenceResolver(d) + return func() { + semaCtx.Properties.Restore(origProps) + semaCtx.FunctionResolver = origResolver } - return sema } -// cdcTypeReferenceReesolver is responsible for resolving user defined types. -type cdcTypeReferenceResolver struct { - byName map[string]*types.T - byOID map[oid.Oid]*types.T -} - -var _ tree.TypeReferenceResolver = (*cdcTypeReferenceResolver)(nil) - -func newTypeReferenceResolver(d *cdcevent.EventDescriptor) tree.TypeReferenceResolver { - // Because EventDescriptor is built with hydrated table descriptors, and the - // expression must have been normalized, we don't need to do any fancy - // resolution; just go through user defined columns in the descriptor and - // build the lookup maps. - r := &cdcTypeReferenceResolver{ - byName: make(map[string]*types.T), - byOID: make(map[oid.Oid]*types.T), +// predicateAsProjection replaces predicate (where clause) with a projection +// (select clause). The "matches" predicate will be the first predicate. This +// step is done so that distSQL notifies us about the events that should be +// filtered, as opposed to filtering those events directly, since we need to +// perform cleanup tasks (release allocation, update metrics, etc.), even +// for events that do not match the predicate. +func predicateAsProjection(n *NormalizedSelectClause) { + filter := tree.SelectExpr{ + Expr: tree.DBoolTrue, + As: "__crdb_filter", } - for _, c := range d.ResultColumns() { - if c.Typ.UserDefined() { - r.byName[c.Typ.TypeMeta.Name.FQName()] = c.Typ - r.byOID[c.Typ.Oid()] = c.Typ - } - } - return r -} - -// ResolveType implements tree.TypeReferenceResolver. -func (r *cdcTypeReferenceResolver) ResolveType( - ctx context.Context, name *tree.UnresolvedObjectName, -) (*types.T, error) { - // NB: normalization step fully qualifies types, so use the full name to - // lookup. - if typ, found := r.byName[name.String()]; found { - return typ, nil + if n.Where != nil { + filter.Expr = &tree.ParenExpr{Expr: n.Where.Expr} + n.Where = nil } - return nil, pgerror.Newf(pgcode.UndefinedObject, "undefined object %s", name) -} -// ResolveTypeByOID implements tree.TypeReferenceResolver. -func (r *cdcTypeReferenceResolver) ResolveTypeByOID( - ctx context.Context, oid oid.Oid, -) (*types.T, error) { - if typ, found := r.byOID[oid]; found { - return typ, nil - } - return nil, pgerror.Newf(pgcode.UndefinedObject, "undefined object with OID %d", oid) + n.SelectClause.Exprs = append(tree.SelectExprs{filter}, n.SelectClause.Exprs...) } diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go index 40387181a655..5e72fc91e906 100644 --- a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go +++ b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go @@ -21,15 +21,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/distsql" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -53,17 +52,20 @@ CREATE TABLE foo ( a INT, b STRING, c STRING, - d STRING AS (concat(b, c)) VIRTUAL, + d STRING AS (concat(b, c)) VIRTUAL, e status DEFAULT 'inactive', + f STRING, + g STRING, PRIMARY KEY (b, a), FAMILY main (a, b, e), - FAMILY only_c (c) + FAMILY only_c (c), + FAMILY f_g_fam(f,g) )`) desc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") type decodeExpectation struct { expectUnwatchedErr bool - projectionErr string + evalErr string // current value expectations. expectFiltered bool @@ -85,22 +87,25 @@ CREATE TABLE foo ( return expectations[0], expectations[1:] } + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + for _, tc := range []struct { testName string familyName string // Must be set if targetType ChangefeedTargetSpecification_COLUMN_FAMILY setupActions []string // SQL statements to execute before starting rangefeed. actions []string // SQL statements to execute after starting rangefeed. - predicate string - predicateErr string // Expect to get an error when configuring predicates + stmt string + expectErr string // Expect to get an error when configuring predicates expectMainFamily []decodeExpectation expectOnlyCFamily []decodeExpectation + expectFGFamily []decodeExpectation }{ { testName: "main/star", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (1, '1st test')"}, - predicate: "SELECT * FROM _", + stmt: "SELECT * FROM foo", expectMainFamily: []decodeExpectation{ { keyValues: []string{"1st test", "1"}, @@ -112,7 +117,7 @@ CREATE TABLE foo ( testName: "main/qualified_star", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (1, 'qualified')"}, - predicate: "SELECT foo.* FROM _", + stmt: "SELECT foo.* FROM foo", expectMainFamily: []decodeExpectation{ { keyValues: []string{"qualified", "1"}, @@ -127,7 +132,7 @@ CREATE TABLE foo ( "INSERT INTO foo (a, b) VALUES (2, '2nd test')", "DELETE FROM foo WHERE a=2 AND b='2nd test'", }, - predicate: "SELECT *, cdc_is_delete() FROM _", + stmt: "SELECT *, cdc_is_delete() FROM foo WHERE 'hello' != 'world'", expectMainFamily: []decodeExpectation{ { keyValues: []string{"2nd test", "2"}, @@ -143,7 +148,7 @@ CREATE TABLE foo ( testName: "main/projection", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (3, '3rd test')"}, - predicate: "SELECT e, a FROM _", + stmt: "SELECT e, a FROM foo", expectMainFamily: []decodeExpectation{ { keyValues: []string{"3rd test", "3"}, @@ -155,7 +160,7 @@ CREATE TABLE foo ( testName: "main/projection_aliased", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (3, '3rd test')"}, - predicate: "SELECT bar.e, a FROM foo AS bar", + stmt: "SELECT bar.e, a FROM foo AS bar", expectMainFamily: []decodeExpectation{ { keyValues: []string{"3rd test", "3"}, @@ -173,7 +178,7 @@ CREATE TABLE foo ( "INSERT INTO foo (a, b, e) VALUES (4, '4th test', 'closed')", "INSERT INTO foo (a, b, e) VALUES (5, '4th test', 'inactive')", }, - predicate: "SELECT a FROM _ WHERE e IN ('open', 'inactive')", + stmt: "SELECT a FROM _ WHERE e IN ('open', 'inactive')", expectMainFamily: []decodeExpectation{ { expectFiltered: true, @@ -197,11 +202,25 @@ CREATE TABLE foo ( }, }, }, + { + testName: "main/select_udts", + familyName: "main", + actions: []string{ + "INSERT INTO foo (a, b, e) VALUES (1, '4th test', 'closed')", + }, + stmt: "SELECT a, 'inactive'::status as inactive, e FROM foo", + expectMainFamily: []decodeExpectation{ + { + keyValues: []string{"4th test", "1"}, + allValues: map[string]string{"a": "1", "inactive": "inactive", "e": "closed"}, + }, + }, + }, { testName: "main/same_column_many_times", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (1, '5th test')"}, - predicate: "SELECT *, a, a as one_more, a FROM _", + stmt: "SELECT *, a, a as one_more, a FROM foo", expectMainFamily: []decodeExpectation{ { keyValues: []string{"5th test", "1"}, @@ -216,19 +235,21 @@ CREATE TABLE foo ( testName: "main/no_col_c", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (1, 'no_c')"}, - predicate: "SELECT *, c FROM _", - expectMainFamily: []decodeExpectation{ - { - projectionErr: `column "c" does not exist`, - keyValues: []string{"no_c", "1"}, - }, - }, + stmt: "SELECT a, c FROM _", + expectErr: `column "foo.c" does not exist`, + }, + { + testName: "main/no_col_c_star", + familyName: "main", + actions: []string{"INSERT INTO foo (a, b) VALUES (1, 'no_c')"}, + stmt: "SELECT *, c FROM _", + expectErr: `column "foo.c" does not exist`, }, { testName: "main/non_primary_family_with_var_free", familyName: "only_c", actions: []string{"INSERT INTO foo (a, b, c) VALUES (42, '6th test', 'c value')"}, - predicate: "SELECT sin(pi()/2) AS var_free, c, b ", + stmt: "SELECT sin(pi()/2) AS var_free, c, b FROM foo", expectMainFamily: []decodeExpectation{ { expectUnwatchedErr: true, @@ -241,6 +262,25 @@ CREATE TABLE foo ( }, }, }, + { + testName: "main/concat", + familyName: "f_g_fam", + actions: []string{ + "INSERT INTO foo (a, b, f) VALUES (42, 'concat', 'hello')", + }, + stmt: "SELECT a, b, f || f AS ff, g || g AS gg FROM foo", + expectMainFamily: []decodeExpectation{ + { + expectUnwatchedErr: true, + }, + }, + expectFGFamily: []decodeExpectation{ + { + keyValues: []string{"concat", "42"}, + allValues: map[string]string{"a": "42", "b": "concat", "ff": "hellohello", "gg": "NULL"}, + }, + }, + }, { testName: "main/cdc_prev_select", familyName: "only_c", @@ -248,7 +288,7 @@ CREATE TABLE foo ( "INSERT INTO foo (a, b, c) VALUES (42, 'prev_select', 'c value old')", "UPSERT INTO foo (a, b, c) VALUES (42, 'prev_select', 'c value updated')", }, - predicate: "SELECT a, b, c, (CASE WHEN cdc_prev()->>'c' IS NULL THEN 'not there' ELSE cdc_prev()->>'c' END) AS old_c", + stmt: "SELECT a, b, c, (CASE WHEN cdc_prev.c IS NULL THEN 'not there' ELSE cdc_prev.c END) AS old_c FROM foo", expectMainFamily: []decodeExpectation{ { expectUnwatchedErr: true, @@ -272,7 +312,7 @@ CREATE TABLE foo ( "INSERT INTO foo (a, b) VALUES (123, 'select_if')", "DELETE FROM foo where a=123", }, - predicate: "SELECT IF(cdc_is_delete(),'deleted',a::string) AS conditional FROM _", + stmt: "SELECT IF(cdc_is_delete(),'deleted',a::string) AS conditional FROM _", expectMainFamily: []decodeExpectation{ { keyValues: []string{"select_if", "123"}, @@ -290,7 +330,7 @@ CREATE TABLE foo ( actions: []string{ "INSERT INTO foo (a, b) VALUES (1, ' spaced out ')", }, - predicate: "SELECT btrim(b), parse_timetz('1:00-0') AS past FROM _", + stmt: "SELECT btrim(b), parse_timetz('1:00-0') AS past FROM _", expectMainFamily: []decodeExpectation{ { keyValues: []string{" spaced out ", "1"}, @@ -298,49 +338,78 @@ CREATE TABLE foo ( }, }, }, + // TODO(yevgeniy): enable this test. + // This requires adding support to "fetch" those magic system columns from + // row fetcher, or replace them with a function call. + //{ + // testName: "main/magic_column", + // familyName: "main", + // actions: []string{ + // "INSERT INTO foo (a, b) VALUES (1, 'hello')", + // }, + // stmt: "SELECT a, crdb_internal_mvcc_timestamp FROM foo", + // expectMainFamily: []decodeExpectation{ + // { + // keyValues: []string{"hello", "1"}, + // allValues: map[string]string{"a": "1", "crdb_internal_mvcc_timestamp": "xxx"}, + // }, + // }, + //}, + // { + // // TODO(yevgeniy): Test currently disable since session data is not serialized. + // // Issue #90421 + // testName: "main/trigram", + // familyName: "main", + // actions: []string{ + // "INSERT INTO foo (a, b) VALUES (1, 'hello')", + // }, + // stmt: "SELECT a, b % 'hel' as trigram, b % 'heh' AS trigram2 FROM foo", + // expectMainFamily: []decodeExpectation{ + // { + // keyValues: []string{"hello", "1"}, + // allValues: map[string]string{"a": "1", "trigram": "true", "trigram2": "false"}, + // }, + // }, + //}, { testName: "main/btrim_wrong_type", familyName: "main", actions: []string{ "INSERT INTO foo (a, b) VALUES (1, ' spaced out ')", }, - predicate: "SELECT btrim(a) FROM _", - expectMainFamily: []decodeExpectation{ - { - keyValues: []string{" spaced out ", "1"}, - projectionErr: "unknown signature: btrim\\(int\\)", - }, - }, + stmt: "SELECT btrim(a) FROM foo", + expectErr: "unknown signature: btrim\\(int\\)", }, { testName: "main/contradiction", familyName: "main", actions: []string{"INSERT INTO foo (a, b) VALUES (1, 'contradiction')"}, - predicate: "SELECT * FROM _ WHERE 1 > 2", - expectMainFamily: []decodeExpectation{ - { - projectionErr: `filter .* is a contradiction`, - keyValues: []string{"contradiction", "1"}, - }, - }, + stmt: "SELECT * FROM foo WHERE 1 > 2", + expectErr: "does not match any rows", }, { - testName: "main/no_sleep", - familyName: "main", - predicate: "SELECT *, pg_sleep(86400) AS wake_up FROM _", - predicateErr: `function "pg_sleep" unsupported by CDC`, + testName: "main/no_sleep", + familyName: "main", + stmt: "SELECT *, pg_sleep(86400) AS wake_up FROM _", + expectErr: `function "pg_sleep" unsupported by CDC`, }, { - testName: "main/no_subselect", - familyName: "main", - predicate: "SELECT cdc_prev(), cdc_is_delete(123), (select column1 from (values (1,2,3))) FROM _", - predicateErr: `subquery expressions not supported by CDC`, + testName: "main/no_subselect", + familyName: "main", + stmt: "SELECT cdc_prev, cdc_is_delete(), (select column1 from (values (1,2,3))) FROM foo", + expectErr: `sub-query expressions not supported by CDC`, }, { - testName: "main/no_subselect_in_where", - familyName: "main", - predicate: "SELECT cdc_prev() FROM _ WHERE a = 2 AND (select 3) = 3", - predicateErr: `subquery expressions not supported by CDC`, + testName: "main/no_subselect_in_where", + familyName: "main", + stmt: "SELECT cdc_prev FROM foo WHERE a = 2 AND (select 3) = 3", + expectErr: `sub-query expressions not supported by CDC`, + }, + { + testName: "main/exists_subselect", + familyName: "main", + stmt: "SELECT 1 FROM foo WHERE EXISTS (SELECT true)", + expectErr: "sub-query expressions not supported by CDC", }, { testName: "main/filter_many", @@ -350,7 +419,7 @@ CREATE TABLE foo ( "(SELECT generate_series as x FROM generate_series(1, 100)) " + "SELECT x, 'filter_many', x::string FROM s", }, - predicate: "SELECT * FROM _ WHERE a % 33 = 0", + stmt: "SELECT * FROM foo WHERE a % 33 = 0", expectMainFamily: repeatExpectation(decodeExpectation{expectUnwatchedErr: true}, 100), expectOnlyCFamily: func() (expectations []decodeExpectation) { for i := 1; i <= 100; i++ { @@ -359,7 +428,7 @@ CREATE TABLE foo ( keyValues: []string{"filter_many", iStr}, } if i%33 == 0 { - e.allValues = map[string]string{"c": iStr} + e.allValues = map[string]string{"a": iStr, "b": "filter_many", "c": iStr} } else { e.expectFiltered = true } @@ -377,14 +446,15 @@ CREATE TABLE foo ( "SELECT x, 'only_some_deleted_values', x::string FROM s", }, actions: []string{"DELETE FROM foo WHERE b='only_some_deleted_values'"}, - predicate: `SELECT * FROM _ WHERE cdc_is_delete() AND cast(cdc_prev()->>'a' as int) % 33 = 0`, + stmt: `SELECT * FROM foo WHERE cdc_is_delete() AND cdc_prev.a % 33 = 0`, expectMainFamily: repeatExpectation(decodeExpectation{expectUnwatchedErr: true}, 100), expectOnlyCFamily: func() (expectations []decodeExpectation) { for i := 1; i <= 100; i++ { + iStr := strconv.FormatInt(int64(i), 10) e := decodeExpectation{ - keyValues: []string{"only_some_deleted_values", strconv.FormatInt(int64(i), 10)}, + keyValues: []string{"only_some_deleted_values", iStr}, expectFiltered: i%33 != 0, - allValues: map[string]string{"c": "NULL"}, + allValues: map[string]string{"a": iStr, "b": "only_some_deleted_values", "c": "NULL"}, } expectations = append(expectations, e) } @@ -394,30 +464,30 @@ CREATE TABLE foo ( } { t.Run(tc.testName, func(t *testing.T) { sqlDB.Exec(t, "DELETE FROM foo WHERE true") - - // Setup evaluator. - evaluator, err := makeEvaluator(t, s.ClusterSettings(), tc.predicate) - if tc.predicateErr != "" { - require.Regexp(t, tc.predicateErr, err) - return - } - require.NoError(t, err) - targetType := jobspb.ChangefeedTargetSpecification_EACH_FAMILY if tc.familyName != "" { targetType = jobspb.ChangefeedTargetSpecification_COLUMN_FAMILY } targets := changefeedbase.Targets{} - targets.Add(changefeedbase.Target{ + target := changefeedbase.Target{ Type: targetType, TableID: desc.GetID(), FamilyName: tc.familyName, - }) + } + targets.Add(target) + + // Setup evaluator. + e, err := newEvaluatorWithNormCheck(&execCfg, desc, s.Clock().Now(), target, tc.stmt) + if tc.expectErr != "" { + require.Regexp(t, tc.expectErr, err, err) + return + } + require.NoError(t, err) + defer e.Close() - serverCfg := s.DistSQLServer().(*distsql.ServerImpl).ServerConfig ctx := context.Background() - decoder, err := cdcevent.NewEventDecoder(ctx, &serverCfg, targets, false, false) + decoder, err := cdcevent.NewEventDecoder(ctx, &execCfg, targets, false, false) require.NoError(t, err) for _, action := range tc.setupActions { @@ -431,7 +501,7 @@ CREATE TABLE foo ( sqlDB.Exec(t, action) } - expectedEvents := len(tc.expectMainFamily) + len(tc.expectOnlyCFamily) + expectedEvents := len(tc.expectMainFamily) + len(tc.expectOnlyCFamily) + len(tc.expectFGFamily) vals := readSortedRangeFeedValues(t, expectedEvents, popRow) for _, v := range vals { eventFamilyID, err := cdcevent.TestingGetFamilyIDFromKey(decoder, v.Key, v.Timestamp()) @@ -440,8 +510,10 @@ CREATE TABLE foo ( var expect decodeExpectation if eventFamilyID == 0 { expect, tc.expectMainFamily = popExpectation(t, tc.expectMainFamily) - } else { + } else if eventFamilyID == 1 { expect, tc.expectOnlyCFamily = popExpectation(t, tc.expectOnlyCFamily) + } else { + expect, tc.expectFGFamily = popExpectation(t, tc.expectFGFamily) } updatedRow, err := decodeRowErr(decoder, &v, false) @@ -455,25 +527,23 @@ CREATE TABLE foo ( prevRow := decodeRow(t, decoder, &v, true) require.NoError(t, err) - if expect.expectFiltered { - require.Equal(t, expect.keyValues, slurpKeys(t, updatedRow), - "isDelete=%t fid=%d", updatedRow.IsDeleted(), eventFamilyID) - matches, err := evaluator.MatchesFilter(ctx, updatedRow, v.Timestamp(), prevRow) - require.NoError(t, err) - require.False(t, matches, "keys: %v", slurpKeys(t, updatedRow)) + require.Equal(t, expect.keyValues, slurpKeys(t, updatedRow), + "isDelete=%t fid=%d", updatedRow.IsDeleted(), eventFamilyID) + + projection, err := e.Eval(ctx, updatedRow, prevRow) + if expect.evalErr != "" { + require.Regexp(t, expect.evalErr, err) continue } + require.NoError(t, err) - projection, err := evaluator.Projection(ctx, updatedRow, v.Timestamp(), prevRow) - if expect.projectionErr != "" { - require.Regexp(t, expect.projectionErr, err) - // Sanity check we get error for the row we expected to get an error for. - require.Equal(t, expect.keyValues, slurpKeys(t, updatedRow)) - } else { - require.NoError(t, err) - require.Equal(t, expect.keyValues, slurpKeys(t, projection)) - require.Equal(t, expect.allValues, slurpValues(t, projection)) + if expect.expectFiltered { + require.False(t, projection.IsInitialized(), "keys: %v", slurpKeys(t, updatedRow)) + continue } + + require.Equal(t, expect.keyValues, slurpKeys(t, projection)) + require.Equal(t, expect.allValues, slurpValues(t, projection)) } }) } @@ -488,9 +558,14 @@ func TestUnsupportedCDCFunctions(t *testing.T) { s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, "CREATE TABLE foo (a INT PRIMARY KEY)") - + desc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") + target := changefeedbase.Target{ + TableID: desc.GetID(), + FamilyName: desc.GetFamilies()[0].Name, + } for fnCall, errFn := range map[string]string{ // Some volatile functions. "version()": "version", @@ -512,134 +587,31 @@ func TestUnsupportedCDCFunctions(t *testing.T) { "crdb_internal.get_namespace_id()": "crdb_internal.get_namespace_id", } { t.Run(fmt.Sprintf("select/%s", errFn), func(t *testing.T) { - _, err := makeEvaluator(t, s.ClusterSettings(), fmt.Sprintf("SELECT %s", fnCall)) + _, err := newEvaluatorWithNormCheck(&execCfg, desc, execCfg.Clock.Now(), target, + fmt.Sprintf("SELECT %s FROM foo", fnCall)) require.Regexp(t, fmt.Sprintf(`function "%s" unsupported by CDC`, errFn), err) }) // Same thing, but with the WHERE clause t.Run(fmt.Sprintf("where/%s", errFn), func(t *testing.T) { - _, err := makeEvaluator(t, s.ClusterSettings(), - fmt.Sprintf("SELECT 1 WHERE %s IS NOT NULL", fnCall)) + _, err := newEvaluatorWithNormCheck(&execCfg, desc, s.Clock().Now(), target, + fmt.Sprintf("SELECT 1 FROM foo WHERE %s IS NOT NULL", fnCall)) require.Regexp(t, fmt.Sprintf(`function "%s" unsupported by CDC`, errFn), err) }) } } -func TestEvaluatesProjection(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) - defer s.Stopper().Stop(context.Background()) - - sqlDB := sqlutils.MakeSQLRunner(db) - sqlDB.Exec(t, ""+ - "CREATE TABLE foo (a INT PRIMARY KEY, b STRING, c STRING, d INT, "+ - "FAMILY most (a,b,c), FAMILY only_d (d))") - desc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") - testRow := cdcevent.TestingMakeEventRow(desc, 0, randEncDatumRow(t, desc, 0), false) - - verifyConstantsFolded := func(p *exprEval) { - for _, expr := range p.selectors { - _ = expr.(tree.Datum) - } - } - - for _, tc := range []struct { - name string - predicate string - input rowenc.EncDatumRow - expectErr string - expectation map[string]string - verifyFold bool - }{ - { - name: "constants", - predicate: "SELECT 1, 2, 3", - expectation: map[string]string{"column_1": "1", "column_2": "2", "column_3": "3"}, - verifyFold: true, - }, - { - name: "constants_functions_and_aliases", - predicate: "SELECT 0 as zero, abs(-2) two, 42", - expectation: map[string]string{"zero": "0", "two": "2", "column_3": "42"}, - verifyFold: true, - }, - { - name: "trig_fun", - predicate: "SELECT cos(0), sin(pi()/2) as sin_90, 39 + pi()::int", - expectation: map[string]string{"cos": "1.0", "sin_90": "1.0", "column_3": "42"}, - verifyFold: true, - }, - { - name: "div_by_zero", - predicate: "SELECT 3 / sin(pi() - pi()) as result", - expectErr: "division by zero", - }, - { - name: "projection_with_bound_vars", - predicate: "SELECT sqrt(a::float) + sin(pi()/2) as result, foo.*", - input: makeEncDatumRow(tree.NewDInt(4), tree.DNull, tree.DNull), - expectation: map[string]string{"result": "3.0", "a": "4", "b": "NULL", "c": "NULL"}, - }, - } { - t.Run(tc.name, func(t *testing.T) { - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, tc.predicate) - if tc.expectErr != "" { - require.Regexp(t, tc.expectErr, err) - return - } - - require.NoError(t, err) - if tc.verifyFold { - verifyConstantsFolded(e) - } - row := testRow - if tc.input != nil { - row = cdcevent.TestingMakeEventRow(desc, 0, tc.input, false) - } - - p, err := e.evalProjection(context.Background(), row, hlc.Timestamp{}, row) - require.NoError(t, err) - require.Equal(t, tc.expectation, slurpValues(t, p)) - }) - } -} - -// makeEvaluator creates Evaluator and configures it with specified -// select statement predicate. -func makeEvaluator(t *testing.T, st *cluster.Settings, selectStr string) (*Evaluator, error) { - t.Helper() - s, err := parser.ParseOne(selectStr) - require.NoError(t, err) - slct := s.AST.(*tree.Select).Select.(*tree.SelectClause) - evalCtx := eval.MakeTestingEvalContext(st) - return NewEvaluator(context.Background(), &evalCtx, slct) -} - -func makeExprEval( - t *testing.T, st *cluster.Settings, ed *cdcevent.EventDescriptor, selectStr string, -) (*exprEval, error) { - t.Helper() - e, err := makeEvaluator(t, st, selectStr) - require.NoError(t, err) - - if err := e.initEval(context.Background(), ed); err != nil { - return nil, err - } - return e.evaluator, nil -} - func decodeRowErr( decoder cdcevent.Decoder, v *roachpb.RangeFeedValue, prev bool, ) (cdcevent.Row, error) { - kv := roachpb.KeyValue{Key: v.Key} + keyVal := roachpb.KeyValue{Key: v.Key} if prev { - kv.Value = v.PrevValue + keyVal.Value = v.PrevValue } else { - kv.Value = v.Value + keyVal.Value = v.Value } - return decoder.DecodeKV(context.Background(), kv, v.Timestamp(), false) + const keyOnly = false + return decoder.DecodeKV(context.Background(), keyVal, v.Timestamp(), keyOnly) } func decodeRow( @@ -669,13 +641,13 @@ func slurpValues(t *testing.T, r cdcevent.Row) map[string]string { return res } -func randEncDatumRow( - t *testing.T, desc catalog.TableDescriptor, familyID descpb.FamilyID, +func randEncDatumPrimaryFamily( + t *testing.T, desc catalog.TableDescriptor, ) (row rowenc.EncDatumRow) { t.Helper() rng, _ := randutil.NewTestRand() - family, err := desc.FindFamilyByID(familyID) + family, err := desc.FindFamilyByID(0) require.NoError(t, err) for _, colID := range family.ColumnIDs { col, err := desc.FindColumnWithID(colID) @@ -685,13 +657,6 @@ func randEncDatumRow( return row } -func makeEncDatumRow(datums ...tree.Datum) (row rowenc.EncDatumRow) { - for _, d := range datums { - row = append(row, rowenc.EncDatum{Datum: d}) - } - return row -} - // readSortedRangeFeedValues reads n values, and sorts them based on key order. func readSortedRangeFeedValues( t *testing.T, n int, row func(t *testing.T) *roachpb.RangeFeedValue, @@ -706,3 +671,40 @@ func readSortedRangeFeedValues( }) return res } + +// Evaluator gets constructed w/ normalization steps already performed. +// This test utility function adds (usually un-needed) normalization step +// so that errors in expression can be picked up without calling evaluator.Eval(). +func newEvaluatorWithNormCheck( + execCfg *sql.ExecutorConfig, + desc catalog.TableDescriptor, + schemaTS hlc.Timestamp, + target changefeedbase.Target, + expr string, +) (*Evaluator, error) { + sc, err := ParseChangefeedExpression(expr) + if err != nil { + return nil, err + } + + const splitFamilies = true + norm, err := NormalizeExpression( + context.Background(), execCfg, username.RootUserName(), defaultDBSessionData, desc, schemaTS, + jobspb.ChangefeedTargetSpecification{ + Type: target.Type, + TableID: target.TableID, + FamilyName: target.FamilyName, + }, + sc, splitFamilies, + ) + if err != nil { + return nil, err + } + + return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName()) +} + +var defaultDBSessionData = sessiondatapb.SessionData{ + Database: "defaultdb", + SearchPath: sessiondata.DefaultSearchPath.GetPathArray(), +} diff --git a/pkg/ccl/changefeedccl/cdceval/func_resolver.go b/pkg/ccl/changefeedccl/cdceval/func_resolver.go index f5b0d0ee3fe7..3c46b0d6b164 100644 --- a/pkg/ccl/changefeedccl/cdceval/func_resolver.go +++ b/pkg/ccl/changefeedccl/cdceval/func_resolver.go @@ -12,15 +12,17 @@ import ( "context" "strings" + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" ) // CDCFunctionResolver is a function resolver specific used by CDC expression // evaluation. -type CDCFunctionResolver struct{} +type CDCFunctionResolver struct { + prevRowFn *tree.ResolvedFunctionDefinition +} // ResolveFunction implements FunctionReferenceResolver interface. func (rs *CDCFunctionResolver) ResolveFunction( @@ -31,28 +33,25 @@ func (rs *CDCFunctionResolver) ResolveFunction( return nil, err } - if fn.ExplicitSchema { - // CDC functions don't have schema prefixes. So if given explicit schema - // name, we only need to look at other non-CDC builtin function. - funcDef, err := tree.GetBuiltinFuncDefinition(fn, path) - if err != nil { - return nil, err - } - if funcDef == nil { - return nil, errors.AssertionFailedf("function %s does not exist", fn.String()) - } - return funcDef, nil + // Check CDC function first. + cdcFuncDef, found := cdcFunctions[fn.Object()] + if !found { + // Try a bit harder + cdcFuncDef, found = cdcFunctions[strings.ToLower(fn.Object())] } - // Check CDC function first. - if cdcFuncDef, found := cdcFunctions[fn.Object()]; found { - return cdcFuncDef, nil + if !found { + // Try internal cdc function. + if funDef := rs.resolveInternalCDCFn(name); funDef != nil { + return funDef, nil + } } - if cdcFuncDef, found := cdcFunctions[strings.ToLower(fn.Object())]; found { + if found && cdcFuncDef != useDefaultBuiltin { return cdcFuncDef, nil } + // Resolve builtin. funcDef, err := tree.GetBuiltinFuncDefinition(fn, path) if err != nil { return nil, err @@ -63,12 +62,6 @@ func (rs *CDCFunctionResolver) ResolveFunction( return funcDef, nil } -// WrapFunction implements the CustomBuiltinFunctionWrapper interface. -func (rs *CDCFunctionResolver) WrapFunction(name string) (*tree.ResolvedFunctionDefinition, error) { - un := tree.MakeUnresolvedName(name) - return rs.ResolveFunction(context.Background(), &un, &sessiondata.DefaultSearchPath) -} - // ResolveFunctionByOID implements FunctionReferenceResolver interface. func (rs *CDCFunctionResolver) ResolveFunctionByOID( ctx context.Context, oid oid.Oid, @@ -77,3 +70,41 @@ func (rs *CDCFunctionResolver) ResolveFunctionByOID( // resolve function by OID. return "", nil, errors.AssertionFailedf("unimplemented yet") } + +// resolveInternalCDCFn resolves special internal functions we install for CDC. +// Resolve functions which are used internally by CDC, but are not exposed to +// end users. +func (rs *CDCFunctionResolver) resolveInternalCDCFn( + name *tree.UnresolvedName, +) *tree.ResolvedFunctionDefinition { + fnName := name.Parts[0] + switch name.NumParts { + case 1: + case 2: + if name.Parts[1] != "crdb_internal" { + return nil + } + default: + return nil + } + + switch fnName { + case prevRowFnName.Parts[0]: + return rs.prevRowFn + } + return nil +} + +func (rs *CDCFunctionResolver) setPrevFuncForEventDescriptor( + d *cdcevent.EventDescriptor, +) *tree.DTuple { + if d == nil { + rs.prevRowFn = nil + return nil + } + + tupleType := cdcPrevType(d) + rowTuple := tree.NewDTupleWithLen(tupleType, len(tupleType.InternalType.TupleContents)) + rs.prevRowFn = makePrevRowFn(rowTuple.ResolvedType()) + return rowTuple +} diff --git a/pkg/ccl/changefeedccl/cdceval/func_resolver_test.go b/pkg/ccl/changefeedccl/cdceval/func_resolver_test.go index 2749930abc8f..2974cf8b3a02 100644 --- a/pkg/ccl/changefeedccl/cdceval/func_resolver_test.go +++ b/pkg/ccl/changefeedccl/cdceval/func_resolver_test.go @@ -47,12 +47,12 @@ func TestResolveFunction(t *testing.T) { }, { testName: "cdc name without schema", - fnName: tree.UnresolvedName{NumParts: 1, Parts: tree.NameParts{"cdc_prev", "", "", ""}}, + fnName: tree.UnresolvedName{NumParts: 1, Parts: tree.NameParts{"cdc_mvcc_timestamp", "", "", ""}}, expectedSchema: "public", }, { testName: "uppercase cdc name without schema", - fnName: tree.UnresolvedName{NumParts: 1, Parts: tree.NameParts{"cdc_PREV", "", "", ""}}, + fnName: tree.UnresolvedName{NumParts: 1, Parts: tree.NameParts{"cdc_mVCC_timeStamp", "", "", ""}}, expectedSchema: "public", }, } diff --git a/pkg/ccl/changefeedccl/cdceval/functions.go b/pkg/ccl/changefeedccl/cdceval/functions.go index 34c200b7f807..2956025b8d83 100644 --- a/pkg/ccl/changefeedccl/cdceval/functions.go +++ b/pkg/ccl/changefeedccl/cdceval/functions.go @@ -13,50 +13,63 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" - jsonb "github.com/cockroachdb/cockroach/pkg/util/json" + "github.com/cockroachdb/errors" ) -// In general, we want to only support functions that produce the same -// value given the same data -- i.e. immutable functions. -// However, we can provide reasonable overrides to a small set of stable -// functions that make sense in the context of CDC. -var supportedVolatileBuiltinFunctions = makeStringSet( - // These functions can be supported given that we set the statement and - // transaction timestamp to be equal to MVCC timestamp of the event. - "statement_timestamp", - "transaction_timestamp", - "timezone", +// sentinel value indicating we should use default builtin +// implementation. +var useDefaultBuiltin *tree.ResolvedFunctionDefinition + +// cdcFunctions is a list of supported stable and immutable builtin functions. +// Some builtin functions have alternative implementation. +// This map also specifies the list of supported CDC specific function definitions. +// +// Any builtin stable (or volatile) function that accesses eval.Context will +// have to have an override due to the fact that we should not manipulate +// underlying eval.Context directly (this is due to the fact that DistSQL copies eval +// context, and, in addition, eval.Context is not thread safe). +// Instead, we have to do so through annotations object stored inside eval.Context. +var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{ + // {statement,transaction}_timestamp functions can be supported given that we + // set the statement and transaction timestamp to be equal to MVCC timestamp + // of the event. However, we provide our own override which uses annotation to + // return the MVCC timestamp of the update. + "statement_timestamp": makeBuiltinOverride( + tree.FunDefs["statement_timestamp"], timestampBuiltinOverloads..., + ), + "transaction_timestamp": makeBuiltinOverride( + tree.FunDefs["transaction_timestamp"], timestampBuiltinOverloads..., + ), + + "timezone": useDefaultBuiltin, // jsonb functions are stable because they depend on eval // context DataConversionConfig - "jsonb_build_array", - "jsonb_build_object", - "to_json", - "to_jsonb", - "row_to_json", + "jsonb_build_array": useDefaultBuiltin, + "jsonb_build_object": useDefaultBuiltin, + "to_json": useDefaultBuiltin, + "to_jsonb": useDefaultBuiltin, + "row_to_json": useDefaultBuiltin, // Misc functions that depend on eval context. - "overlaps", - "pg_collation_for", - "pg_typeof", - "quote_literal", - "quote_nullable", + "overlaps": useDefaultBuiltin, + "pg_collation_for": useDefaultBuiltin, + "pg_typeof": useDefaultBuiltin, + "quote_literal": useDefaultBuiltin, + "quote_nullable": useDefaultBuiltin, // TODO(yevgeniy): Support geometry. //"st_asgeojson", //"st_estimatedextent", -) -// CDC Specific functions. -// TODO(yevgeniy): Finalize function naming: e.g. cdc.is_delete() vs cdc_is_delete() -var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{ "cdc_is_delete": makeCDCBuiltIn( "cdc_is_delete", tree.Overload{ @@ -69,14 +82,19 @@ var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{ } return tree.DBoolFalse, nil }, - Info: "Returns true if the event is a deletion", - Volatility: volatility.Stable, + Info: "Returns true if the event is a deletion", + // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()), + // we should not mark custom CDC functions as stable. Doing so will cause + // optimizer to (constant) fold this function during optimization step -- something + // we definitely don't want to do because we need to evaluate those functions + // for each event. + Volatility: volatility.Volatile, }), "cdc_mvcc_timestamp": cdcTimestampBuiltin( "cdc_mvcc_timestamp", "Returns event MVCC HLC timestamp", func(rowEvalCtx *rowEvalContext) hlc.Timestamp { - return rowEvalCtx.mvccTS + return rowEvalCtx.updatedRow.MvccTimestamp }, ), "cdc_updated_timestamp": cdcTimestampBuiltin( @@ -86,31 +104,12 @@ var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{ return rowEvalCtx.updatedRow.SchemaTS }, ), - "cdc_prev": makeCDCBuiltIn( - "cdc_prev", - tree.Overload{ - Types: tree.ParamTypes{}, - ReturnType: tree.FixedReturnType(types.Jsonb), - Fn: prevRowAsJSON, - Info: "Returns previous value of a row as JSONB", - Volatility: volatility.Stable, - }), } -// TODO(yevgeniy): Implement additional functions (some ideas, not all should be implemented): -// * cdc_is_delete is easy; what about update? does update include new events? -// * cdc_is_new -- true if event is a new row -// * tuple overload (or cdc_prev_tuple) to return previous value as a tuple -// * cdc_key -- effectively key_in_value where key columns returned as either a tuple or a json. -// * cdc_key_cols -- return key column names; -// * this can come in handy when working with jsonb; for example, emit previous JSONB excluding -// key columns can be done with `SELECT cdc_prev() - cdc_key_cols() -// * cdc_event_family_is(fam): return true if cdc event family is specified family; overload both for -// family ID and family name. -// function can be used to write complex conditionals when dealing with multi-family table(s) +const cdcFnCategory = "CDC builtin" var cdcFnProps = &tree.FunctionProperties{ - Category: "CDC builtin", + Category: cdcFnCategory, } func makeCDCBuiltIn(fnName string, overloads ...tree.Overload) *tree.ResolvedFunctionDefinition { @@ -135,8 +134,13 @@ func cdcTimestampBuiltin( rowEvalCtx := rowEvalContextFromEvalContext(evalCtx) return eval.TimestampToDecimalDatum(tsFn(rowEvalCtx)), nil }, - Info: doc, - Volatility: volatility.Stable, + Info: doc, + // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()), + // we should not mark custom CDC functions as stable. Doing so will cause + // optimizer to (constant) fold this function during optimization step -- something + // we definitely don't want to do because we need to evaluate those functions + // for each event. + Volatility: volatility.Volatile, }, }, ) @@ -146,45 +150,193 @@ func cdcTimestampBuiltin( return tree.QualifyBuiltinFunctionDefinition(def, catconstants.PublicSchemaName) } -func prevRowAsJSON(ctx context.Context, evalCtx *eval.Context, _ tree.Datums) (tree.Datum, error) { - rec := rowEvalContextFromEvalContext(evalCtx) - if rec.memo.prevJSON != nil { - return rec.memo.prevJSON, nil +// cdcPrevType returns a types.T for the tuple corresponding to the +// event descriptor. +func cdcPrevType(desc *cdcevent.EventDescriptor) *types.T { + tupleTypes := make([]*types.T, 0, len(desc.ResultColumns())) + tupleLabels := make([]string, 0, len(desc.ResultColumns())) + + for _, c := range desc.ResultColumns() { + // TODO(yevgeniy): Handle virtual columns in cdc_prev. + // In order to do this, we have to emit default expression with + // all named references replaced with tuple field access. + tupleLabels = append(tupleLabels, c.Name) + tupleTypes = append(tupleTypes, c.Typ) } + return types.MakeLabeledTuple(tupleTypes, tupleLabels) +} - var prevJSON *tree.DJSON - if rec.prevRow.IsInitialized() { - b := jsonb.NewObjectBuilder(0) - if err := rec.prevRow.ForEachColumn().Datum(func(d tree.Datum, col cdcevent.ResultColumn) error { - j, err := tree.AsJSON(d, sessiondatapb.DataConversionConfig{}, time.UTC) - if err != nil { - return err - } - b.Add(col.Name, j) - return nil - }); err != nil { - return nil, err +// makePrevRowFn creates a function to return a tuple corresponding +// to the previous value of the row. +func makePrevRowFn(retType *types.T) *tree.ResolvedFunctionDefinition { + return makeCDCBuiltIn(prevRowFnName.Parts[0], + tree.Overload{ + Types: tree.ParamTypes{}, + ReturnType: tree.FixedReturnType(retType), + Fn: func(ctx context.Context, evalCtx *eval.Context, datums tree.Datums) (tree.Datum, error) { + rec := rowEvalContextFromEvalContext(evalCtx) + return rec.prevRowTuple, nil + }, + Info: "Returns previous value of a row as tuple", + // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()), + // we should not mark custom CDC functions as stable. Doing so will cause + // optimizer to (constant) fold this function during optimization step -- something + // we definitely don't want to do because we need to evaluate those functions + // for each event. + Volatility: volatility.Volatile, + DistsqlBlocklist: true, + }, + ) +} + +// The state of the previous row is made available via cdc_prev tuple. +// This tuple is generated by rewriting expressions using cdc_prev +// (i.e. row_to_json(cdc_prev.*)) to generate a table which returns +// correctly typed tuple: +// +// SELECT ... FROM tbl, (SELECT ((crdb_internal.cdc_prev_row()).*)) AS cdc_prev +// +// The crdb_internal.cdc_prev_row() function is in turn configured to return +// previous row datums. +const prevTupleName = "cdc_prev" + +var prevRowFnName = tree.MakeUnresolvedName("crdb_internal", "cdc_prev_row") + +// checkFunctionSupported checks if the function (expression) is supported. +// Returns (possibly modified) function expression if supported; error otherwise. +func checkFunctionSupported( + ctx context.Context, fnCall *tree.FuncExpr, semaCtx *tree.SemaContext, +) (*tree.FuncExpr, error) { + if semaCtx.FunctionResolver == nil { + return nil, errors.AssertionFailedf("function resolver must be configured for CDC") + } + + // Returns function call expression, provided the function with specified + // name is supported. + cdcFunctionWithOverride := func(name string, fnCall *tree.FuncExpr) (*tree.FuncExpr, error) { + funDef, isSafe := cdcFunctions[name] + if !isSafe { + return nil, pgerror.Newf(pgcode.UndefinedFunction, "function %q unsupported by CDC", name) } - prevJSON = tree.NewDJSON(b.Build()) - } else { - prevJSON = tree.NewDJSON(jsonb.NullJSONValue) + if funDef != useDefaultBuiltin { + // Install our override + fnCall.Func = tree.ResolvableFunctionReference{FunctionReference: funDef} + } + return fnCall, nil } - rec.memo.prevJSON = prevJSON - return prevJSON, nil -} + switch fn := fnCall.Func.FunctionReference.(type) { + case *tree.UnresolvedName: + funDef, err := semaCtx.FunctionResolver.ResolveFunction(ctx, fn, semaCtx.SearchPath) + if err != nil { + return nil, pgerror.Newf(pgcode.UndefinedFunction, "function %q unsupported by CDC", fnCall.Func.String()) + } + fnCall = &tree.FuncExpr{ + Func: tree.ResolvableFunctionReference{FunctionReference: funDef}, + Type: fnCall.Type, + Exprs: fnCall.Exprs, + } + if _, isCDCFn := cdcFunctions[funDef.Name]; isCDCFn { + return fnCall, nil + } + return checkFunctionSupported(ctx, fnCall, semaCtx) + case *tree.ResolvedFunctionDefinition: + var fnVolatility volatility.V + for _, overload := range fn.Overloads { + // Aggregates, generators and window functions are not supported. + switch overload.Class { + case tree.AggregateClass, tree.GeneratorClass, tree.WindowClass: + return nil, pgerror.Newf(pgcode.UndefinedFunction, "function %q unsupported by CDC", fn.Name) + } + if overload.Volatility > fnVolatility { + fnVolatility = overload.Volatility + } + } + if fnVolatility <= volatility.Immutable { + // Remaining immutable functions are safe. + return fnCall, nil + } + + return cdcFunctionWithOverride(fn.Name, fnCall) + case *tree.FunctionDefinition: + switch fn.Class { + case tree.AggregateClass, tree.GeneratorClass, tree.WindowClass: + return nil, pgerror.Newf(pgcode.UndefinedFunction, "function %q unsupported by CDC", fn.Name) + } -func makeStringSet(vals ...string) map[string]struct{} { - m := make(map[string]struct{}, len(vals)) - for _, v := range vals { - m[v] = struct{}{} + var fnVolatility volatility.V + if fnCall.ResolvedOverload() != nil { + if _, isCDC := cdcFunctions[fn.Name]; isCDC { + return fnCall, nil + } + fnVolatility = fnCall.ResolvedOverload().Volatility + } else { + // Pick highest volatility overload. + for _, o := range fn.Definition { + if o.Volatility > fnVolatility { + fnVolatility = o.Volatility + } + } + } + if fnVolatility <= volatility.Immutable { + // Remaining immutable functions are safe. + return fnCall, nil + } + + return cdcFunctionWithOverride(fn.Name, fnCall) + default: + return nil, errors.AssertionFailedf("unexpected function expression of type %T", fn) } - return m } // TestingEnableVolatileFunction allows functions with the given name (lowercase) // to be used in expressions if their volatility level would disallow them by default. // Used for testing. func TestingEnableVolatileFunction(fnName string) { - supportedVolatileBuiltinFunctions[fnName] = struct{}{} + cdcFunctions[fnName] = useDefaultBuiltin +} + +// For some functions (specifically the volatile ones), we do +// not want to use the provided builtin. Instead, we opt for +// our own function definition. +func makeBuiltinOverride( + builtin *tree.FunctionDefinition, overloads ...tree.Overload, +) *tree.ResolvedFunctionDefinition { + props := builtin.FunctionProperties + override := tree.NewFunctionDefinition(builtin.Name, &props, overloads) + // The schema name is actually not important since CDC doesn't use any user + // defined functions. And, we're sure that we always return the first + // function definition found. + return tree.QualifyBuiltinFunctionDefinition(override, catconstants.PublicSchemaName) +} + +// tree.Overload definitions for statement_timestamp and transaction_timestamp functions. +var timestampBuiltinOverloads = []tree.Overload{ + { + Types: tree.ParamTypes{}, + ReturnType: tree.FixedReturnType(types.TimestampTZ), + PreferredOverload: true, + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + rowEvalCtx := rowEvalContextFromEvalContext(evalCtx) + return tree.MakeDTimestampTZ(rowEvalCtx.updatedRow.MvccTimestamp.GoTime(), time.Microsecond) + }, + Info: "Returns MVCC timestamp of the event", + // NB: Default builtin implementation uses volatility.Stable + // We override volatility to be Volatile so that function + // is not folded. + Volatility: volatility.Volatile, + }, + { + Types: tree.ParamTypes{}, + ReturnType: tree.FixedReturnType(types.Timestamp), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + rowEvalCtx := rowEvalContextFromEvalContext(evalCtx) + return tree.MakeDTimestamp(rowEvalCtx.updatedRow.MvccTimestamp.GoTime(), time.Microsecond) + }, + Info: "Returns MVCC timestamp of the event", + // NB: Default builtin implementation uses volatility.Stable + // We override volatility to be Volatile so that function + // is not folded. + Volatility: volatility.Volatile, + }, } diff --git a/pkg/ccl/changefeedccl/cdceval/functions_test.go b/pkg/ccl/changefeedccl/cdceval/functions_test.go index d977cf1761d5..f4b21757be41 100644 --- a/pkg/ccl/changefeedccl/cdceval/functions_test.go +++ b/pkg/ccl/changefeedccl/cdceval/functions_test.go @@ -17,11 +17,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/hlc" jsonb "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -43,34 +47,48 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { desc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") ctx := context.Background() + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + + semaCtx := tree.MakeSemaContext() + testRow := makeEventRow(t, desc, s.Clock().Now(), false, hlc.Timestamp{}) + defer configSemaForCDC(&semaCtx, testRow.EventDescriptor)() t.Run("time", func(t *testing.T) { // We'll run tests against some future time stamp to ensure // that time functions use correct values. futureTS := s.Clock().Now().Add(int64(60*time.Minute), 0) - expectTSTZ := func() string { + expectTSTZ := func(ts hlc.Timestamp) string { t.Helper() - d, err := tree.MakeDTimestampTZ(futureTS.GoTime(), time.Microsecond) + d, err := tree.MakeDTimestampTZ(ts.GoTime(), time.Microsecond) require.NoError(t, err) return tree.AsStringWithFlags(d, tree.FmtExport) - }() + } for _, tc := range []struct { fn string expect string }{ - {fn: "statement_timestamp", expect: expectTSTZ}, - {fn: "transaction_timestamp", expect: expectTSTZ}, + {fn: "statement_timestamp", expect: expectTSTZ(futureTS)}, + {fn: "transaction_timestamp", expect: expectTSTZ(futureTS)}, } { t.Run(tc.fn, func(t *testing.T) { - testRow := cdcevent.TestingMakeEventRow(desc, 0, - randEncDatumRow(t, desc, 0), false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf("SELECT %s()", tc.fn)) + testRow := makeEventRow(t, desc, s.Clock().Now(), false, futureTS) + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + fmt.Sprintf("SELECT %s() FROM foo", tc.fn)) require.NoError(t, err) - p, err := e.evalProjection(ctx, testRow, futureTS, testRow) + defer e.Close() + + p, err := e.Eval(ctx, testRow, testRow) require.NoError(t, err) require.Equal(t, map[string]string{tc.fn: tc.expect}, slurpValues(t, p)) + + // Emit again, this time advancing MVCC timestamp of the row. + // We want to make sure that optimizer did not constant fold the call + // to the function, even though this function is marked stable. + testRow.MvccTimestamp = testRow.MvccTimestamp.Add(int64(time.Hour), 0) + p, err = e.Eval(ctx, testRow, testRow) + require.NoError(t, err) + require.Equal(t, map[string]string{tc.fn: expectTSTZ(testRow.MvccTimestamp)}, slurpValues(t, p)) }) } @@ -80,14 +98,14 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { // timezone. Since we don't do any special setup with session data, the // default timezone is UTC. We'll use a "strange" timezone of -1h33m from // UTC to test conversion. - testRow := cdcevent.TestingMakeEventRow(desc, 0, - randEncDatumRow(t, desc, 0), false) - - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf("SELECT timezone('+01:33:00', '%s'::time)", + testRow := makeEventRow(t, desc, s.Clock().Now(), false, futureTS) + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + fmt.Sprintf("SELECT timezone('+01:33:00', '%s'::time) FROM foo", futureTS.GoTime().Format("15:04:05"))) require.NoError(t, err) - p, err := e.evalProjection(ctx, testRow, futureTS, testRow) + defer e.Close() + + p, err := e.Eval(ctx, testRow, testRow) require.NoError(t, err) expectedTZ := fmt.Sprintf("%s-01:33:00", @@ -97,14 +115,16 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { }) t.Run("cdc_is_delete", func(t *testing.T) { - for _, expectDelete := range []bool{true, false} { - testRow := cdcevent.TestingMakeEventRow(desc, 0, - randEncDatumRow(t, desc, 0), expectDelete) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT cdc_is_delete()") - require.NoError(t, err) + schemaTS := s.Clock().Now() + testRow := makeEventRow(t, desc, schemaTS, false, s.Clock().Now()) + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + "SELECT cdc_is_delete() FROM foo") + require.NoError(t, err) + defer e.Close() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), testRow) + for _, expectDelete := range []bool{true, false} { + testRow := makeEventRow(t, desc, schemaTS, expectDelete, s.Clock().Now()) + p, err := e.Eval(ctx, testRow, testRow) require.NoError(t, err) require.Equal(t, map[string]string{"cdc_is_delete": fmt.Sprintf("%t", expectDelete)}, @@ -119,77 +139,55 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { return j } - t.Run("cdc_prev", func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT cdc_prev()") - require.NoError(t, err) - - // When previous row is not set -- i.e. if running without diff, cdc_prev returns - // null json. - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) - require.NoError(t, err) - require.Equal(t, map[string]string{"cdc_prev": jsonb.NullJSONValue.String()}, slurpValues(t, p)) + t.Run("cdc_{mvcc,updated}_timestamp", func(t *testing.T) { + for _, cast := range []string{"", "::decimal", "::string"} { + t.Run(cast, func(t *testing.T) { + schemaTS := s.Clock().Now() + mvccTS := schemaTS.Add(int64(30*time.Minute), 0) + testRow := makeEventRow(t, desc, schemaTS, false, mvccTS) + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, fmt.Sprintf( + "SELECT cdc_mvcc_timestamp()%[1]s as mvcc, cdc_updated_timestamp()%[1]s as updated FROM foo", cast, + )) + require.NoError(t, err) + defer e.Close() - // Otherwise, expect to get JSONB. - b := jsonb.NewObjectBuilder(len(rowDatums)) - for i, d := range rowDatums { - b.Add(desc.PublicColumns()[i].GetName(), mustParseJSON(d.Datum)) + p, err := e.Eval(ctx, testRow, testRow) + require.NoError(t, err) + require.Equal(t, + map[string]string{ + "mvcc": eval.TimestampToDecimalDatum(mvccTS).String(), + "updated": eval.TimestampToDecimalDatum(schemaTS).String(), + }, + slurpValues(t, p), + ) + }) } - - expectedJSON := b.Build() - p, err = e.evalProjection(ctx, testRow, s.Clock().Now(), testRow) - require.NoError(t, err) - require.Equal(t, map[string]string{"cdc_prev": expectedJSON.String()}, slurpValues(t, p)) }) - for _, cast := range []string{"", "::decimal", "::string"} { - t.Run(fmt.Sprintf("cdc_{mvcc,updated}_timestamp()%s", cast), func(t *testing.T) { - schemaTS := s.Clock().Now().Add(int64(60*time.Minute), 0) - mvccTS := schemaTS.Add(int64(30*time.Minute), 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, - randEncDatumRow(t, desc, 0), false) - testRow.EventDescriptor.SchemaTS = schemaTS - - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf( - "SELECT cdc_mvcc_timestamp()%[1]s as mvcc, cdc_updated_timestamp()%[1]s as updated", cast, - )) - require.NoError(t, err) - - p, err := e.evalProjection(ctx, testRow, mvccTS, testRow) - require.NoError(t, err) - require.Equal(t, - map[string]string{ - "mvcc": eval.TimestampToDecimalDatum(mvccTS).String(), - "updated": eval.TimestampToDecimalDatum(schemaTS).String(), - }, - slurpValues(t, p)) - }) - } - t.Run("pg_collation_for", func(t *testing.T) { - testRow := cdcevent.TestingMakeEventRow(desc, 0, - randEncDatumRow(t, desc, 0), false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - `SELECT pg_collation_for('hello' COLLATE de_DE) AS col`) + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + `SELECT pg_collation_for('hello' COLLATE de_DE) AS col FROM foo`) require.NoError(t, err) + defer e.Close() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), testRow) + p, err := e.Eval(ctx, testRow, testRow) require.NoError(t, err) - require.Equal(t, map[string]string{"col": "\"de_de\""}, slurpValues(t, p)) + require.Equal(t, map[string]string{"col": "\"de_DE\""}, slurpValues(t, p)) }) for _, fn := range []string{"to_json", "to_jsonb"} { t.Run(fn, func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf("SELECT %s(a)", fn)) + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + rowDatums := testRow.EncDatums() + + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(a) FROM foo", fn)) require.NoError(t, err) + defer e.Close() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{fn: mustParseJSON(rowDatums[0].Datum).String()}, @@ -198,11 +196,13 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { } t.Run("row_to_json", func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT row_to_json(row(a, b, c))") + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + + rowDatums := testRow.EncDatums() + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + "SELECT row_to_json(row(a, b, c)) FROM foo") require.NoError(t, err) + defer e.Close() b := jsonb.NewObjectBuilder(len(rowDatums)) for i, d := range rowDatums { @@ -210,17 +210,18 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { } expectedJSON := b.Build() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{"row_to_json": expectedJSON.String()}, slurpValues(t, p)) }) t.Run("jsonb_build_array", func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT jsonb_build_array(a, a, 42) AS three_ints") + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + rowDatums := testRow.EncDatums() + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + "SELECT jsonb_build_array(a, a, 42) AS three_ints FROM foo") require.NoError(t, err) + defer e.Close() b := jsonb.NewArrayBuilder(3) j := mustParseJSON(rowDatums[0].Datum) @@ -229,17 +230,18 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { b.Add(jsonb.FromInt(42)) expectedJSON := b.Build() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{"three_ints": expectedJSON.String()}, slurpValues(t, p)) }) t.Run("jsonb_build_object", func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - "SELECT jsonb_build_object('a', a, 'b', b, 'c', c) AS obj") + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + rowDatums := testRow.EncDatums() + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + "SELECT jsonb_build_object('a', a, 'b', b, 'c', c) AS obj FROM foo") require.NoError(t, err) + defer e.Close() b := jsonb.NewObjectBuilder(3) b.Add("a", mustParseJSON(rowDatums[0].Datum)) @@ -247,22 +249,22 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { b.Add("c", mustParseJSON(rowDatums[2].Datum)) expectedJSON := b.Build() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{"obj": expectedJSON.String()}, slurpValues(t, p)) }) for _, fn := range []string{"quote_literal", "quote_nullable"} { // These functions have overloads; call the one that's stable overload - // (i.e. one that needs to convert types.Any to string. + // (i.e. one that needs to convert types.Any to string). t.Run(fn, func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf("SELECT %s(42)", fn)) + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(42) FROM foo", fn)) require.NoError(t, err) + defer e.Close() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{fn: fmt.Sprintf("'%s'", jsonb.FromInt(42).String())}, @@ -272,13 +274,14 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { // overlaps has many overloads; most of them are immutable, but 1 is stable. t.Run("overlaps", func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - e, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - `SELECT overlaps(transaction_timestamp(), interval '0', transaction_timestamp(), interval '-1s')`) + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) + + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + `SELECT overlaps(transaction_timestamp(), interval '0', transaction_timestamp(), interval '-1s') FROM foo`) require.NoError(t, err) + defer e.Close() - p, err := e.evalProjection(ctx, testRow, s.Clock().Now(), cdcevent.Row{}) + p, err := e.Eval(ctx, testRow, cdcevent.Row{}) require.NoError(t, err) require.Equal(t, map[string]string{"overlaps": "false"}, slurpValues(t, p)) }) @@ -286,6 +289,7 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { // Test that cdc specific functions correctly resolve overload, and that an // error is returned when cdc function called with wrong arguments. t.Run("cdc function errors", func(t *testing.T) { + testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now()) // currently, all cdc functions take no args, so call these functions with // some arguments. rng, _ := randutil.NewTestRand() @@ -300,14 +304,47 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) { } } - for fn := range cdcFunctions { - t.Run(fn, func(t *testing.T) { - rowDatums := randEncDatumRow(t, desc, 0) - testRow := cdcevent.TestingMakeEventRow(desc, 0, rowDatums, false) - _, err := makeExprEval(t, s.ClusterSettings(), testRow.EventDescriptor, - fmt.Sprintf("SELECT %s(%s)", fn, fnArgs())) - require.Regexp(t, "unknown signature", err) - }) + for fn, def := range cdcFunctions { + // Run this test only for CDC specific functions. + if def != useDefaultBuiltin && def.Overloads[0].FunctionProperties.Category == cdcFnCategory { + t.Run(fn, func(t *testing.T) { + e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, + fmt.Sprintf("SELECT %s(%s) FROM foo", fn, fnArgs())) + require.NoError(t, err) + _, err = e.Eval(ctx, testRow, testRow) + require.Regexp(t, "unknown signature", err) + }) + } } }) } + +func makeEventRow( + t *testing.T, + desc catalog.TableDescriptor, + schemaTS hlc.Timestamp, + deleted bool, + mvccTS hlc.Timestamp, +) cdcevent.Row { + t.Helper() + datums := randEncDatumPrimaryFamily(t, desc) + r := cdcevent.TestingMakeEventRow(desc, 0, datums, deleted) + r.SchemaTS = schemaTS + r.MvccTimestamp = mvccTS + return r +} + +func newEvaluator( + execCfg *sql.ExecutorConfig, semaCtx *tree.SemaContext, ed *cdcevent.EventDescriptor, expr string, +) (*Evaluator, error) { + sc, err := ParseChangefeedExpression(expr) + if err != nil { + return nil, err + } + + norm, err := normalizeSelectClause(context.Background(), semaCtx, sc, ed) + if err != nil { + return nil, err + } + return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName()) +} diff --git a/pkg/ccl/changefeedccl/cdceval/parse.go b/pkg/ccl/changefeedccl/cdceval/parse.go index 404ce04c54e5..d02337a1047d 100644 --- a/pkg/ccl/changefeedccl/cdceval/parse.go +++ b/pkg/ccl/changefeedccl/cdceval/parse.go @@ -9,20 +9,30 @@ package cdceval import ( + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) // AsStringUnredacted returns unredacted string representation. -// Method is intended to be used when serializing node formatter to be stored -// in protocol messages. +// Method should be used when serializing node formatter to be stored in +// protocol messages. func AsStringUnredacted(n tree.NodeFormatter) string { return tree.AsStringWithFlags(n, tree.FmtParsable|tree.FmtShowPasswords) } -// ParseChangefeedExpression is a helper to parse changefeed "select clause". +// ParseChangefeedExpression is a helper to parse changefeed "select clause", +// and perform minimal validation checks. func ParseChangefeedExpression(selectClause string) (*tree.SelectClause, error) { + sc, err := parseChangefeedExpression(selectClause) + if err != nil { + return nil, changefeedbase.WithTerminalError(err) + } + return sc, nil +} + +func parseChangefeedExpression(selectClause string) (*tree.SelectClause, error) { stmt, err := parser.ParseOne(selectClause) if err != nil { return nil, err @@ -32,16 +42,6 @@ func ParseChangefeedExpression(selectClause string) (*tree.SelectClause, error) return sc, nil } } - return nil, errors.AssertionFailedf("expected select clause, found %T", stmt.AST) -} -// tableNameOrAlias returns tree.TableName for the table expression. -func tableNameOrAlias(name string, expr tree.TableExpr) *tree.TableName { - switch t := expr.(type) { - case *tree.AliasedTableExpr: - return tree.NewUnqualifiedTableName(t.As.Alias) - case *tree.TableRef: - return tree.NewUnqualifiedTableName(t.As.Alias) - } - return tree.NewUnqualifiedTableName(tree.Name(name)) + return nil, errors.AssertionFailedf("expected select clause, found %T", stmt.AST) } diff --git a/pkg/ccl/changefeedccl/cdceval/plan.go b/pkg/ccl/changefeedccl/cdceval/plan.go new file mode 100644 index 000000000000..b6d67c304671 --- /dev/null +++ b/pkg/ccl/changefeedccl/cdceval/plan.go @@ -0,0 +1,145 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package cdceval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +// NormalizeExpression normalizes select clause. Returns normalized (and rewritten) +// expression which can be serialized into job record. +func NormalizeExpression( + ctx context.Context, + execCfg *sql.ExecutorConfig, + user username.SQLUsername, + sd sessiondatapb.SessionData, + descr catalog.TableDescriptor, + schemaTS hlc.Timestamp, + target jobspb.ChangefeedTargetSpecification, + sc *tree.SelectClause, + splitFams bool, +) (norm *NormalizedSelectClause, err error) { + if err := withPlanner(ctx, execCfg, user, schemaTS, sd, + func(ctx context.Context, execCtx sql.JobExecContext) error { + norm, err = normalizeAndValidateSelectForTarget( + ctx, execCfg, descr, schemaTS, target, sc, false /* keyOnly */, splitFams, execCtx.SemaCtx()) + if err != nil { + return changefeedbase.WithTerminalError(err) + } + + defer configSemaForCDC(execCtx.SemaCtx(), norm.desc)() + // Plan execution; this steps triggers optimizer, which + // performs various validation steps. + _, err := sql.PlanCDCExpression(ctx, execCtx, norm.SelectStatement()) + if err == nil { + return nil + } + + // Wrap error with some additional information. + if descr.NumFamilies() > 1 && pgerror.GetPGCode(err) == pgcode.UndefinedColumn { + err = errors.WithHintf(err, + "column may not exist in the target column family %q", target.FamilyName) + } + return err + }, + ); err != nil { + return nil, err + } + + return norm, nil +} + +// SpansForExpression returns spans that must be scanned in order to evaluate +// changefeed expression. Select clause expression assumed to be normalized. +func SpansForExpression( + ctx context.Context, + execCfg *sql.ExecutorConfig, + user username.SQLUsername, + sd sessiondatapb.SessionData, + descr catalog.TableDescriptor, + schemaTS hlc.Timestamp, + target jobspb.ChangefeedTargetSpecification, + sc *tree.SelectClause, +) (_ roachpb.Spans, err error) { + d, err := newEventDescriptorForTarget(descr, target, schemaTS, false, false) + if err != nil { + return nil, err + } + + var plan sql.CDCExpressionPlan + if err := withPlanner(ctx, execCfg, user, schemaTS, sd, + func(ctx context.Context, execCtx sql.JobExecContext) error { + defer configSemaForCDC(execCtx.SemaCtx(), d)() + + plan, err = sql.PlanCDCExpression(ctx, execCtx, &tree.Select{Select: sc}) + if err == nil { + return nil + } + + // Wrap error with some additional information. + if descr.NumFamilies() > 1 && pgerror.GetPGCode(err) == pgcode.UndefinedColumn { + err = errors.WithHintf(err, + "column may not exist in the target column family %q", target.FamilyName) + } + return err + }, + ); err != nil { + return nil, err + } + + return plan.Spans, nil +} + +// withPlanner is a helper which invokes provided function inside +// a DescsTxn transaction to ensure that descriptors get acquired +// as of correct schema timestamp. +func withPlanner( + ctx context.Context, + execCfg *sql.ExecutorConfig, + user username.SQLUsername, + schemaTS hlc.Timestamp, + sd sessiondatapb.SessionData, + fn func(ctx context.Context, execCtx sql.JobExecContext) error, +) error { + return sql.DescsTxn(ctx, execCfg, + func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error { + if err := txn.SetFixedTimestamp(ctx, schemaTS); err != nil { + return err + } + + // Current implementation relies on row-by-row evaluation; + // so, ensure vectorized engine is off. + sd.VectorizeMode = sessiondatapb.VectorizeOff + planner, cleanup := sql.NewInternalPlanner( + "cdc-expr", txn, + user, + &sql.MemoryMetrics{}, + execCfg, + sd, + sql.WithDescCollection(col), + ) + defer cleanup() + return fn(ctx, planner.(sql.JobExecContext)) + }) +} diff --git a/pkg/ccl/changefeedccl/cdceval/plan_test.go b/pkg/ccl/changefeedccl/cdceval/plan_test.go new file mode 100644 index 000000000000..d56f4c68d1b3 --- /dev/null +++ b/pkg/ccl/changefeedccl/cdceval/plan_test.go @@ -0,0 +1,280 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package cdceval + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestCanPlanCDCExpressions(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(context.Background()) + + sqlDB := sqlutils.MakeSQLRunner(db) + sqlDB.ExecMultiple(t, + `CREATE TYPE status AS ENUM ('open', 'closed', 'inactive')`, + `CREATE SCHEMA alt`, + `CREATE TYPE alt.status AS ENUM ('alt_open', 'alt_closed', 'alt_inactive')`, + `CREATE TABLE foo ( +a INT PRIMARY KEY, +status status, +alt alt.status, +extra STRING, +FAMILY main (a, status, alt), +FAMILY extra (extra) +)`, + `CREATE TABLE bar (a INT PRIMARY KEY, b string)`, + ) + + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + codec := execCfg.Codec + fooDesc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") + statusT := fooDesc.AllColumns()[1].GetType() + altStatusT := fooDesc.AllColumns()[2].GetType() + primarySpan := fooDesc.PrimaryIndexSpan(codec) + + pkEnd := primarySpan.EndKey + fooID := fooDesc.GetID() + + ctx := context.Background() + schemaTS := s.Clock().Now() + eventDesc, err := newEventDescriptorForTarget( + fooDesc, jobspb.ChangefeedTargetSpecification{}, schemaTS, false, false) + require.NoError(t, err) + extraFamDesc, err := newEventDescriptorForTarget( + fooDesc, jobspb.ChangefeedTargetSpecification{ + Type: jobspb.ChangefeedTargetSpecification_COLUMN_FAMILY, + FamilyName: "extra", + }, schemaTS, false, false) + require.NoError(t, err) + + rc := func(n string, typ *types.T) colinfo.ResultColumn { + return colinfo.ResultColumn{Name: n, Typ: typ} + } + mainColumns := colinfo.ResultColumns{ + rc("a", types.Int), rc("status", statusT), rc("alt", altStatusT), + } + extraColumns := colinfo.ResultColumns{rc("a", types.Int), rc("extra", types.String)} + + checkPresentation := func(t *testing.T, expected, found colinfo.ResultColumns) { + t.Helper() + require.Equal(t, len(expected), len(found), "e=%v f=%v", expected, found) + for i := 0; i < len(found); i++ { + require.Equal(t, expected[i].Name, found[i].Name, "e=%v f=%v", expected[i], found[i]) + require.True(t, expected[i].Typ.Equal(found[i].Typ), "e=%v f=%v", expected[i], found[i]) + } + } + + for _, tc := range []struct { + name string + desc catalog.TableDescriptor + stmt string + targetFamily string + expectErr string + planSpans roachpb.Spans + presentation colinfo.ResultColumns + }{ + { + name: "reject contradiction", + desc: fooDesc, + stmt: "SELECT * FROM foo WHERE a IS NULL", + expectErr: `does not match any rows`, + }, + { + name: "full table - main", + desc: fooDesc, + stmt: "SELECT * FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: mainColumns, + }, + { + name: "full table - double star", + desc: fooDesc, + stmt: "SELECT *, * FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: append(mainColumns, mainColumns...), + }, + { + name: "full table extra family", + desc: fooDesc, + targetFamily: "extra", + stmt: "SELECT * FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: extraColumns, + }, + { + name: "expression scoped to column family", + desc: fooDesc, + targetFamily: "extra", + stmt: "SELECT a, status, extra FROM foo", + expectErr: `column "foo.status" does not exist`, + }, + { + name: "full table extra family with cdc_prev tuple", + desc: fooDesc, + targetFamily: "extra", + stmt: "SELECT *, cdc_prev FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: append(extraColumns, rc("cdc_prev", cdcPrevType(extraFamDesc))), + }, + { + name: "full table with cdc_prev tuple", + desc: fooDesc, + stmt: "SELECT *, cdc_prev FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: append(mainColumns, rc("cdc_prev", cdcPrevType(eventDesc))), + }, + { + name: "full table with cdc_prev expanded", + desc: fooDesc, + stmt: "SELECT *, cdc_prev.* FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: append(mainColumns, mainColumns...), + }, + { + name: "full table with cdc_prev json", + desc: fooDesc, + stmt: "SELECT *, row_to_json(cdc_prev.*) AS prev_json FROM foo", + planSpans: roachpb.Spans{primarySpan}, + presentation: append(mainColumns, rc("prev_json", types.Jsonb)), + }, + { + name: "a > 10", + desc: fooDesc, + stmt: "SELECT * FROM foo WHERE a > 10", + planSpans: roachpb.Spans{{Key: mkPkKey(t, fooID, 11), EndKey: pkEnd}}, + presentation: mainColumns, + }, + { + name: "a > 10 with cdc_prev", + desc: fooDesc, + stmt: "SELECT * FROM foo WHERE a > 10 AND cdc_prev.status = 'closed'", + planSpans: roachpb.Spans{{Key: mkPkKey(t, fooID, 11), EndKey: pkEnd}}, + presentation: mainColumns, + }, + { + name: "can cast to standard type", + desc: fooDesc, + stmt: "SELECT 'cast'::string AS a, 'type_annotation':::string AS b FROM foo AS bar", + planSpans: roachpb.Spans{primarySpan}, + presentation: colinfo.ResultColumns{rc("a", types.String), rc("b", types.String)}, + }, + } { + t.Run(tc.name, func(t *testing.T) { + sc, err := ParseChangefeedExpression(tc.stmt) + if err != nil { + require.Regexp(t, tc.expectErr, err) + return + } + target := jobspb.ChangefeedTargetSpecification{ + TableID: tc.desc.GetID(), + StatementTimeName: tc.desc.GetName(), + FamilyName: tc.targetFamily, + } + + if tc.targetFamily != "" { + target.Type = jobspb.ChangefeedTargetSpecification_COLUMN_FAMILY + } + + const splitFamilies = true + _, plan, err := normalizeAndPlan(ctx, &execCfg, username.RootUserName(), + defaultDBSessionData, tc.desc, schemaTS, target, sc, splitFamilies) + + if tc.expectErr != "" { + require.Regexp(t, tc.expectErr, err) + return + } + + require.NoError(t, err) + require.Equal(t, tc.planSpans, plan.Spans) + checkPresentation(t, tc.presentation, plan.Presentation) + }) + } +} + +func mkPkKey(t *testing.T, tableID descpb.ID, vals ...int) roachpb.Key { + t.Helper() + + // Encode index id, then each value. + key, err := keyside.Encode( + keys.SystemSQLCodec.TablePrefix(uint32(tableID)), + tree.NewDInt(tree.DInt(1)), encoding.Ascending) + + require.NoError(t, err) + for _, v := range vals { + d := tree.NewDInt(tree.DInt(v)) + key, err = keyside.Encode(key, d, encoding.Ascending) + require.NoError(t, err) + } + + return key +} + +// normalizeAndPlan normalizes select clause, and plans CDC expression execution +// (but does not execute). Returns the plan along with normalized expression. +func normalizeAndPlan( + ctx context.Context, + execCfg *sql.ExecutorConfig, + user username.SQLUsername, + sd sessiondatapb.SessionData, + descr catalog.TableDescriptor, + schemaTS hlc.Timestamp, + target jobspb.ChangefeedTargetSpecification, + sc *tree.SelectClause, + splitFams bool, +) (norm *NormalizedSelectClause, plan sql.CDCExpressionPlan, err error) { + norm, err = NormalizeExpression(ctx, execCfg, user, sd, descr, schemaTS, target, sc, splitFams) + if err != nil { + return nil, sql.CDCExpressionPlan{}, err + } + + d, err := newEventDescriptorForTarget(descr, target, schemaTS, false, false) + if err != nil { + return nil, sql.CDCExpressionPlan{}, err + } + + if err := withPlanner(ctx, execCfg, user, schemaTS, sd, + func(ctx context.Context, execCtx sql.JobExecContext) error { + defer configSemaForCDC(execCtx.SemaCtx(), d)() + + plan, err = sql.PlanCDCExpression(ctx, execCtx, norm.SelectStatement()) + return err + }, + ); err != nil { + return nil, sql.CDCExpressionPlan{}, err + } + + return norm, plan, nil +} diff --git a/pkg/ccl/changefeedccl/cdceval/validation.go b/pkg/ccl/changefeedccl/cdceval/validation.go index f1e27b908681..f72a763ccc8e 100644 --- a/pkg/ccl/changefeedccl/cdceval/validation.go +++ b/pkg/ccl/changefeedccl/cdceval/validation.go @@ -12,7 +12,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" - "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql" @@ -26,134 +25,139 @@ import ( "github.com/lib/pq/oid" ) -// NormalizeAndValidateSelectForTarget normalizes select expression and verifies -// expression is valid for a table and target family. includeVirtual indicates -// if virtual columns should be considered valid in the expressions. -// Normalization steps include: -// - Table name replaces with table reference -// - UDTs values replaced with their physical representation (to keep expression stable -// across data type changes). +// NormalizedSelectClause represents normalized and error checked cdc expression. +// Basically, it is a select clause returned by normalizeSelectClause. +// Methods on this expression modify the select clause in place, but this +// marker type is needed so that we can ensure functions that rely on +// normalized input aren't called out of order. +type NormalizedSelectClause struct { + *tree.SelectClause + desc *cdcevent.EventDescriptor +} + +// RequiresPrev returns true if expression requires access to the previous +// version of the row. +func (n *NormalizedSelectClause) RequiresPrev() bool { + return len(n.From.Tables) > 1 +} + +// SelectStatement returns tree.Select representing this object. +func (n *NormalizedSelectClause) SelectStatement() *tree.Select { + return &tree.Select{Select: n.SelectClause} +} + +// normalizeAndValidateSelectForTarget normalizes select expression and verifies +// expression is valid for a table and target family. // // The normalized (updated) select clause expression can be serialized into protocol // buffer using cdceval.AsStringUnredacted. -func NormalizeAndValidateSelectForTarget( +// TODO(yevgeniy): Add support for virtual columns. +func normalizeAndValidateSelectForTarget( ctx context.Context, - execCtx sql.PlanHookState, + execCfg *sql.ExecutorConfig, desc catalog.TableDescriptor, + schemaTS hlc.Timestamp, target jobspb.ChangefeedTargetSpecification, sc *tree.SelectClause, - includeVirtual bool, keyOnly bool, splitColFams bool, -) (n NormalizedSelectClause, _ jobspb.ChangefeedTargetSpecification, retErr error) { + semaCtx *tree.SemaContext, +) (_ *NormalizedSelectClause, retErr error) { defer func() { - if pan := recover(); pan != nil { - retErr = changefeedbase.WithTerminalError( - errors.Newf("expression currently unsupported in CREATE CHANGEFEED: %s", pan)) + if r := recover(); r != nil { + retErr = errors.Newf("expression (%s) currently unsupported in CREATE CHANGEFEED: %s", + tree.AsString(sc), r) } }() - execCtx.SemaCtx() - execCfg := execCtx.ExecCfg() + if !execCfg.Settings.Version.IsActive(ctx, clusterversion.V22_2EnablePredicateProjectionChangefeed) { - return n, target, errors.Newf( + return nil, errors.Newf( `filters and projections not supported until upgrade to version %s or higher is finalized`, clusterversion.V22_2EnablePredicateProjectionChangefeed.String()) } // This really shouldn't happen as it's enforced by sql.y. if len(sc.From.Tables) != 1 { - return n, target, pgerror.Newf(pgcode.Syntax, "invalid CDC expression: only 1 table supported") + return nil, pgerror.Newf(pgcode.Syntax, + "expected 1 table, found %d", len(sc.From.Tables)) } // Sanity check target and descriptor refer to the same table. if target.TableID != desc.GetID() { - return n, target, errors.AssertionFailedf("target table id (%d) does not match descriptor id (%d)", + return nil, errors.AssertionFailedf("target table id (%d) does not match descriptor id (%d)", target.TableID, desc.GetID()) } - // This method is meant to be called early on when changefeed is created -- - // i.e. during planning. As such, we expect execution context to have - // associated Txn() -- without which we cannot perform normalization. Verify - // this assumption (txn is needed for type resolution). - if execCtx.Txn() == nil { - return n, target, errors.AssertionFailedf("expected non-nil transaction") - } - - // Perform normalization. - var err error - normalized, err := normalizeSelectClause(ctx, *execCtx.SemaCtx(), sc, desc) - if err != nil { - return n, target, err - } - columnVisitor := checkColumnsVisitor{ desc: desc, splitColFams: splitColFams, } - - err = columnVisitor.FindColumnFamilies(normalized) + err := columnVisitor.FindColumnFamilies(sc) if err != nil { - return n, target, err + return nil, err } - - target, err = setTargetType(desc, target, &columnVisitor) + target, err = getExpressionTargetSpecification(desc, target, &columnVisitor) if err != nil { - return n, target, err + return nil, err } - ed, err := newEventDescriptorForTarget(desc, target, schemaTS(execCtx), includeVirtual, keyOnly) + // TODO(yevgeniy): support virtual columns. + const includeVirtual = false + d, err := newEventDescriptorForTarget(desc, target, schemaTS, includeVirtual, keyOnly) if err != nil { - return n, target, err - } - - evalCtx := &execCtx.ExtendedEvalContext().Context - // Try to constrain spans by select clause. We don't care about constrained - // spans here, but constraining spans kicks off optimizer which detects many - // errors. - if _, _, err := constrainSpansBySelectClause( - ctx, execCtx, evalCtx, execCfg.Codec, sc, ed, - ); err != nil { - return n, target, err + return nil, err } - // Construct and initialize evaluator. This performs some static checks, - // and (importantly) type checks expressions. - evaluator, err := NewEvaluator(ctx, evalCtx, sc) + // Perform normalization. + normalized, err := normalizeSelectClause(ctx, semaCtx, sc, d) if err != nil { - return n, target, err + return nil, err } - return normalized, target, evaluator.initEval(ctx, ed) + return normalized, nil } -func setTargetType( +func getExpressionTargetSpecification( desc catalog.TableDescriptor, target jobspb.ChangefeedTargetSpecification, cv *checkColumnsVisitor, ) (jobspb.ChangefeedTargetSpecification, error) { allFamilies := desc.GetFamilies() + if target.FamilyName != "" { + // Use target if family name set explicitly. + return target, nil + } + if len(allFamilies) == 1 { + // There is only 1 column family, so use that. target.Type = jobspb.ChangefeedTargetSpecification_PRIMARY_FAMILY_ONLY return target, nil } - var referencedFamilies []string - keyColSet := desc.GetPrimaryIndex().CollectKeyColumnIDs() refColSet := catalog.MakeTableColSet(cv.columns...) nonKeyColSet := refColSet.Difference(keyColSet) + numReferencedNonKeyFamilies := func() (ref int) { + _ = desc.ForeachFamily(func(family *descpb.ColumnFamilyDescriptor) error { + if catalog.MakeTableColSet(family.ColumnIDs...).Intersects(nonKeyColSet) { + ref++ + } + return nil + }) + return ref + }() if cv.seenStar { - if nonKeyColSet.Len() > 0 { - return target, pgerror.Newf(pgcode.InvalidParameterValue, "can't reference non-primary key columns as well as star on a multi column family table") + if nonKeyColSet.Len() > 0 && numReferencedNonKeyFamilies > 1 { + return target, pgerror.Newf(pgcode.InvalidParameterValue, + "can't reference non-primary key columns as well as star on a multi column family table") } - if cv.splitColFams { - target.Type = jobspb.ChangefeedTargetSpecification_EACH_FAMILY - return target, pgerror.Newf(pgcode.FeatureNotSupported, - "split_column_families is not supported with changefeed expressions yet") + if !cv.splitColFams { + return target, pgerror.Newf(pgcode.InvalidParameterValue, + "targeting a table with multiple column families requires "+ + "WITH split_column_families and will emit multiple events per row.") } - return target, pgerror.Newf(pgcode.InvalidParameterValue, "targeting a table with multiple column families requires WITH split_column_families and will emit multiple events per row.") } // If no non-primary key columns are being referenced, then we can assume that if @@ -172,7 +176,8 @@ func setTargetType( } } - // If referenced families aren't being retrived properly try using rowenc.NeededFamilyIDs + // If referenced families aren't being retrieved properly try using rowenc.NeededFamilyIDs + var referencedFamilies []string for _, family := range allFamilies { famColSet := catalog.MakeTableColSet(family.ColumnIDs...) if nonKeyColSet.Intersects(famColSet) { @@ -185,6 +190,10 @@ func setTargetType( return target, pgerror.Newf(pgcode.InvalidParameterValue, "expressions can't reference columns from more than one column family") } + if len(referencedFamilies) == 0 { + return target, pgerror.Newf( + pgcode.AssertFailure, "expression does not reference any column family") + } target.Type = jobspb.ChangefeedTargetSpecification_COLUMN_FAMILY target.FamilyName = referencedFamilies[0] return target, nil @@ -232,180 +241,172 @@ func getTargetFamilyDescriptor( } } -// NormalizedSelectClause is a select clause returned by normalizeSelectClause. -// normalizeSelectClause also modifies the select clause in place, but this -// marker type is needed so that we can ensure functions that rely on -// normalized input aren't called out of order. -type NormalizedSelectClause tree.SelectClause - -// Clause returns a pointer to the underlying SelectClause (still in normalized -// form). -func (n NormalizedSelectClause) Clause() *tree.SelectClause { - sc := tree.SelectClause(n) - return &sc -} - // normalizeSelectClause performs normalization step for select clause. // Returns normalized select clause. func normalizeSelectClause( ctx context.Context, - semaCtx tree.SemaContext, + semaCtx *tree.SemaContext, sc *tree.SelectClause, - desc catalog.TableDescriptor, -) (normalizedSelectClause NormalizedSelectClause, _ error) { + desc *cdcevent.EventDescriptor, +) (*NormalizedSelectClause, error) { // Turn FROM clause to table reference. // Note: must specify AliasClause for TableRef expression; otherwise we // won't be able to deserialize string representation (grammar requires - // "select ... from [table_id as alias]") - var alias tree.AliasClause + // "select ... from [table_id as tableAlias]") + var tableAlias tree.AliasClause switch t := sc.From.Tables[0].(type) { case *tree.AliasedTableExpr: - alias = t.As + tableAlias = t.As case tree.TablePattern: + case *tree.TableRef: + tableAlias = t.As default: // This is verified by sql.y -- but be safe. - return normalizedSelectClause, errors.AssertionFailedf("unexpected table expression type %T", + return nil, errors.AssertionFailedf("unexpected table expression type %T", sc.From.Tables[0]) } - if alias.Alias == "" { - alias.Alias = tree.Name(desc.GetName()) + if tableAlias.Alias == "" { + tableAlias.Alias = tree.Name(desc.TableName) + } + + if tableAlias.Alias == prevTupleName { + return nil, pgerror.Newf(pgcode.ReservedName, + "%s is a reserved name in CDC; Specify different alias with AS clause", prevTupleName) } + sc.From.Tables[0] = &tree.TableRef{ - TableID: int64(desc.GetID()), - As: alias, + TableID: int64(desc.TableID), + As: tableAlias, } // Setup sema ctx to handle cdc expressions. We want to make sure we only // override some properties, while keeping other properties (type resolver) // intact. - semaCtx.FunctionResolver = &CDCFunctionResolver{} - semaCtx.Properties.Require("cdc", rejectInvalidCDCExprs) + defer configSemaForCDC(semaCtx, desc)() + + // Keep track of user defined types used in the expression. + var udts map[oid.Oid]struct{} - resolveType := func(ref tree.ResolvableTypeReference) (tree.ResolvableTypeReference, bool, error) { + resolveType := func(ref tree.ResolvableTypeReference) (tree.ResolvableTypeReference, error) { typ, err := tree.ResolveType(ctx, ref, semaCtx.GetTypeResolver()) if err != nil { - return nil, false, pgerror.Wrapf(err, pgcode.IndeterminateDatatype, + return nil, pgerror.Wrapf(err, pgcode.IndeterminateDatatype, "could not resolve type %s", ref.SQLString()) } - return &tree.OIDTypeReference{OID: typ.Oid()}, typ.UserDefined(), nil - } - // Verify that any UDTs used in the statement reference only the UDTs that are - // part of the target table descriptor. - v := &tree.TypeCollectorVisitor{ - OIDs: make(map[oid.Oid]struct{}), - } - - stmt, err := tree.SimpleStmtVisit(sc, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { - // Replace type references with resolved type. - switch e := expr.(type) { - case *tree.AnnotateTypeExpr: - typ, udt, err := resolveType(e.Type) - if err != nil { - return false, expr, err - } - if !udt { - // Only care about user defined types. - return true, expr, nil + if typ.UserDefined() { + if udts == nil { + udts = make(map[oid.Oid]struct{}) } - e.Type = typ + udts[typ.Oid()] = struct{}{} + } + return typ, nil + } - case *tree.CastExpr: - typ, udt, err := resolveType(e.Type) - if err != nil { - return false, expr, err - } - if !udt { - // Only care about user defined types. + requiresPrev := false + stmt, err := tree.SimpleStmtVisit( + sc, + func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { + // Replace type references with resolved type. + switch e := expr.(type) { + case *tree.AnnotateTypeExpr: + typ, err := resolveType(e.Type) + if err != nil { + return false, expr, err + } + e.Type = typ + return true, e, nil + case *tree.CastExpr: + typ, err := resolveType(e.Type) + if err != nil { + return false, expr, err + } + e.Type = typ + return true, e, nil + case *tree.FuncExpr: + fn, err := checkFunctionSupported(ctx, e, semaCtx) + if err != nil { + return false, e, err + } + return true, fn, nil + case *tree.Subquery: + return false, e, pgerror.New( + pgcode.FeatureNotSupported, "sub-query expressions not supported by CDC") + case *tree.UnresolvedName: + switch e.NumParts { + case 1: + if e.Parts[0] == prevTupleName { + if _, err := desc.TableDescriptor().FindColumnWithName(prevTupleName); err == nil { + return false, e, + pgerror.Newf(pgcode.AmbiguousColumn, + "ambiguous cdc_prev column collides with CDC reserved keyword. "+ + "Disambiguate with %s.cdc_prev", desc.TableName) + } + + requiresPrev = true + return true, e, nil + } + + // Qualify unqualified names. Since we might be adding access to the + // previous row, column names become ambiguous if they are not + // qualified. + return true, tree.NewUnresolvedName(string(tableAlias.Alias), e.Parts[0]), nil + case 2: + if e.Parts[1] == prevTupleName { + requiresPrev = true + } + } + return true, e, nil + case tree.UnqualifiedStar: + // Qualify unqualified stars. Since we might be adding + // access to the previous row, column names become ambiguous. + return true, &tree.AllColumnsSelector{ + TableName: tree.NewUnqualifiedTableName(tableAlias.Alias).ToUnresolvedObjectName(), + }, nil + default: return true, expr, nil } - - e.Type = typ - } - - // Collect resolved type OIDs. - recurse, newExpr = v.VisitPre(expr) - return recurse, newExpr, nil - }) + }) if err != nil { - return normalizedSelectClause, err + return nil, err } + + var norm *NormalizedSelectClause switch t := stmt.(type) { case *tree.SelectClause: - normalizedSelectClause = NormalizedSelectClause(*t) + if err := scopeAndRewrite(t, desc, requiresPrev); err != nil { + return nil, err + } + norm = &NormalizedSelectClause{ + SelectClause: t, + desc: desc, + } default: // We walked tree.SelectClause -- getting anything else would be surprising. - return normalizedSelectClause, errors.AssertionFailedf("unexpected result type %T", stmt) + return nil, errors.AssertionFailedf("unexpected result type %T", stmt) } - if len(v.OIDs) == 0 { - return normalizedSelectClause, nil + if len(udts) == 0 { + return norm, nil } // Verify that the only user defined types used are the types referenced by // target table. allowedOIDs := make(map[oid.Oid]struct{}) - for _, c := range desc.UserDefinedTypeColumns() { + for _, c := range desc.TableDescriptor().UserDefinedTypeColumns() { allowedOIDs[c.GetType().Oid()] = struct{}{} } - for id := range v.OIDs { + for id := range udts { if _, isAllowed := allowedOIDs[id]; !isAllowed { - return normalizedSelectClause, pgerror.Newf(pgcode.FeatureNotSupported, + return nil, pgerror.Newf(pgcode.FeatureNotSupported, "use of user defined types not referenced by target table is not supported") } } - return normalizedSelectClause, nil -} - -type checkForPrevVisitor struct { - semaCtx tree.SemaContext - ctx context.Context - foundPrev bool -} - -// VisitPre implements the Visitor interface. -func (v *checkForPrevVisitor) VisitPre(expr tree.Expr) (bool, tree.Expr) { - if exprRequiresPreviousValue(v.ctx, v.semaCtx, expr) { - v.foundPrev = true - // no need to keep recursing - return false, expr - } - return true, expr -} - -// VisitPost implements the Visitor interface. -func (v *checkForPrevVisitor) VisitPost(e tree.Expr) tree.Expr { - return e -} - -// exprRequiresPreviousValue returns true if the top-level expression -// is a function call that cdc implements using the diff from a rangefeed. -func exprRequiresPreviousValue(ctx context.Context, semaCtx tree.SemaContext, e tree.Expr) bool { - if f, ok := e.(*tree.FuncExpr); ok { - funcDef, err := f.Func.Resolve(ctx, semaCtx.SearchPath, semaCtx.FunctionResolver) - if err != nil { - return false - } - return funcDef.Name == "cdc_prev" - } - return false -} - -// SelectClauseRequiresPrev checks whether a changefeed expression will need a row's previous values -// to be fetched in order to evaluate it. -func SelectClauseRequiresPrev( - ctx context.Context, semaCtx tree.SemaContext, sc NormalizedSelectClause, -) (bool, error) { - c := checkForPrevVisitor{semaCtx: semaCtx, ctx: ctx} - _, err := tree.SimpleStmtVisit(sc.Clause(), func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { - recurse, newExpr = c.VisitPre(expr) - return recurse, newExpr, nil - }) - return c.foundPrev, err + return norm, nil } type checkColumnsVisitor struct { @@ -432,19 +433,125 @@ func (c *checkColumnsVisitor) VisitCols(expr tree.Expr) (bool, tree.Expr) { c.err = err return false, expr } - colID := col.GetID() - c.columns = append(c.columns, colID) - case tree.UnqualifiedStar: + c.columns = append(c.columns, col.GetID()) + case tree.UnqualifiedStar, *tree.AllColumnsSelector: c.seenStar = true } return true, expr } -func (c *checkColumnsVisitor) FindColumnFamilies(sc NormalizedSelectClause) error { - _, err := tree.SimpleStmtVisit(sc.Clause(), func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { +func (c *checkColumnsVisitor) FindColumnFamilies(sc *tree.SelectClause) error { + _, err := tree.SimpleStmtVisit(sc, func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { recurse, newExpr = c.VisitCols(expr) return recurse, newExpr, nil }) return err } + +// scopeAndRewrite restricts this expression scope only to the columns +// being accessed, and rewrites select clause as needed to reflect that. +func scopeAndRewrite( + sc *tree.SelectClause, desc *cdcevent.EventDescriptor, requiresPrev bool, +) error { + tables := append(tree.TableExprs(nil), sc.From.Tables...) + if len(tables) != 1 { + return errors.AssertionFailedf("expected single table") + } + + table := tables[0] + if aliased, ok := table.(*tree.AliasedTableExpr); ok { + table = aliased.Expr + } + tableRef, ok := table.(*tree.TableRef) + if !ok { + return errors.AssertionFailedf("expected table reference, found %T", tables[0]) + } + + tables[0] = maybeScopeTable(desc, tableRef) + + if requiresPrev { + // prevTupleTableExpr is a table expression to select contents of tuple + // representing the previous row state. + // That's a bit of a mouthful, but all we're doing here is adding + // another table sub-select to the query to produce cdc_prev tuple: + // SELECT ... FROM tbl, (SELECT ((crdb_internal.cdc_prev_row()).*)) AS cdc_prev + // Note: even though this expression is the same for all queries, we should not + // make it global because the underlying call (FunctionReference) to previous row + // function will be replaced with function definition (concrete implementation). + // Thus, if we reuse the same expression across different versions of event + // descriptors, we will get unexpected errors. + prevTupleTableExpr := &tree.AliasedTableExpr{ + As: tree.AliasClause{Alias: prevTupleName}, + Expr: &tree.Subquery{ + Select: &tree.ParenSelect{ + Select: &tree.Select{ + Select: &tree.SelectClause{ + Exprs: tree.SelectExprs{ + tree.SelectExpr{ + Expr: &tree.TupleStar{ + Expr: &tree.FuncExpr{ + Func: tree.ResolvableFunctionReference{FunctionReference: &prevRowFnName}, + }, + }, + }, + }, + }, + }, + }, + }, + } + + tables = append(tables, prevTupleTableExpr) + } + + sc.From = tree.From{Tables: tables} + return nil +} + +// maybeScopeTable returns possibly "scoped" table expression. +// If event descriptor targets all columns, then table expression returned +// unmodified. However, if the event descriptor targets a subset of columns, +// then returns table expression restricted to targeted columns. +func maybeScopeTable(ed *cdcevent.EventDescriptor, tableRef *tree.TableRef) tree.TableExpr { + // If the event descriptor targets all columns in the table, we can use + // table as is. + if ed.FamilyID == 0 && !ed.HasVirtual && !ed.HasOtherFamilies { + return tableRef + } + + // If the event descriptor targets specific column family, we need to scope + // expression to select only the columns in the event descriptor. + // The code below is a bit of a mouth full. Assuming that we were selecting from + // table named 'tbl', all we're doing here is turning + // from clause (FROM tbl) into something that looks like: + // FROM (SELECT col1, col2, ... FROM [tableID AS t]) AS tbl + // Where col1, col2, ... are columns in the target column family, tableID is the table + // ID of the target table. + scopedTable := &tree.SelectClause{ + From: tree.From{ + Tables: tree.TableExprs{&tree.TableRef{ + TableID: tableRef.TableID, + As: tree.AliasClause{Alias: "t"}, + }}, + }, + Exprs: func() (exprs tree.SelectExprs) { + exprs = make(tree.SelectExprs, len(ed.ResultColumns())) + for i, c := range ed.ResultColumns() { + exprs[i] = tree.SelectExpr{Expr: &tree.ColumnItem{ColumnName: tree.Name(c.Name)}} + } + return exprs + }(), + } + + return &tree.AliasedTableExpr{ + Expr: &tree.Subquery{ + Select: &tree.ParenSelect{ + Select: &tree.Select{ + Select: scopedTable, + }, + }, + }, + As: tableRef.As, + } +} diff --git a/pkg/ccl/changefeedccl/cdceval/validation_test.go b/pkg/ccl/changefeedccl/cdceval/validation_test.go index 9bfddfb7d0a8..706bbb17ccbd 100644 --- a/pkg/ccl/changefeedccl/cdceval/validation_test.go +++ b/pkg/ccl/changefeedccl/cdceval/validation_test.go @@ -20,8 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -33,7 +31,7 @@ func TestNormalizeAndValidate(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(db) @@ -57,15 +55,6 @@ func TestNormalizeAndValidate(t *testing.T) { ctx := context.Background() execCfg := s.ExecutorConfig().(sql.ExecutorConfig) - p, cleanup := sql.NewInternalPlanner("test", - kvDB.NewTxn(ctx, "test-planner"), - username.RootUserName(), &sql.MemoryMetrics{}, &execCfg, - sessiondatapb.SessionData{ - Database: "defaultdb", - SearchPath: sessiondata.DefaultSearchPath.GetPathArray(), - }) - defer cleanup() - execCtx := p.(sql.PlanHookState) for _, tc := range []struct { name string @@ -76,18 +65,10 @@ func TestNormalizeAndValidate(t *testing.T) { splitColFams bool }{ { - name: "reject multiple tables", - desc: fooDesc, - stmt: "SELECT * FROM foo, other.foo", - expectErr: "invalid CDC expression: only 1 table supported", - splitColFams: false, - }, - { - name: "reject contradiction", - desc: fooDesc, - stmt: "SELECT * FROM foo WHERE a IS NULL", - expectErr: `filter "a IS NULL" is a contradiction`, - splitColFams: false, + name: "reject multiple tables", + desc: fooDesc, + stmt: "SELECT * FROM foo, other.foo", + expectErr: "expected 1 table", }, { name: "enum must be referenced", @@ -107,28 +88,28 @@ func TestNormalizeAndValidate(t *testing.T) { name: "reject multiple column families with star", desc: bazDesc, stmt: "SELECT * FROM baz", - expectErr: `targeting a table with multiple column families requires WITH split_column_families and will emit multiple events per row.`, + expectErr: `requires WITH split_column_families`, splitColFams: false, }, { name: "replaces table name with ref", desc: fooDesc, stmt: "SELECT * FROM foo", - expectStmt: fmt.Sprintf("SELECT * FROM [%d AS foo]", fooDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT foo.* FROM [%d AS foo]", fooDesc.GetID()), splitColFams: false, }, { name: "replaces table name with other.ref", desc: otherFooDesc, stmt: "SELECT * FROM other.foo", - expectStmt: fmt.Sprintf("SELECT * FROM [%d AS foo]", otherFooDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT foo.* FROM [%d AS foo]", otherFooDesc.GetID()), splitColFams: false, }, { name: "replaces table name with ref aliased", desc: fooDesc, stmt: "SELECT * FROM foo AS bar", - expectStmt: fmt.Sprintf("SELECT * FROM [%d AS bar]", fooDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT bar.* FROM [%d AS bar]", fooDesc.GetID()), splitColFams: false, }, { @@ -136,8 +117,8 @@ func TestNormalizeAndValidate(t *testing.T) { desc: fooDesc, stmt: "SELECT *, 'inactive':::status FROM foo AS bar WHERE status = 'open':::status", expectStmt: fmt.Sprintf( - "SELECT *, 'inactive':::defaultdb.public.status "+ - "FROM [%d AS bar] WHERE status = 'open':::defaultdb.public.status", + "SELECT bar.*, 'inactive':::defaultdb.public.status "+ + "FROM [%d AS bar] WHERE bar.status = 'open':::defaultdb.public.status", fooDesc.GetID()), splitColFams: false, }, @@ -154,28 +135,28 @@ func TestNormalizeAndValidate(t *testing.T) { name: "can target one column family", desc: bazDesc, stmt: "SELECT a, b FROM baz", - expectStmt: fmt.Sprintf("SELECT a, b FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.a, baz.b FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, { name: "SELECT a, b FROM bop", desc: bopDesc, stmt: "SELECT a, b FROM bop", - expectStmt: fmt.Sprintf("SELECT a, b FROM [%d AS bop]", bopDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT bop.a, bop.b FROM (SELECT a, b FROM [%d AS t]) AS bop", bopDesc.GetID()), splitColFams: false, }, { name: "SELECT a, c FROM baz", desc: bazDesc, stmt: "SELECT a, c FROM baz", - expectStmt: fmt.Sprintf("SELECT a, c FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.a, baz.c FROM (SELECT a, c FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, { name: "SELECT b, b+1 AS c FROM baz", desc: bazDesc, stmt: "SELECT b, b+1 AS c FROM baz", - expectStmt: fmt.Sprintf("SELECT b, b + 1 AS c FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.b, baz.b + 1 AS c FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, { @@ -189,16 +170,30 @@ func TestNormalizeAndValidate(t *testing.T) { name: "SELECT B FROM baz", desc: bazDesc, stmt: "SELECT B FROM baz", - expectStmt: fmt.Sprintf("SELECT b FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.b FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, { name: "SELECT baz.b FROM baz", desc: bazDesc, stmt: "SELECT baz.b FROM baz", - expectStmt: fmt.Sprintf("SELECT baz.b FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.b FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, + { + name: "SELECT baz.b, row_to_json(cdc_prev.*) FROM baz", + desc: bazDesc, + stmt: "SELECT baz.b, row_to_json(cdc_prev.*) FROM baz", + expectStmt: fmt.Sprintf("SELECT baz.b, row_to_json(cdc_prev.*) FROM "+ + "(SELECT a, b FROM [%d AS t]) AS baz, "+ + "(SELECT (crdb_internal.cdc_prev_row()).*) AS cdc_prev", + bazDesc.GetID()), + // Currently, accessing cdc_prev.* is treated in such a way as to require + // split column families option. This might not be needed since the above + // expression targets main column family only. Perhaps this restriction + // can be relaxed. + splitColFams: true, + }, { name: "SELECT b FROM baz WHERE c IS NULL", desc: bazDesc, @@ -217,33 +212,59 @@ func TestNormalizeAndValidate(t *testing.T) { name: "SELECT b::string = 'c' FROM baz", desc: bazDesc, stmt: "SELECT b::string = 'c' FROM baz", - expectStmt: fmt.Sprintf("SELECT b::STRING = 'c' FROM [%d AS baz]", bazDesc.GetID()), + expectStmt: fmt.Sprintf("SELECT baz.b::STRING = 'c' FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), splitColFams: false, }, { name: "SELECT *, c FROM baz", desc: bazDesc, stmt: "SELECT *, c FROM baz", - expectErr: `can't reference non-primary key columns as well as star on a multi column family table`, + expectErr: `requires WITH split_column_families`, splitColFams: false, }, { - name: "SELECT * FROM baz WITH split_column_families", + name: "no explicit column references", desc: bazDesc, - stmt: "SELECT * FROM baz", - expectErr: `split_column_families is not supported with changefeed expressions yet`, - splitColFams: true, + stmt: "SELECT pi() FROM baz", + expectStmt: fmt.Sprintf("SELECT pi() FROM (SELECT a, b FROM [%d AS t]) AS baz", bazDesc.GetID()), + splitColFams: false, + }, + { + name: "cdc_prev is not a function", + desc: fooDesc, + stmt: "SELECT *, cdc_prev() FROM foo AS bar", + expectErr: `function "cdc_prev" unsupported by CDC`, }, } { t.Run(tc.name, func(t *testing.T) { sc, err := ParseChangefeedExpression(tc.stmt) - require.NoError(t, err) + if err != nil { + require.NotEmpty(t, tc.expectErr, "expected no error, got %s", err) + require.Regexp(t, tc.expectErr, err) + return + } target := jobspb.ChangefeedTargetSpecification{ TableID: tc.desc.GetID(), StatementTimeName: tc.desc.GetName(), } - _, _, err = NormalizeAndValidateSelectForTarget(ctx, execCtx, tc.desc, target, sc, false, false, tc.splitColFams) + d, err := newEventDescriptorForTarget(tc.desc, target, execCfg.Clock.Now(), false, false) + require.NoError(t, err) + + schemaTS := s.Clock().Now() + err = withPlanner(ctx, &execCfg, username.RootUserName(), schemaTS, defaultDBSessionData, + func(ctx context.Context, execCtx sql.JobExecContext) error { + defer configSemaForCDC(execCtx.SemaCtx(), d)() + norm, err := normalizeAndValidateSelectForTarget( + ctx, execCtx.ExecCfg(), tc.desc, schemaTS, target, sc, + false, tc.splitColFams, execCtx.SemaCtx()) + if err == nil { + sc = norm.SelectClause + } + return err + }, + ) + if tc.expectErr != "" { require.Regexp(t, tc.expectErr, err) return @@ -251,8 +272,6 @@ func TestNormalizeAndValidate(t *testing.T) { require.NoError(t, err) serialized := AsStringUnredacted(sc) - log.Infof(context.Background(), "DEBUG: %s", tree.StmtDebugString(sc)) - log.Infof(context.Background(), "Serialized: %s", serialized) require.Equal(t, tc.expectStmt, serialized) // Make sure we can deserialize back. @@ -266,7 +285,7 @@ func TestSelectClauseRequiresPrev(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(context.Background()) sqlDB := sqlutils.MakeSQLRunner(db) @@ -281,63 +300,67 @@ func TestSelectClauseRequiresPrev(t *testing.T) { ctx := context.Background() execCfg := s.ExecutorConfig().(sql.ExecutorConfig) - p, cleanup := sql.NewInternalPlanner("test", - kvDB.NewTxn(ctx, "test-planner"), - username.RootUserName(), &sql.MemoryMetrics{}, &execCfg, - sessiondatapb.SessionData{ - Database: "defaultdb", - SearchPath: sessiondata.DefaultSearchPath.GetPathArray(), - }) - defer cleanup() - execCtx := p.(sql.PlanHookState) for _, tc := range []struct { - name string - desc catalog.TableDescriptor - stmt string - expect bool + name string + desc catalog.TableDescriptor + stmt string + requiresPrev bool + expectErr string }{ { - name: "top level call to cdc_prev", - desc: descs[`foo`], - stmt: "SELECT cdc_prev() from foo", - expect: true, + name: "top level call to cdc_prev", + desc: descs[`foo`], + stmt: "SELECT row_to_json(cdc_prev.*) from foo", + requiresPrev: true, + }, + { + name: "nested call to cdc_prev", + desc: descs[`foo`], + stmt: "SELECT jsonb_build_object('op',IF(cdc_is_delete(),'u',IF(row_to_json(cdc_prev.*)::string='null','c','u'))) from foo", + requiresPrev: true, }, { - name: "nested call to cdc_prev", - desc: descs[`foo`], - stmt: "SELECT jsonb_build_object('op',IF(cdc_is_delete(),'u',IF(cdc_prev()::string='null','c','u'))) from foo", - expect: true, + name: "cdc_prev in the stmt", + desc: descs[`foo`], + stmt: "SELECT * from foo WHERE cdc_prev.s != s", + requiresPrev: true, }, { - name: "cdc_prev in the predicate", - desc: descs[`foo`], - stmt: "SELECT * from foo WHERE (cdc_prev()->'s')::string != s", - expect: true, + name: "cdc_prev case insensitive", + desc: descs[`foo`], + stmt: "SELECT row_to_json(CdC_pReV.*) from foo", + requiresPrev: true, }, { - name: "case insensitive", - desc: descs[`foo`], - stmt: "SELECT CDC_PREV() from foo", - expect: true, + name: "contains misleading substring", + desc: descs[`foo`], + stmt: "SELECT 'cdc_prev()', s FROM foo", + requiresPrev: false, }, { - name: "contains misleading substring", - desc: descs[`foo`], - stmt: "SELECT 'cdc_prev()', s FROM foo", - expect: false, + name: "misleading table name", + desc: descs[`cdc_prev`], + stmt: "SELECT * FROM cdc_prev", + expectErr: "cdc_prev is a reserved name in CDC", }, { - name: "misleading table name", - desc: descs[`cdc_prev`], - stmt: "SELECT * FROM cdc_prev", - expect: false, + name: "misleading table name with alias", + desc: descs[`cdc_prev`], + stmt: "SELECT * FROM cdc_prev AS real_prev", + requiresPrev: false, }, { - name: "misleading column name", - desc: descs[`misleading_column_name`], - stmt: "SELECT cdc_prev FROM misleading_column_name", - expect: false, + name: "misleading column name", + desc: descs[`misleading_column_name`], + stmt: "SELECT cdc_prev FROM misleading_column_name", + expectErr: "ambiguous cdc_prev column collides with CDC reserved keyword. Disambiguate with misleading_column_name.cdc_prev", + }, + { + name: "misleading column name disambiguated", + desc: descs[`misleading_column_name`], + stmt: "SELECT misleading_column_name.cdc_prev FROM misleading_column_name", + requiresPrev: false, }, } { t.Run(tc.name, func(t *testing.T) { @@ -347,11 +370,28 @@ func TestSelectClauseRequiresPrev(t *testing.T) { TableID: tc.desc.GetID(), StatementTimeName: tc.desc.GetName(), } - normalized, _, err := NormalizeAndValidateSelectForTarget(ctx, execCtx, tc.desc, target, sc, false, false, false) - require.NoError(t, err) - actual, err := SelectClauseRequiresPrev(context.Background(), *execCtx.SemaCtx(), normalized) + + schemaTS := s.Clock().Now() + d, err := newEventDescriptorForTarget(tc.desc, target, schemaTS, false, false) require.NoError(t, err) - require.Equal(t, tc.expect, actual) + + var normalized *NormalizedSelectClause + err = withPlanner(ctx, &execCfg, username.RootUserName(), schemaTS, defaultDBSessionData, + func(ctx context.Context, execCtx sql.JobExecContext) error { + defer configSemaForCDC(execCtx.SemaCtx(), d)() + normalized, err = normalizeAndValidateSelectForTarget( + ctx, execCtx.ExecCfg(), tc.desc, schemaTS, target, sc, + false, false, execCtx.SemaCtx()) + return err + }, + ) + + if tc.expectErr != "" { + require.Regexp(t, tc.expectErr, err) + } else { + require.NoError(t, err) + require.Equal(t, tc.requiresPrev, normalized.RequiresPrev()) + } }) } } diff --git a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel index 028db5e7a49b..d49ae73713b9 100644 --- a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel @@ -18,17 +18,16 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/roachpb", + "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/lease", - "//pkg/sql/execinfra", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/row", "//pkg/sql/rowenc", - "//pkg/sql/sem/cast", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/types", @@ -62,6 +61,7 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/settings/cluster", + "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/ccl/changefeedccl/cdcevent/event.go b/pkg/ccl/changefeedccl/cdcevent/event.go index 7a75de9a500a..5a1959408b95 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event.go +++ b/pkg/ccl/changefeedccl/cdcevent/event.go @@ -11,15 +11,15 @@ package cdcevent import ( "context" "fmt" + "strings" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -43,6 +43,7 @@ type Metadata struct { FamilyID descpb.FamilyID // Column family ID. FamilyName string // Column family name. HasOtherFamilies bool // True if the table multiple families. + HasVirtual bool // True if table has virtual columns. SchemaTS hlc.Timestamp // Schema timestamp for table descriptor. } @@ -55,6 +56,7 @@ type Decoder interface { // Row holds a row corresponding to an event. type Row struct { *EventDescriptor + MvccTimestamp hlc.Timestamp // Mvcc timestamp of this row. // datums is the new value of a changed table row. datums rowenc.EncDatumRow @@ -83,6 +85,11 @@ type Iterator interface { Col(fn ColumnFn) error } +// EncDatums returns EncDatumRow. +func (r Row) EncDatums() rowenc.EncDatumRow { + return r.datums +} + // ForEachKeyColumn returns Iterator for each key column func (r Row) ForEachKeyColumn() Iterator { return iter{r: r, cols: r.keyCols} @@ -119,6 +126,23 @@ func (r Row) DatumAt(at int) (tree.Datum, error) { return encDatum.Datum, nil } +// CopyInto decodes and copies encdatums to specified tuple. +func (r Row) CopyInto(tuple *tree.DTuple) error { + tupleTypes := tuple.ResolvedType().InternalType.TupleContents + if len(tupleTypes) != len(r.datums) { + return errors.AssertionFailedf("cannot copy row with %d datums into tuple with %d", + len(r.datums), len(tupleTypes)) + } + + for i, typ := range tupleTypes { + if err := r.datums[i].EnsureDecoded(typ, r.alloc); err != nil { + return errors.Wrapf(err, "error decoding column [%d] as type %s", i, typ) + } + tuple.D[i] = r.datums[i].Datum + } + return nil +} + // IsDeleted returns true if event corresponds to a deletion event. func (r Row) IsDeleted() bool { return r.deleted @@ -134,6 +158,31 @@ func (r Row) HasValues() bool { return r.datums != nil } +// DebugString returns debug string describing event source. +func (m Metadata) DebugString() string { + return fmt.Sprintf("{table: %d family: %d}", m.TableID, m.FamilyID) +} + +// DebugString returns row string. +func (r Row) DebugString() string { + var sb strings.Builder + sb.WriteString(fmt.Sprintf("Row%s{", r.Metadata.DebugString())) + first := true + err := r.ForAllColumns().Datum(func(d tree.Datum, col ResultColumn) error { + if !first { + sb.WriteString(", ") + } + sb.WriteString(fmt.Sprintf("%s: %s", col.Name, d.String())) + first = false + return nil + }) + if err != nil { + sb.WriteString(fmt.Sprintf("err: %s", err)) + } + sb.WriteByte('}') + return sb.String() +} + // forEachColumn is a helper which invokes fn for reach column in the ordColumn list. func (r Row) forEachDatum(fn DatumFn, colIndexes []int) error { numVirtualCols := 0 @@ -275,6 +324,9 @@ func NewEventDescriptor( ord := 0 for _, col := range desc.PublicColumns() { isInFamily := inFamily.Contains(col.GetID()) + if col.IsVirtual() { + sd.HasVirtual = true + } virtual := col.IsVirtual() && includeVirtualColumns pKeyOrd, isPKey := primaryKeyOrdinal.Get(col.GetID()) if keyOnly { @@ -345,11 +397,6 @@ func (d *EventDescriptor) EqualsWithUDTCheck( return false, false } -// HasUserDefinedTypes returns true if this descriptor contains user defined columns. -func (d *EventDescriptor) HasUserDefinedTypes() bool { - return len(d.udtCols) > 0 -} - // TableDescriptor returns underlying table descriptor. This method is exposed // to make it easier to integrate with the rest of descriptor APIs; prefer to use // higher level methods/structs (e.g. Metadata) instead. @@ -411,7 +458,7 @@ func getEventDescriptorCached( // NewEventDecoder returns key value decoder. func NewEventDecoder( ctx context.Context, - cfg *execinfra.ServerConfig, + cfg *sql.ExecutorConfig, targets changefeedbase.Targets, includeVirtual bool, keyOnly bool, @@ -419,7 +466,7 @@ func NewEventDecoder( rfCache, err := newRowFetcherCache( ctx, cfg.Codec, - cfg.LeaseManager.(*lease.Manager), + cfg.LeaseManager, cfg.CollectionFactory, cfg.DB, targets, @@ -469,6 +516,7 @@ func (d *eventDecoder) DecodeKV( return Row{ EventDescriptor: ed, + MvccTimestamp: kv.Value.Timestamp, datums: datums, deleted: isDeleted, alloc: &d.alloc, @@ -524,11 +572,6 @@ func (d *eventDecoder) nextRow(ctx context.Context) (rowenc.EncDatumRow, bool, e return datums, isDeleted, nil } -// String returns debug string describing event source. -func (m Metadata) String() string { - return fmt.Sprintf("{table: %d family: %d}", m.TableID, m.FamilyID) -} - type iter struct { r Row cols []int @@ -611,10 +654,10 @@ func MakeRowFromTuple(ctx context.Context, evalCtx *eval.Context, t *tree.DTuple name = names[i] } r.AddValueColumn(name, d.ResolvedType()) - if err := r.SetValueDatumAt(ctx, evalCtx, i, d); err != nil { + if err := r.SetValueDatumAt(i, d); err != nil { if build.IsRelease() { log.Warningf(ctx, "failed to set row value from tuple due to error %v", err) - _ = r.SetValueDatumAt(ctx, evalCtx, i, tree.DNull) + _ = r.SetValueDatumAt(i, tree.DNull) } else { panic(err) } diff --git a/pkg/ccl/changefeedccl/cdcevent/event_test.go b/pkg/ccl/changefeedccl/cdcevent/event_test.go index 30cc8ee1ad00..e6556a31628b 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event_test.go +++ b/pkg/ccl/changefeedccl/cdcevent/event_test.go @@ -19,10 +19,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -358,9 +358,9 @@ CREATE TABLE foo ( TableID: tableDesc.GetID(), FamilyName: tc.familyName, }) - serverCfg := s.DistSQLServer().(*distsql.ServerImpl).ServerConfig + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) ctx := context.Background() - decoder, err := NewEventDecoder(ctx, &serverCfg, targets, tc.includeVirtual, tc.keyOnly) + decoder, err := NewEventDecoder(ctx, &execCfg, targets, tc.includeVirtual, tc.keyOnly) require.NoError(t, err) expectedEvents := len(tc.expectMainFamily) + len(tc.expectOnlyCFamily) for i := 0; i < expectedEvents; i++ { @@ -574,9 +574,9 @@ func TestEventColumnOrderingWithSchemaChanges(t *testing.T) { TableID: tableDesc.GetID(), FamilyName: tc.familyName, }) - serverCfg := s.DistSQLServer().(*distsql.ServerImpl).ServerConfig + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) ctx := context.Background() - decoder, err := NewEventDecoder(ctx, &serverCfg, targets, tc.includeVirtual, false) + decoder, err := NewEventDecoder(ctx, &execCfg, targets, tc.includeVirtual, false) require.NoError(t, err) expectedEvents := len(tc.expectMainFamily) + len(tc.expectECFamily) diff --git a/pkg/ccl/changefeedccl/cdcevent/projection.go b/pkg/ccl/changefeedccl/cdcevent/projection.go index 6558be2b76ef..2bd17bdac077 100644 --- a/pkg/ccl/changefeedccl/cdcevent/projection.go +++ b/pkg/ccl/changefeedccl/cdcevent/projection.go @@ -9,25 +9,22 @@ package cdcevent import ( - "context" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/sql/sem/cast" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) // Projection is a helper to facilitate construction of "projection" rows. -// Projection is constructed given the underlying event descriptor. Only the key -// columns from the descriptor are initialized upon construction. All other -// value columns returned by projection need to be configured separated via AddValueColumn, -// and the value for that column must later be set via SetValueDatumAt. -// All columns added to this projection are in the ordinal order. +// Projection is constructed given the underlying event descriptor. Only the +// key columns from the descriptor are initialized upon construction. All other +// value columns returned by projection need to be configured separately via +// AddValueColumn, and the value for that column must later be set via +// SetValueDatumAt. All columns added to this projection are in the ordinal +// order. type Projection Row // MakeProjection returns Projection builder given underlying descriptor. @@ -56,6 +53,7 @@ func (p *Projection) addColumn(name string, typ *types.T, sqlString string, colI }) p.datums = append(p.datums, rowenc.EncDatum{}) + p.allCols = append(p.allCols, ord) *colIdxSlice = append(*colIdxSlice, ord) if typ.UserDefined() { p.udtCols = append(p.udtCols, ord) @@ -68,36 +66,25 @@ func (p *Projection) AddValueColumn(name string, typ *types.T) { } // SetValueDatumAt sets value datum at specified position. -func (p *Projection) SetValueDatumAt( - ctx context.Context, evalCtx *eval.Context, pos int, d tree.Datum, -) error { +func (p *Projection) SetValueDatumAt(pos int, d tree.Datum) error { pos += len(p.keyCols) if pos >= len(p.datums) { return errors.AssertionFailedf("%d out of bounds", pos) } col := p.cols[pos] - if !col.Typ.Equal(d.ResolvedType()) { - if !cast.ValidCast(d.ResolvedType(), col.Typ, cast.ContextImplicit) { - return pgerror.Newf(pgcode.DatatypeMismatch, - "expected type %s for column %s@%d, found %s", col.Typ, col.Name, pos, d.ResolvedType()) - } - cd, err := eval.PerformCast(ctx, evalCtx, d, col.Typ) - if err != nil { - return errors.Wrapf(err, "expected type %s for column %s@%d, found %s", - col.Typ, col.Name, pos, d.ResolvedType()) - } - d = cd + if d == tree.DNull || col.Typ.Equal(d.ResolvedType()) { + p.datums[pos].Datum = d + return nil } - p.datums[pos].Datum = d - return nil + return pgerror.Newf(pgcode.DatatypeMismatch, + "expected type %s for column %s@%d, found %s", col.Typ, col.Name, pos, d.ResolvedType()) } // Project returns row projection. func (p *Projection) Project(r Row) (Row, error) { p.deleted = r.IsDeleted() - // Copy key datums. idx := 0 if err := r.ForEachKeyColumn().Datum(func(d tree.Datum, col ResultColumn) error { diff --git a/pkg/ccl/changefeedccl/cdcevent/projection_test.go b/pkg/ccl/changefeedccl/cdcevent/projection_test.go index e0c88c59a3de..e15e45f01f34 100644 --- a/pkg/ccl/changefeedccl/cdcevent/projection_test.go +++ b/pkg/ccl/changefeedccl/cdcevent/projection_test.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -43,7 +42,6 @@ CREATE TABLE foo ( PRIMARY KEY (b, a) )`) - evalCtx := eval.MakeTestingEvalContext(s.ClusterSettings()) desc := cdctest.GetHydratedTableDescriptor(t, s.ExecutorConfig(), "foo") encDatums := makeEncDatumRow(tree.NewDInt(1), tree.NewDString("one"), tree.DNull) @@ -62,7 +60,7 @@ CREATE TABLE foo ( idx := 0 require.NoError(t, input.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error { p.AddValueColumn(col.Name, col.Typ) - err := p.SetValueDatumAt(ctx, &evalCtx, idx, d) + err := p.SetValueDatumAt(idx, d) idx++ return err })) @@ -77,9 +75,9 @@ CREATE TABLE foo ( input := TestingMakeEventRow(desc, 0, encDatums, false) p := MakeProjection(input.EventDescriptor) p.AddValueColumn("wrong_type", types.Int) - require.Regexp(t, "expected type int", p.SetValueDatumAt(ctx, &evalCtx, 0, tree.NewDString("fail"))) + require.Regexp(t, "expected type int", p.SetValueDatumAt(0, tree.NewDString("fail"))) // But we allow NULL. - require.NoError(t, p.SetValueDatumAt(ctx, &evalCtx, 0, tree.DNull)) + require.NoError(t, p.SetValueDatumAt(0, tree.DNull)) }) t.Run("project_extra_column", func(t *testing.T) { @@ -88,12 +86,12 @@ CREATE TABLE foo ( idx := 0 require.NoError(t, input.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error { p.AddValueColumn(col.Name, col.Typ) - err := p.SetValueDatumAt(ctx, &evalCtx, idx, d) + err := p.SetValueDatumAt(idx, d) idx++ return err })) p.AddValueColumn("test", types.Int) - require.NoError(t, p.SetValueDatumAt(ctx, &evalCtx, idx, tree.NewDInt(5))) + require.NoError(t, p.SetValueDatumAt(idx, tree.NewDInt(5))) pr, err := p.Project(input) require.NoError(t, err) diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index f2de70a5e008..ec562bd8f5da 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -175,34 +175,36 @@ var changefeedResultTypes = []*types.T{ // fetchSpansForTable returns the set of spans for the specified table. // Usually, this is just the primary index span. // However, if details.Select is not empty, the set of spans returned may be -// restricted to satisfy predicate in the select clause. In that case, -// possibly updated select clause returned representing the remaining expression -// that still needs to be applied to the events. +// restricted to satisfy predicate in the select clause. func fetchSpansForTables( ctx context.Context, execCtx sql.JobExecContext, tableDescs []catalog.TableDescriptor, details jobspb.ChangefeedDetails, -) (_ []roachpb.Span, updatedExpression string, _ error) { + initialHighwater hlc.Timestamp, +) (roachpb.Spans, error) { var trackedSpans []roachpb.Span if details.Select == "" { for _, d := range tableDescs { trackedSpans = append(trackedSpans, d.PrimaryIndexSpan(execCtx.ExecCfg().Codec)) } - return trackedSpans, "", nil + return trackedSpans, nil } if len(tableDescs) != 1 { - return nil, "", pgerror.Newf(pgcode.InvalidParameterValue, + return nil, pgerror.Newf(pgcode.InvalidParameterValue, "filter can only be used with single target (found %d)", len(tableDescs)) } target := details.TargetSpecifications[0] - includeVirtual := details.Opts[changefeedbase.OptVirtualColumns] == string(changefeedbase.OptVirtualColumnsNull) - keyOnly := details.Opts[changefeedbase.OptEnvelope] == string(changefeedbase.OptEnvelopeKeyOnly) - - return cdceval.ConstrainPrimaryIndexSpanByFilter( - ctx, execCtx, details.Select, tableDescs[0], target, includeVirtual, keyOnly) + sc, err := cdceval.ParseChangefeedExpression(details.Select) + if err != nil { + return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, + "could not parse changefeed expression") + } + return cdceval.SpansForExpression( + ctx, execCtx.ExecCfg(), execCtx.User(), execCtx.SessionData().SessionData, + tableDescs[0], initialHighwater, target, sc) } var replanChangefeedThreshold = settings.RegisterFloatSetting( @@ -236,7 +238,11 @@ func startDistChangefeed( if err != nil { return err } - trackedSpans, selectClause, err := fetchSpansForTables(ctx, execCtx, tableDescs, details) + + if schemaTS.IsEmpty() { + schemaTS = details.StatementTime + } + trackedSpans, err := fetchSpansForTables(ctx, execCtx, tableDescs, details, schemaTS) if err != nil { return err } @@ -248,7 +254,7 @@ func startDistChangefeed( dsp := execCtx.DistSQLPlanner() evalCtx := execCtx.ExtendedEvalContext() - p, planCtx, err := makePlan(execCtx, jobID, details, initialHighWater, checkpoint, trackedSpans, selectClause)(ctx, dsp) + p, planCtx, err := makePlan(execCtx, jobID, details, initialHighWater, checkpoint, trackedSpans)(ctx, dsp) if err != nil { return err } @@ -264,7 +270,7 @@ func startDistChangefeed( replanner, stopReplanner := sql.PhysicalPlanChangeChecker(ctx, p, - makePlan(execCtx, jobID, details, initialHighWater, checkpoint, trackedSpans, selectClause), + makePlan(execCtx, jobID, details, initialHighWater, checkpoint, trackedSpans), execCtx, replanOracle, func() time.Duration { return replanChangefeedFrequency.Get(execCtx.ExecCfg().SV()) }, @@ -331,7 +337,6 @@ func makePlan( initialHighWater hlc.Timestamp, checkpoint jobspb.ChangefeedProgress_Checkpoint, trackedSpans []roachpb.Span, - selectClause string, ) func(context.Context, *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { return func(ctx context.Context, dsp *sql.DistSQLPlanner) (*sql.PhysicalPlan, *sql.PlanningCtx, error) { var blankTxn *kv.Txn @@ -401,7 +406,7 @@ func makePlan( Feed: details, UserProto: execCtx.User().EncodeProto(), JobID: jobID, - Select: execinfrapb.Expression{Expr: selectClause}, + Select: execinfrapb.Expression{Expr: details.Select}, } } diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index b8d902850025..a48438cd87d0 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -282,8 +282,8 @@ func (ca *changeAggregator) Start(ctx context.Context) { } ca.eventConsumer, ca.sink, err = newEventConsumer( - ctx, ca.flowCtx, feed, ca.frontier.SpanFrontier(), kvFeedHighWater, - ca.sink, feed, ca.spec.Select, ca.knobs, ca.metrics, ca.isSinkless()) + ctx, ca.flowCtx.Cfg, ca.spec, feed, ca.frontier.SpanFrontier(), kvFeedHighWater, + ca.sink, ca.metrics, ca.knobs) if err != nil { // Early abort in the case that there is an error setting up the consumption. @@ -620,10 +620,6 @@ func (ca *changeAggregator) ConsumerClosed() { ca.close() } -func (ca *changeAggregator) isSinkless() bool { - return ca.spec.JobID == 0 -} - const ( emitAllResolved = 0 emitNoResolved = -1 diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index de37bfff0edb..833b568fd803 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -403,7 +403,8 @@ func createChangefeedJobRecord( return nil, err } - targets, tables, err := getTargetsAndTables(ctx, p, targetDescs, changefeedStmt.Targets, changefeedStmt.originalSpecs, opts.ShouldUseFullStatementTimeName()) + targets, tables, err := getTargetsAndTables(ctx, p, targetDescs, changefeedStmt.Targets, + changefeedStmt.originalSpecs, opts.ShouldUseFullStatementTimeName()) if err != nil { return nil, err } @@ -429,33 +430,19 @@ func createChangefeedJobRecord( if changefeedStmt.Select != nil { // Serialize changefeed expression. - normalized, _, err := validateAndNormalizeChangefeedExpression( - ctx, p, changefeedStmt.Select, targetDescs, targets, opts.IncludeVirtual(), opts.KeyOnly(), opts.IsSet(changefeedbase.OptSplitColumnFamilies), + normalized, err := validateAndNormalizeChangefeedExpression( + ctx, p, opts, changefeedStmt.Select, targetDescs, targets, statementTime, ) if err != nil { return nil, err } - needDiff, err := cdceval.SelectClauseRequiresPrev(ctx, *p.SemaCtx(), normalized) - if err != nil { - return nil, err - } - if needDiff { + if normalized.RequiresPrev() { opts.ForceDiff() } // TODO: Set the default envelope to row here when using a sink and format // that support it. - details.Select = cdceval.AsStringUnredacted(normalized.Clause()) - opts.SetDefaultEnvelope(changefeedbase.OptEnvelopeBare) - - // TODO(#85143): do not enforce schema_change_policy='stop' for changefeed expressions. - schemachangeOptions, err := opts.GetSchemaChangeHandlingOptions() - if err != nil { - return nil, err - } - if schemachangeOptions.Policy != changefeedbase.OptSchemaChangePolicyStop { - return nil, errors.Errorf(`using "AS SELECT" requires option schema_change_policy='stop'`) - } + details.Select = cdceval.AsStringUnredacted(normalized) } // TODO(dan): In an attempt to present the most helpful error message to the @@ -895,25 +882,31 @@ func validateDetailsAndOptions( // validateAndNormalizeChangefeedExpression validates and normalizes changefeed expressions. // This method modifies passed in select clause to reflect normalization step. +// TODO(yevgeniy): Add virtual column support. func validateAndNormalizeChangefeedExpression( ctx context.Context, - execCtx sql.PlanHookState, + execCtx sql.JobExecContext, + opts changefeedbase.StatementOptions, sc *tree.SelectClause, descriptors map[tree.TablePattern]catalog.Descriptor, targets []jobspb.ChangefeedTargetSpecification, - includeVirtual bool, - keyOnly bool, - splitColFams bool, -) (n cdceval.NormalizedSelectClause, target jobspb.ChangefeedTargetSpecification, _ error) { + statementTime hlc.Timestamp, +) (*cdceval.NormalizedSelectClause, error) { if len(descriptors) != 1 || len(targets) != 1 { - return n, target, pgerror.Newf(pgcode.InvalidParameterValue, "CDC expressions require single table") + return nil, pgerror.Newf(pgcode.InvalidParameterValue, "CDC expressions require single table") } var tableDescr catalog.TableDescriptor for _, d := range descriptors { tableDescr = d.(catalog.TableDescriptor) } - return cdceval.NormalizeAndValidateSelectForTarget( - ctx, execCtx, tableDescr, targets[0], sc, includeVirtual, keyOnly, splitColFams) + splitColFams := opts.IsSet(changefeedbase.OptSplitColumnFamilies) + norm, err := cdceval.NormalizeExpression( + ctx, execCtx.ExecCfg(), execCtx.User(), execCtx.SessionData().SessionData, + tableDescr, statementTime, targets[0], sc, splitColFams) + if err != nil { + return nil, err + } + return norm, nil } type changefeedResumer struct { diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index ce6784a9bc14..e1b27d6bd507 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -67,9 +67,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/sqllivenesstestutils" "github.com/cockroachdb/cockroach/pkg/storage" @@ -4550,7 +4547,7 @@ func TestChangefeedPanicRecovery(t *testing.T) { prep(t, sqlDB) // Check that disallowed expressions have a good error message. // Also regression test for https://github.com/cockroachdb/cockroach/issues/90416 - sqlDB.ExpectErr(t, "expression currently unsupported in CREATE CHANGEFEED", + sqlDB.ExpectErr(t, "sub-query expressions not supported by CDC", `CREATE CHANGEFEED WITH schema_change_policy='stop' AS SELECT 1 FROM foo WHERE EXISTS (SELECT true)`) }) @@ -4794,21 +4791,21 @@ func TestCDCPrev(t *testing.T) { sqlDB.Exec(t, `INSERT INTO foo VALUES (0, 'initial')`) sqlDB.Exec(t, `UPSERT INTO foo VALUES (0, 'updated')`) // TODO(#85143): remove schema_change_policy='stop' from this test. - foo := feed(t, f, `CREATE CHANGEFEED WITH envelope='row', schema_change_policy='stop' AS SELECT cdc_prev()->'b' AS old FROM foo`) + foo := feed(t, f, `CREATE CHANGEFEED WITH envelope='row', schema_change_policy='stop' AS SELECT cdc_prev.b AS old FROM foo`) defer closeFeed(t, foo) - // cdc_prev() values are null during initial scan + // cdc_prev values are null during initial scan assertPayloads(t, foo, []string{ `foo: [0]->{"old": null}`, }) - // cdc_prev() values are null for an insert event + // cdc_prev values are null for an insert event sqlDB.Exec(t, `INSERT INTO foo VALUES (1, 'original')`) assertPayloads(t, foo, []string{ `foo: [1]->{"old": null}`, }) - // cdc_prev() returns the previous value on an update + // cdc_prev returns the previous value on an update sqlDB.Exec(t, `UPSERT INTO foo VALUES (1, 'updated')`) assertPayloads(t, foo, []string{ `foo: [1]->{"old": "original"}`, @@ -6781,105 +6778,6 @@ INSERT INTO foo (a, b, e) VALUES (3, 'tres', 'closed'); -- should be emitted }) } -// Verify when running predicate changefeed, the set of spans is constrained -// based on predicate expression. -func TestChangefeedConstrainsSpansBasedOnPredicate(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - testFn := func(t *testing.T, s TestServer, f cdctest.TestFeedFactory) { - sqlDB := sqlutils.MakeSQLRunner(s.DB) - sqlDB.Exec(t, `CREATE TYPE status AS ENUM ('open', 'closed', 'inactive')`) - sqlDB.Exec(t, ` -CREATE TABLE foo ( - a INT, - b STRING, - c STRING, - d STRING AS (concat(b, c)) VIRTUAL, - e status DEFAULT 'inactive', - PRIMARY KEY (a, b) -)`) - - sqlDB.Exec(t, ` -INSERT INTO foo (a, b) VALUES (0, 'zero'), (1, 'one'); -INSERT INTO foo (a, b, e) VALUES (2, 'two', 'closed'); -INSERT INTO foo (a, b, e) VALUES (11, 'eleven', 'closed'); -`) - // Save change aggregator specs. - knobs := s.TestingKnobs. - DistSQL.(*execinfra.TestingKnobs). - Changefeed.(*TestingKnobs) - specs := make(chan []*execinfrapb.ChangeAggregatorSpec, 1) - knobs.OnDistflowSpec = func( - aggregatorSpecs []*execinfrapb.ChangeAggregatorSpec, _ *execinfrapb.ChangeFrontierSpec, - ) { - specs <- aggregatorSpecs - } - - // TODO(#85143): remove schema_change_policy='stop' from this test. - feed := feed(t, f, ` -CREATE CHANGEFEED -WITH schema_change_policy='stop' -AS SELECT * FROM foo -WHERE a > 10 AND e IN ('open', 'closed') AND NOT cdc_is_delete()`) - defer closeFeed(t, feed) - - assertPayloads(t, feed, []string{ - `foo: [11, "eleven"]->{"a": 11, "b": "eleven", "c": null, "e": "closed"}`, - }) - - aggSpec := <-specs - require.Equal(t, 1, len(aggSpec)) - require.Equal(t, 1, len(aggSpec[0].Watches)) - - // Verify span is "smaller" than the primary index span. - fooDesc := desctestutils.TestingGetPublicTableDescriptor( - s.Server.ExecutorConfig().(sql.ExecutorConfig).DB, s.Codec, "d", "foo") - span := aggSpec[0].Watches[0].Span - require.Equal(t, -1, fooDesc.PrimaryIndexSpan(s.Codec).Key.Compare(span.Key)) - - // Aggregators should get modified select expression reflecting the fact - // that the set of spans was reduced (note: we no longer expect to see a > - // 10). - expectedExpr := normalizeCDCExpression(t, s.Server.ExecutorConfig(), - `SELECT * FROM foo WHERE (e IN ('open':::d.public.status, 'closed':::d.public.status)) AND (NOT cdc_is_delete())`) - require.Equal(t, expectedExpr, aggSpec[0].Select.Expr) - } - - cdcTest(t, testFn) -} - -func normalizeCDCExpression(t *testing.T, execCfgI interface{}, exprStr string) string { - t.Helper() - - sc, err := cdceval.ParseChangefeedExpression(exprStr) - require.NoError(t, err) - - desc := cdctest.GetHydratedTableDescriptor(t, execCfgI, - "d", "public", tree.Name(tree.AsString(sc.From.Tables[0]))) - target := jobspb.ChangefeedTargetSpecification{TableID: desc.GetID()} - - ctx := context.Background() - execCfg := execCfgI.(sql.ExecutorConfig) - - p, cleanup := sql.NewInternalPlanner("test", - execCfg.DB.NewTxn(ctx, "test-planner"), - username.RootUserName(), &sql.MemoryMetrics{}, &execCfg, - sessiondatapb.SessionData{ - Database: "d", - SearchPath: sessiondata.DefaultSearchPath.GetPathArray(), - }) - defer cleanup() - - execCtx := p.(sql.PlanHookState) - _, _, err = cdceval.NormalizeAndValidateSelectForTarget( - context.Background(), execCtx, desc, target, sc, false, false, false, - ) - require.NoError(t, err) - log.Infof(context.Background(), "PostNorm: %s", tree.StmtDebugString(sc)) - return cdceval.AsStringUnredacted(sc) -} - // Some predicates and projections can be verified when creating changefeed. // The types of errors that can be detected early on is restricted to simple checks // (such as type checking, non-existent columns, etc). More complex errors detected @@ -6912,7 +6810,7 @@ CREATE TABLE foo ( { name: "no such column", create: `CREATE CHANGEFEED INTO 'null://' AS SELECT no_such_column FROM foo`, - err: `column "no_such_column" does not exist`, + err: `column "foo.no_such_column" does not exist`, }, { name: "wrong type", @@ -6927,12 +6825,12 @@ CREATE TABLE foo ( { name: "contradiction: a > 1 && a < 1", create: `CREATE CHANGEFEED INTO 'null://' AS SELECT * FROM foo WHERE a > 1 AND a < 1`, - err: `filter .* is a contradiction`, + err: `does not match any rows`, }, { name: "contradiction: a IS null", create: `CREATE CHANGEFEED INTO 'null://' AS SELECT * FROM foo WHERE a IS NULL`, - err: `filter .* is a contradiction`, + err: `does not match any rows`, }, { name: "wrong table name", @@ -7013,7 +6911,6 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { } for _, tc := range []testCase{ - // The default policy is to skip schema changes which add new columns which { name: "add column no default", createFeedStmt: "CREATE CHANGEFEED AS SELECT * FROM foo", @@ -7066,7 +6963,7 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { createFeedStmt: "CREATE CHANGEFEED AS SELECT a, b, c, e FROM foo", initialPayload: initialPayload, alterStmt: "ALTER TABLE foo DROP COLUMN c", - expectErr: `while evaluating projection: SELECT .*: column "c" does not exist`, + expectErr: `column "foo.c" does not exist`, }, { name: "drop referenced column filter", @@ -7075,7 +6972,7 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { `foo: [2, "two"]->{"a": 2, "b": "two", "c": "c string", "e": "open"}`, }, alterStmt: "ALTER TABLE foo DROP COLUMN c", - expectErr: `while matching filter: SELECT .*: column "c" does not exist`, + expectErr: `column "foo.c" does not exist`, }, { name: "rename referenced column projection", @@ -7083,7 +6980,7 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { initialPayload: initialPayload, alterStmt: "ALTER TABLE foo RENAME COLUMN c TO c_new", afterAlterStmt: "INSERT INTO foo (a, b) VALUES (3, 'tres')", - expectErr: `while evaluating projection: SELECT .*: column "c" does not exist`, + expectErr: `column "foo.c" does not exist`, }, { name: "rename referenced column filter", @@ -7093,7 +6990,7 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { }, alterStmt: "ALTER TABLE foo RENAME COLUMN c TO c_new", afterAlterStmt: "INSERT INTO foo (a, b) VALUES (3, 'tres')", - expectErr: `while matching filter: SELECT .*: column "c" does not exist`, + expectErr: `column "foo.c" does not exist`, }, { name: "alter enum", @@ -7117,7 +7014,7 @@ func TestChangefeedPredicateWithSchemaChange(t *testing.T) { }, { name: "alter enum use correct enum version", - createFeedStmt: "CREATE CHANGEFEED AS SELECT e, cdc_prev()->'e' AS prev_e FROM foo", + createFeedStmt: "CREATE CHANGEFEED AS SELECT e, cdc_prev.e AS prev_e FROM foo", initialPayload: []string{ `foo: [1, "one"]->{"e": "inactive", "prev_e": null}`, `foo: [2, "two"]->{"e": "open", "prev_e": null}`, diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go index 1ca642fe8e59..9ba1c2b00af2 100644 --- a/pkg/ccl/changefeedccl/event_processing.go +++ b/pkg/ccl/changefeedccl/event_processing.go @@ -20,10 +20,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" @@ -65,7 +65,6 @@ type kvEventToRowConsumer struct { decoder cdcevent.Decoder details ChangefeedConfig evaluator *cdceval.Evaluator - safeExpr string encodingFormat changefeedbase.FormatType topicDescriptorCache map[TopicIdentifier]TopicDescriptor @@ -85,20 +84,15 @@ type kvEventToRowConsumer struct { func newEventConsumer( ctx context.Context, - flowCtx *execinfra.FlowCtx, + cfg *execinfra.ServerConfig, + spec execinfrapb.ChangeAggregatorSpec, feed ChangefeedConfig, spanFrontier *span.Frontier, cursor hlc.Timestamp, sink EventSink, - details ChangefeedConfig, - expr execinfrapb.Expression, - knobs TestingKnobs, metrics *Metrics, - isSinkless bool, + knobs TestingKnobs, ) (eventConsumer, EventSink, error) { - cfg := flowCtx.Cfg - evalCtx := flowCtx.EvalCtx - encodingOpts, err := feed.Opts.GetEncodingOptions() if err != nil { return nil, nil, err @@ -130,6 +124,7 @@ func newEventConsumer( if !ok { tenantID = roachpb.SystemTenantID } + pacer = cfg.AdmissionPacerFactory.NewPacer( pacerRequestUnit, admission.WorkInfo{ @@ -141,8 +136,9 @@ func newEventConsumer( ) } - return newKVEventToRowConsumer(ctx, cfg, evalCtx, frontier, cursor, s, - encoder, details, expr, knobs, topicNamer, pacer) + execCfg := cfg.ExecutorConfig.(*sql.ExecutorConfig) + return newKVEventToRowConsumer(ctx, execCfg, frontier, cursor, s, + encoder, feed, spec.Select, spec.User(), knobs, topicNamer, pacer) } numWorkers := changefeedbase.EventConsumerWorkers.Get(&cfg.Settings.SV) @@ -160,6 +156,7 @@ func newEventConsumer( // does not work for parquet format. // // TODO (jayshrivastava) enable parallel consumers for sinkless changefeeds. + isSinkless := spec.JobID == 0 if numWorkers <= 1 || isSinkless || encodingOpts.Format == changefeedbase.OptFormatParquet { c, err := makeConsumer(sink, spanFrontier) if err != nil { @@ -208,35 +205,28 @@ func makeHasher() hash.Hash32 { func newKVEventToRowConsumer( ctx context.Context, - cfg *execinfra.ServerConfig, - evalCtx *eval.Context, + cfg *sql.ExecutorConfig, frontier frontier, cursor hlc.Timestamp, sink EventSink, encoder Encoder, details ChangefeedConfig, expr execinfrapb.Expression, + userName username.SQLUsername, knobs TestingKnobs, topicNamer *TopicNamer, pacer *admission.Pacer, -) (*kvEventToRowConsumer, error) { +) (_ *kvEventToRowConsumer, err error) { includeVirtual := details.Opts.IncludeVirtual() keyOnly := details.Opts.KeyOnly() decoder, err := cdcevent.NewEventDecoder(ctx, cfg, details.Targets, includeVirtual, keyOnly) - if err != nil { return nil, err } var evaluator *cdceval.Evaluator - var safeExpr string if expr.Expr != "" { - expr, err := cdceval.ParseChangefeedExpression(expr.Expr) - if err != nil { - return nil, err - } - safeExpr = tree.AsString(expr) - evaluator, err = cdceval.NewEvaluator(ctx, evalCtx, expr) + evaluator, err = newEvaluator(cfg, userName, expr) if err != nil { return nil, err } @@ -258,12 +248,22 @@ func newKVEventToRowConsumer( topicDescriptorCache: make(map[TopicIdentifier]TopicDescriptor), topicNamer: topicNamer, evaluator: evaluator, - safeExpr: safeExpr, encodingFormat: encodingOpts.Format, pacer: pacer, }, nil } +func newEvaluator( + cfg *sql.ExecutorConfig, user username.SQLUsername, expr execinfrapb.Expression, +) (*cdceval.Evaluator, error) { + sc, err := cdceval.ParseChangefeedExpression(expr.Expr) + if err != nil { + return nil, err + } + + return cdceval.NewEvaluator(sc, cfg, user) +} + func (c *kvEventToRowConsumer) topicForEvent(eventMeta cdcevent.Metadata) (TopicDescriptor, error) { if topic, ok := c.topicDescriptorCache[TopicIdentifier{TableID: eventMeta.TableID, FamilyID: eventMeta.FamilyID}]; ok { if topic.GetVersion() == eventMeta.Version { @@ -279,7 +279,8 @@ func (c *kvEventToRowConsumer) topicForEvent(eventMeta cdcevent.Metadata) (Topic c.topicDescriptorCache[topic.GetTopicIdentifier()] = topic return topic, nil } - return noTopic{}, errors.AssertionFailedf("no TargetSpecification for row %s", eventMeta) + return noTopic{}, errors.AssertionFailedf("no TargetSpecification for row %s", + eventMeta.DebugString()) } // ConsumeEvent manages kv event lifetime: parsing, encoding and event being emitted to the sink. @@ -299,7 +300,6 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even schemaTimestamp := ev.KV().Value.Timestamp prevSchemaTimestamp := schemaTimestamp - mvccTimestamp := ev.MVCCTimestamp() keyOnly := c.details.Opts.KeyOnly() if backfillTs := ev.BackfillTimestamp(); !backfillTs.IsEmpty() { @@ -334,29 +334,34 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even } if c.evaluator != nil { - matches, err := c.evaluator.MatchesFilter(ctx, updatedRow, mvccTimestamp, prevRow) + projection, err := c.evaluator.Eval(ctx, updatedRow, prevRow) if err != nil { - return errors.Wrapf(err, "while matching filter: %s", c.safeExpr) + return err } - if !matches { - // TODO(yevgeniy): Add metrics + if !projection.IsInitialized() { + // Filter did not match. + // TODO(yevgeniy): Add metrics. a := ev.DetachAlloc() a.Release(ctx) return nil } - projection, err := c.evaluator.Projection(ctx, updatedRow, mvccTimestamp, prevRow) - if err != nil { - return errors.Wrapf(err, "while evaluating projection: %s", c.safeExpr) - } - updatedRow = projection - // Clear out prevRow. Projection can already emit previous row; thus // it would be superfluous to also encode prevRow. - prevRow = cdcevent.Row{} + updatedRow, prevRow = projection, cdcevent.Row{} } + return c.encodeAndEmit(ctx, updatedRow, prevRow, schemaTimestamp, ev.DetachAlloc()) +} + +func (c *kvEventToRowConsumer) encodeAndEmit( + ctx context.Context, + updatedRow cdcevent.Row, + prevRow cdcevent.Row, + schemaTS hlc.Timestamp, + alloc kvevent.Alloc, +) error { topic, err := c.topicForEvent(updatedRow.Metadata) if err != nil { return err @@ -368,15 +373,15 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even // it's forwarded before. // TODO(dan): This should be an assertion once we're confident this can never // happen under any circumstance. - if schemaTimestamp.LessEq(c.frontier.Frontier()) && !schemaTimestamp.Equal(c.cursor) { + if schemaTS.LessEq(c.frontier.Frontier()) && !schemaTS.Equal(c.cursor) { log.Errorf(ctx, "cdc ux violation: detected timestamp %s that is less than "+ - "or equal to the local frontier %s.", schemaTimestamp, c.frontier.Frontier()) + "or equal to the local frontier %s.", schemaTS, c.frontier.Frontier()) return nil } evCtx := eventContext{ - updated: schemaTimestamp, - mvcc: mvccTimestamp, + updated: schemaTS, + mvcc: updatedRow.MvccTimestamp, } if c.topicNamer != nil { @@ -395,13 +400,7 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even if c.encodingFormat == changefeedbase.OptFormatParquet { return c.encodeForParquet( - ctx, - updatedRow, - prevRow, - topic, - schemaTimestamp, - mvccTimestamp, - ev.DetachAlloc(), + ctx, updatedRow, prevRow, topic, schemaTS, updatedRow.MvccTimestamp, alloc, ) } var keyCopy, valueCopy []byte @@ -420,11 +419,10 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even // Since we're done processing/converting this event, and will not use much more // than len(key)+len(bytes) worth of resources, adjust allocation to match. - a := ev.DetachAlloc() - a.AdjustBytesToTarget(ctx, int64(len(keyCopy)+len(valueCopy))) + alloc.AdjustBytesToTarget(ctx, int64(len(keyCopy)+len(valueCopy))) if err := c.sink.EmitRow( - ctx, topic, keyCopy, valueCopy, schemaTimestamp, mvccTimestamp, a, + ctx, topic, keyCopy, valueCopy, schemaTS, updatedRow.MvccTimestamp, alloc, ); err != nil { return err } @@ -434,10 +432,12 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even return nil } -// Close is a noop for the kvEventToRowConsumer because it -// has no goroutines in flight. +// Close closes this consumer. func (c *kvEventToRowConsumer) Close() error { c.pacer.Close() + if c.evaluator != nil { + c.evaluator.Close() + } return nil } @@ -589,6 +589,7 @@ func (c *parallelEventConsumer) workerLoop( log.Errorf(ctx, "closing consumer: %v", err) } }() + for { select { case <-ctx.Done(): diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 7a15351815c8..51975df9dd0d 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -242,12 +242,6 @@ func (r *Registry) SetInternalExecutorFactory(factory sqlutil.InternalExecutorFa r.internalExecutorFactory = factory } -// NewSpanConstrainer returns an instance of sql.SpanConstrainer as an interface{}, -// and a cleanup function. -func (r *Registry) NewSpanConstrainer(user username.SQLUsername) (interface{}, func()) { - return r.execCtx("constrain-spans", user) -} - // MetricsStruct returns the metrics for production monitoring of each job type. // They're all stored as the `metric.Struct` interface because of dependency // cycles. diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 9a1177016025..15ca090966c9 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -749,6 +749,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { TenantCostController: cfg.costController, RangeStatsFetcher: rangeStatsFetcher, AdmissionPacerFactory: cfg.admissionPacerFactory, + ExecutorConfig: execCfg, } cfg.TempStorageConfig.Mon.SetMetrics(distSQLMetrics.CurDiskBytesCount, distSQLMetrics.MaxDiskBytesHist) if distSQLTestingKnobs := cfg.TestingKnobs.DistSQL; distSQLTestingKnobs != nil { diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 8fece0c47a02..889d781ea044 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -48,7 +48,6 @@ go_library( "conn_executor_show_commit_timestamp.go", "conn_fsm.go", "conn_io.go", - "constraint.go", "control_jobs.go", "control_schedules.go", "copy.go", @@ -387,12 +386,9 @@ go_library( "//pkg/sql/opt/exec", "//pkg/sql/opt/exec/execbuilder", "//pkg/sql/opt/exec/explain", - "//pkg/sql/opt/idxconstraint", "//pkg/sql/opt/indexrec", "//pkg/sql/opt/memo", - "//pkg/sql/opt/norm", "//pkg/sql/opt/optbuilder", - "//pkg/sql/opt/partition", "//pkg/sql/opt/xform", "//pkg/sql/optionalnodeliveness", "//pkg/sql/paramparse", @@ -558,7 +554,6 @@ go_test( "conn_executor_savepoints_test.go", "conn_executor_test.go", "conn_io_test.go", - "constraint_test.go", "copy_in_test.go", "copy_test.go", "crdb_internal_test.go", diff --git a/pkg/sql/constraint.go b/pkg/sql/constraint.go deleted file mode 100644 index 3d2da7da8b98..000000000000 --- a/pkg/sql/constraint.go +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package sql - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" - "github.com/cockroachdb/cockroach/pkg/sql/opt" - "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder" - "github.com/cockroachdb/cockroach/pkg/sql/opt/idxconstraint" - "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" - "github.com/cockroachdb/cockroach/pkg/sql/opt/norm" - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" - "github.com/cockroachdb/cockroach/pkg/sql/opt/partition" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/span" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/errors" -) - -// SpanConstraintRequirement indicates how strict span constraint logic should be. -type SpanConstraintRequirement int - -const ( - // MustFullyConstrain indicates spans must be fully constrained. - MustFullyConstrain SpanConstraintRequirement = iota - // BestEffortConstrain allows constraint to partially satisfy predicate. - BestEffortConstrain -) - -// SpanConstrainer is an interface for constraining spans. -type SpanConstrainer interface { - // ConstrainPrimaryIndexSpanByExpr constrains primary index span of the - // table using specified filter expression. - // Table name parameter is the name of the table used in the expression. - // Returns constrained spans that satisfy the expression. - // If the caller requires constraint to be MustFullyConstrain, but the - // expression cannot be fully satisfied, returns an error. - // The expression must be boolean expression. - // If the expression is a contradiction, returns an error. - ConstrainPrimaryIndexSpanByExpr( - ctx context.Context, - req SpanConstraintRequirement, - tn *tree.TableName, - desc catalog.TableDescriptor, - evalCtx *eval.Context, - semaCtx *tree.SemaContext, - filter tree.Expr, - ) (_ []roachpb.Span, remainingFilter tree.Expr, _ error) -} - -// ConstrainPrimaryIndexSpanByExpr implements SpanConstrainer -func (p *planner) ConstrainPrimaryIndexSpanByExpr( - ctx context.Context, - req SpanConstraintRequirement, - tn *tree.TableName, - desc catalog.TableDescriptor, - evalCtx *eval.Context, - semaCtx *tree.SemaContext, - filter tree.Expr, -) (_ []roachpb.Span, remainingFilter tree.Expr, _ error) { - var oc optCatalog - oc.init(p) - oc.reset() - - tbl, err := newOptTable(desc, oc.codec(), nil /* stats */, emptyZoneConfig) - if err != nil { - return nil, nil, err - } - - var nf norm.Factory - nf.Init(ctx, evalCtx, &oc) - nf.Metadata().AddTable(tbl, tn) - - b := optbuilder.NewScalar(ctx, semaCtx, evalCtx, &nf) - if err := b.Build(filter); err != nil { - return nil, nil, err - } - - root := nf.Memo().RootExpr().(opt.ScalarExpr) - if root.DataType() != types.Bool { - return nil, nil, pgerror.Newf(pgcode.DatatypeMismatch, - "expected boolean expression, found expression of type %s", root.DataType()) - } - - fe := memo.FiltersExpr{nf.ConstructFiltersItem(root)} - fe = nf.CustomFuncs().SimplifyFilters(fe) - fe = nf.CustomFuncs().ConsolidateFilters(fe) - - if fe.IsTrue() { - return []roachpb.Span{desc.PrimaryIndexSpan(oc.codec())}, tree.DBoolTrue, nil - } - if fe.IsFalse() { - return nil, nil, errors.Newf("filter %q is a contradiction", filter) - } - - primary := desc.GetPrimaryIndex() - indexCols := make([]opt.OrderingColumn, len(primary.IndexDesc().KeyColumnIDs)) - var notNullIndexCols opt.ColSet - for i, colID := range primary.IndexDesc().KeyColumnIDs { - if primary.GetKeyColumnDirection(i) == catpb.IndexColumn_ASC { - indexCols[i] = opt.OrderingColumn(colID) - } else { - indexCols[i] = opt.OrderingColumn(-colID) - } - notNullIndexCols.Add(opt.ColumnID(colID)) - } - - const consolidate = true - var ic idxconstraint.Instance - - ic.Init( - fe, nil, indexCols, notNullIndexCols, nil, - consolidate, evalCtx, &nf, partition.PrefixSorter{}, - ) - - remaining := ic.RemainingFilters() - if req == MustFullyConstrain { - if !remaining.IsTrue() { - err = errors.Newf( - "primary key span %s cannot be fully constrained by expression %q", - desc.PrimaryIndexSpan(oc.codec()), filter) - if len(indexCols) > 1 { - // Constraints over composite keys are hard. Give a bit of a hint. - err = errors.WithHint(err, - "try constraining prefix columns of the composite key with equality or an IN clause") - } - return nil, nil, err - } - - if ic.Constraint().IsUnconstrained() { - return nil, nil, errors.Newf("filter %q is a tautology; use 'true' or omit constraint", filter) - } - } - - if ic.Constraint().IsContradiction() { - return nil, nil, errors.Newf("filter %q is a contradiction", filter) - } - - if remaining.IsTrue() { - remainingFilter = tree.DBoolTrue - } else { - eb := execbuilder.New(ctx, newExecFactory(ctx, p), &p.optPlanningCtx.optimizer, - nf.Memo(), &oc, &remaining, evalCtx, false, p.IsANSIDML()) - eb.SetBuiltinFuncWrapper(semaCtx.FunctionResolver) - expr, err := eb.BuildScalar() - if err != nil { - return nil, nil, err - } - remainingFilter = replaceIndexedVarsWithColumnNames(tbl, tn, expr) - } - - var sb span.Builder - sb.Init(evalCtx, oc.codec(), desc, desc.GetPrimaryIndex()) - spans, err := sb.SpansFromConstraint(ic.Constraint(), span.NoopSplitter()) - if err != nil { - return nil, nil, err - } - return spans, remainingFilter, nil -} - -type replaceIndexedVars struct { - tbl *optTable - tn *tree.UnresolvedObjectName -} - -var _ tree.Visitor = (*replaceIndexedVars)(nil) - -func replaceIndexedVarsWithColumnNames( - tbl *optTable, alias *tree.TableName, expr tree.Expr, -) tree.Expr { - var tn *tree.UnresolvedObjectName - if alias.Table() != "" { - tn = alias.ToUnresolvedObjectName() - } - v := replaceIndexedVars{tbl: tbl, tn: tn} - expr, _ = tree.WalkExpr(&v, expr) - return expr -} - -func (v *replaceIndexedVars) VisitPre(expr tree.Expr) (bool, tree.Expr) { - return true, expr -} - -func (v *replaceIndexedVars) VisitPost(expr tree.Expr) tree.Expr { - switch t := expr.(type) { - case *tree.IndexedVar: - return &tree.ColumnItem{ColumnName: v.tbl.Column(t.Idx).ColName(), TableName: v.tn} - default: - return expr - } -} diff --git a/pkg/sql/constraint_test.go b/pkg/sql/constraint_test.go deleted file mode 100644 index 872158bd7a32..000000000000 --- a/pkg/sql/constraint_test.go +++ /dev/null @@ -1,276 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package sql - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" - "github.com/cockroachdb/cockroach/pkg/sql/parser" - "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" - "github.com/cockroachdb/cockroach/pkg/sql/tests" - "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -func mkPkKey(t *testing.T, tableID descpb.ID, vals ...int) roachpb.Key { - t.Helper() - - // Encode index id, then each value. - key, err := keyside.Encode( - keys.SystemSQLCodec.TablePrefix(uint32(tableID)), - tree.NewDInt(tree.DInt(1)), encoding.Ascending) - - require.NoError(t, err) - for _, v := range vals { - d := tree.NewDInt(tree.DInt(v)) - key, err = keyside.Encode(key, d, encoding.Ascending) - require.NoError(t, err) - } - - return key -} - -func TestSpanConstrainer(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - params, _ := tests.CreateTestServerParams() - s, db, kvDB := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) - - sqlDB := sqlutils.MakeSQLRunner(db) - sqlDB.Exec(t, `CREATE TABLE foo (a INT, b int, c STRING, CONSTRAINT "pk" PRIMARY KEY (a, b), INDEX (c))`) - fooDesc := desctestutils.TestingGetTableDescriptor( - kvDB, keys.SystemSQLCodec, "defaultdb", "public", "foo") - - ctx := context.Background() - execCfg := s.ExecutorConfig().(ExecutorConfig) - p, cleanup := NewInternalPlanner("test", kv.NewTxn(ctx, kvDB, s.NodeID()), - username.RootUserName(), &MemoryMetrics{}, &execCfg, sessiondatapb.SessionData{}, - ) - defer cleanup() - - primarySpan := fooDesc.PrimaryIndexSpan(keys.SystemSQLCodec) - pkStart := primarySpan.Key - pkEnd := primarySpan.EndKey - fooID := fooDesc.GetID() - - sc := p.(SpanConstrainer) - evalCtx := eval.MakeTestingEvalContext(s.ClusterSettings()) - semaCtx := tree.MakeSemaContext() - for _, tc := range []struct { - filter string - req SpanConstraintRequirement - expectErr string - remaining string - alias *tree.TableName - expectSpans []roachpb.Span - }{ - { - filter: "5 > 1", - expectSpans: []roachpb.Span{primarySpan}, - }, - { - filter: "0 != 0", - expectErr: "is a contradiction", - }, - { - filter: "a IS NULL", - expectErr: "is a contradiction", - }, - { - filter: "a > 3 AND a < 3", - expectErr: "is a contradiction", - }, - { - filter: "a >=3 or a < 3", - expectErr: "is a tautology", - }, - { - filter: "a >=3 or a < 3", - req: BestEffortConstrain, - expectSpans: []roachpb.Span{primarySpan}, - }, - { - filter: "5", - expectErr: "expected boolean expression", - }, - { - filter: "no_such_column = 'something'", - expectErr: `column "no_such_column" does not exist`, - }, - { - filter: "true", - expectSpans: []roachpb.Span{primarySpan}, - }, - { - filter: "false", - expectErr: "is a contradiction", - }, - { - filter: "a > 100", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 101), EndKey: pkEnd}}, - }, - { - filter: "a < 100", - expectSpans: []roachpb.Span{{Key: pkStart, EndKey: mkPkKey(t, fooID, 100)}}, - }, - { - filter: "a > 10 AND a > 5", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 11), EndKey: pkEnd}}, - }, - { - filter: "a > 10 OR a > 5", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 6), EndKey: pkEnd}}, - }, - { - filter: "a > 100 AND a <= 101", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 101), EndKey: mkPkKey(t, fooID, 102)}}, - }, - { - filter: "a > 100 and a < 200", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 101), EndKey: mkPkKey(t, fooID, 200)}}, - }, - { - filter: "a > 100 or a <= 99", - expectSpans: []roachpb.Span{ - {Key: pkStart, EndKey: mkPkKey(t, fooID, 100)}, - {Key: mkPkKey(t, fooID, 101), EndKey: pkEnd}, - }, - }, - { - filter: "b < 42", - expectErr: "cannot be fully constrained", - }, - { - filter: "c = 'ten'", - expectErr: "cannot be fully constrained", - }, - { - filter: "a < 42 OR (a > 100 AND b > 11)", - expectErr: "cannot be fully constrained", - }, - { - filter: "a > 100 AND b > 11", - remaining: "b > 11", - req: BestEffortConstrain, - expectSpans: []roachpb.Span{ - {Key: mkPkKey(t, fooID, 101, 12), EndKey: pkEnd}, - }, - }, - { - // Same as above, but with table alias -- we expect remaining expression to - // preserve the alias. - filter: "buz.a > 100 AND b > 11", - remaining: "buz.b > 11", - req: BestEffortConstrain, - alias: tree.NewUnqualifiedTableName("buz"), - expectSpans: []roachpb.Span{ - {Key: mkPkKey(t, fooID, 101, 12), EndKey: pkEnd}, - }, - }, - { - // Same as above, but w/ silly tautology, which should be removed. - filter: "(a > 3 OR a <= 3) AND a > 100 AND b > 11", - alias: tree.NewTableNameWithSchema("defaultdb", "public", "foo"), - remaining: "defaultdb.public.foo.b > 11", // When alias specified, it is preserved. - req: BestEffortConstrain, - expectSpans: []roachpb.Span{ - {Key: mkPkKey(t, fooID, 101, 12), EndKey: pkEnd}, - }, - }, - { - filter: "a < 42 OR (a > 100 AND b > 11)", - remaining: "(a < 42) OR ((a > 100) AND (b > 11))", - req: BestEffortConstrain, - expectSpans: []roachpb.Span{ - {Key: pkStart, EndKey: mkPkKey(t, fooID, 42)}, - {Key: mkPkKey(t, fooID, 101, 12), EndKey: pkEnd}, - }, - }, - { - filter: "a > 2 AND b > 5 AND a > 2", - expectErr: "cannot be fully constrained", - }, - { - // Same as above, but now with tuples. - filter: "a < 42 OR ((a, b) > (100, 11))", - expectSpans: []roachpb.Span{ - {Key: pkStart, EndKey: mkPkKey(t, fooID, 42)}, - // Remember: tuples use lexicographical ordering so the start key is - // /Table/104/1/100/12 (i.e. a="100" and b="12" (because 100/12 lexicographically follows 100). - {Key: mkPkKey(t, fooID, 100, 12), EndKey: pkEnd}, - }, - }, - { - filter: "(a, b) > (2, 5)", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 2, 6), EndKey: pkEnd}}, - }, - { - // Test that aliased table names work. - filter: "(buz.a, buz.b) > (2, 5)", - expectSpans: []roachpb.Span{{Key: mkPkKey(t, fooID, 2, 6), EndKey: pkEnd}}, - alias: tree.NewUnqualifiedTableName("buz"), - }, - { - // This test also uses qualified names for some fields. - filter: "foo.a IN (5, 10, 20) AND b < 25", - alias: tree.NewUnqualifiedTableName("foo"), - expectSpans: []roachpb.Span{ - {Key: mkPkKey(t, fooID, 5), EndKey: mkPkKey(t, fooID, 5, 25)}, - {Key: mkPkKey(t, fooID, 10), EndKey: mkPkKey(t, fooID, 10, 25)}, - {Key: mkPkKey(t, fooID, 20), EndKey: mkPkKey(t, fooID, 20, 25)}, - }, - }, - } { - t.Run(tc.filter, func(t *testing.T) { - filterExpr, err := parser.ParseExpr(tc.filter) - require.NoError(t, err) - - alias := tc.alias - if alias == nil { - var tn tree.TableName - alias = &tn - } - - spans, remaining, err := sc.ConstrainPrimaryIndexSpanByExpr( - ctx, tc.req, alias, fooDesc, &evalCtx, &semaCtx, filterExpr) - if tc.expectErr != "" { - require.Regexp(t, tc.expectErr, err) - require.Nil(t, spans) - return - } - - require.NoError(t, err) - require.Equal(t, tc.expectSpans, spans) - - if tc.remaining == "" { - require.Equal(t, tree.DBoolTrue, remaining, tree.AsStringWithFlags(remaining, tree.FmtExport)) - } else { - require.Equal(t, tc.remaining, tree.AsStringWithFlags(remaining, tree.FmtExport)) - } - }) - } -} diff --git a/pkg/sql/distsql_plan_changefeed.go b/pkg/sql/distsql_plan_changefeed.go index 783bda4220de..5b07d2e2ae96 100644 --- a/pkg/sql/distsql_plan_changefeed.go +++ b/pkg/sql/distsql_plan_changefeed.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" - "github.com/cockroachdb/cockroach/pkg/sql/opt/optbuilder" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -74,31 +73,21 @@ func PlanCDCExpression( cdcCat := &cdcOptCatalog{ optCatalog: opc.catalog.(*optCatalog), + semaCtx: &p.semaCtx, } opc.catalog = cdcCat - // We could use opc.buildExecMemo; alas, it has too much logic we don't - // need, and, it also allows stable fold -- something we don't want to do. - // So, just build memo ourselves. - f := opc.optimizer.Factory() - f.FoldingControl().DisallowStableFolds() - bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), opc.catalog, f, opc.p.stmt.AST) - if err := bld.Build(); err != nil { - return cdcPlan, err - } - - oe, err := opc.optimizer.Optimize() + memo, err := opc.buildExecMemo(ctx) if err != nil { return cdcPlan, err } if log.V(2) { - log.Infof(ctx, "Optimized CDC expression: %s", oe.String()) + log.Infof(ctx, "Optimized CDC expression: %s", memo.RootExpr().String()) } - execMemo := f.Memo() const allowAutoCommit = false if err := opc.runExecBuilder( - ctx, &p.curPlan, &p.stmt, newExecFactory(ctx, p), execMemo, p.EvalContext(), allowAutoCommit, + ctx, &p.curPlan, &p.stmt, newExecFactory(ctx, p), memo, p.EvalContext(), allowAutoCommit, ); err != nil { return cdcPlan, err } @@ -280,11 +269,19 @@ func (n *cdcValuesNode) Values() tree.Datums { // Close implements planNode. func (n *cdcValuesNode) Close(ctx context.Context) { - n.source.ConsumerDone() + // Note that we do not call ConsumerClosed on it since it is not the + // responsibility of this node (the responsibility belongs to + // the caller -- cdc evaluation planNodeToRowSource). + // This node is used in the following tree: + // DistSQLReceiver <- (arbitrary DistSQL processors) <- planNodeToRowSource <- cdcValuesNode <- RowChannel + // RowChannel is added as the "input to drain" by planNodeToRowSource (in SetInput), + // so planNodeToRowSource will call ConsumerDone or ConsumerClosed + // (depending on why the flow is being shutdown). } type cdcOptCatalog struct { *optCatalog + semaCtx *tree.SemaContext } var _ cat.Catalog = (*cdcOptCatalog)(nil) @@ -325,6 +322,21 @@ func (c *cdcOptCatalog) ResolveDataSourceByID( return ds, false, nil } +// ResolveFunction implements cat.Catalog interface. +// We provide custom implementation to resolve CDC specific functions. +func (c *cdcOptCatalog) ResolveFunction( + ctx context.Context, fnName *tree.UnresolvedName, path tree.SearchPath, +) (*tree.ResolvedFunctionDefinition, error) { + if c.semaCtx != nil && c.semaCtx.FunctionResolver != nil { + fnDef, err := c.semaCtx.FunctionResolver.ResolveFunction(ctx, fnName, path) + if err != nil { + return nil, err + } + return fnDef, nil + } + return c.optCatalog.ResolveFunction(ctx, fnName, path) +} + // newCDCDataSource builds an optTable for the target cdc table. // The descriptor presented to the optimizer hides all but the primary index. // TODO(yevgeniy): We should be able to use secondary indexes provided diff --git a/pkg/sql/distsql_plan_changefeed_test.go b/pkg/sql/distsql_plan_changefeed_test.go index b087125ed3da..9b99f18e7f76 100644 --- a/pkg/sql/distsql_plan_changefeed_test.go +++ b/pkg/sql/distsql_plan_changefeed_test.go @@ -21,11 +21,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -34,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -457,3 +460,21 @@ func randEncDatumRow(rng *rand.Rand, desc catalog.TableDescriptor) (row rowenc.E } return row } + +func mkPkKey(t *testing.T, tableID descpb.ID, vals ...int) roachpb.Key { + t.Helper() + + // Encode index id, then each value. + key, err := keyside.Encode( + keys.SystemSQLCodec.TablePrefix(uint32(tableID)), + tree.NewDInt(tree.DInt(1)), encoding.Ascending) + + require.NoError(t, err) + for _, v := range vals { + d := tree.NewDInt(tree.DInt(v)) + key, err = keyside.Encode(key, d, encoding.Ascending) + require.NoError(t, err) + } + + return key +} diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index a9b5cc63e771..3a770b5d84e9 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -201,6 +201,9 @@ type ServerConfig struct { // AdmissionPacerFactory is used to integrate CPU-intensive work // with elastic CPU control. AdmissionPacerFactory admission.PacerFactory + + // *sql.ExecutorConfig exposed as an interface (due to dependency cycles). + ExecutorConfig interface{} } // RuntimeStats is an interface through which the rowexec layer can get diff --git a/pkg/sql/job_exec_context.go b/pkg/sql/job_exec_context.go index c8db7f0f9977..4bc1f7fd6ea4 100644 --- a/pkg/sql/job_exec_context.go +++ b/pkg/sql/job_exec_context.go @@ -11,14 +11,9 @@ package sql import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" - "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -70,19 +65,6 @@ func (e *plannerJobExecContext) SpanConfigReconciler() spanconfig.Reconciler { return e.p.SpanConfigReconciler() } -// ConstrainPrimaryIndexSpanByExpr implements SpanConstrainer -func (e *plannerJobExecContext) ConstrainPrimaryIndexSpanByExpr( - ctx context.Context, - req SpanConstraintRequirement, - tn *tree.TableName, - desc catalog.TableDescriptor, - evalCtx *eval.Context, - semaCtx *tree.SemaContext, - filter tree.Expr, -) ([]roachpb.Span, tree.Expr, error) { - return e.p.ConstrainPrimaryIndexSpanByExpr(ctx, req, tn, desc, evalCtx, semaCtx, filter) -} - // JobExecContext provides the execution environment for a job. It is what is // passed to the Resume/OnFailOrCancel/OnPauseRequested methods of a jobs's // Resumer to give that resumer access to things like ExecutorCfg, LeaseMgr, @@ -93,7 +75,6 @@ func (e *plannerJobExecContext) ConstrainPrimaryIndexSpanByExpr( // (though note that ExtendedEvalContext may transitively include methods that // close over/expect a txn so use it with caution). type JobExecContext interface { - SpanConstrainer SemaCtx() *tree.SemaContext ExtendedEvalContext() *extendedEvalContext SessionData() *sessiondata.SessionData diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 89e801cb000e..ea9881595e34 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -168,12 +168,6 @@ type Builder struct { // ScanCounts records the number of times scans were used in the query. ScanCounts [exec.NumScanCountTypes]int - // wrapFunctionOverride overrides default implementation to return resolvable - // function reference for function with specified function name. - // The default can be overridden by calling SetBuiltinFuncWrapper method to provide - // custom search path implementation. - wrapFunctionOverride func(fnName string) tree.ResolvableFunctionReference - // builtScans collects all scans in the operation tree so post-build checking // for non-local execution can be done. builtScans []*memo.ScanExpr @@ -258,27 +252,15 @@ func (b *Builder) Build() (_ exec.Plan, err error) { return b.factory.ConstructPlan(plan.root, b.subqueries, b.cascades, b.checks, rootRowCount) } -// SetBuiltinFuncWrapper configures this builder to use specified function resolver. -func (b *Builder) SetBuiltinFuncWrapper(resolver tree.FunctionReferenceResolver) { - // TODO(mgartner): The customFnResolver and wrapFunctionOverride could - // probably be replaced by a custom implementation of tree.FunctionResolver. - if customFnResolver, ok := resolver.(tree.CustomBuiltinFunctionWrapper); ok { - b.wrapFunctionOverride = func(fnName string) tree.ResolvableFunctionReference { - fd, err := customFnResolver.WrapFunction(fnName) - if err != nil { - panic(err) - } - if fd == nil { - panic(errors.AssertionFailedf("function %s() not defined", redact.Safe(fnName))) - } - return tree.ResolvableFunctionReference{FunctionReference: fd} - } - } -} - func (b *Builder) wrapFunction(fnName string) tree.ResolvableFunctionReference { - if b.wrapFunctionOverride != nil { - return b.wrapFunctionOverride(fnName) + if b.evalCtx != nil && b.catalog != nil { // Some tests leave those unset. + unresolved := tree.MakeUnresolvedName(fnName) + fnDef, err := b.catalog.ResolveFunction( + context.Background(), &unresolved, &b.evalCtx.SessionData().SearchPath) + if err != nil { + panic(err) + } + return tree.ResolvableFunctionReference{FunctionReference: fnDef} } return tree.WrapFunction(fnName) } diff --git a/pkg/sql/opt/norm/fold_constants_funcs.go b/pkg/sql/opt/norm/fold_constants_funcs.go index bc709aa1fb93..1ea799cd1c83 100644 --- a/pkg/sql/opt/norm/fold_constants_funcs.go +++ b/pkg/sql/opt/norm/fold_constants_funcs.go @@ -11,6 +11,8 @@ package norm import ( + "context" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -23,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" "github.com/lib/pq/oid" ) @@ -670,8 +673,7 @@ func (c *CustomFuncs) FunctionReturnType(private *memo.FunctionPrivate) *types.T } // FoldFunction evaluates a function expression with constant inputs. It returns -// a constant expression as long as the function is contained in the -// FoldFunctionAllowlist, and the evaluation causes no error. Otherwise, it +// a constant expression as long as the evaluation causes no error. Otherwise, it // returns ok=false. func (c *CustomFuncs) FoldFunction( args memo.ScalarListExpr, private *memo.FunctionPrivate, @@ -690,7 +692,20 @@ func (c *CustomFuncs) FoldFunction( for i := range exprs { exprs[i] = memo.ExtractConstDatum(args[i]) } - funcRef := tree.WrapFunction(private.Name) + + var funcRef tree.ResolvableFunctionReference + if c.f.evalCtx != nil && c.f.catalog != nil { // Some tests leave those unset. + unresolved := tree.MakeUnresolvedName(private.Name) + def, err := c.f.catalog.ResolveFunction( + context.Background(), &unresolved, + &c.f.evalCtx.SessionData().SearchPath) + if err != nil { + panic(errors.AssertionFailedf("function %s() not defined", redact.Safe(private.Name))) + } + funcRef = tree.ResolvableFunctionReference{FunctionReference: def} + } else { + funcRef = tree.WrapFunction(private.Name) + } fn := tree.NewTypedFuncExpr( funcRef, 0, /* aggQualifier */ diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index e039e5e9bcbb..ab6a2e5cbdfa 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -95,7 +95,6 @@ func (p *planner) RunParams(ctx context.Context) runParams { // that gets passed back due to this inversion of roles. type PlanHookState interface { resolver.SchemaResolver - SpanConstrainer RunParams(ctx context.Context) runParams SemaCtx() *tree.SemaContext ExtendedEvalContext() *extendedEvalContext diff --git a/pkg/sql/sem/tree/function_name.go b/pkg/sql/sem/tree/function_name.go index 572e69c62c3b..d95ef511d78b 100644 --- a/pkg/sql/sem/tree/function_name.go +++ b/pkg/sql/sem/tree/function_name.go @@ -118,23 +118,6 @@ func (ref *ResolvableFunctionReference) Resolve( } } -// CustomBuiltinFunctionWrapper in an interface providing custom WrapFunction -// functionality. This is hack only being used by CDC to inject CDC custom -// builtin functions. It's not recommended to implement this interface for more -// purpose and this interface could be deleted. -// -// TODO(Chengxiong): consider getting rid of this hack entirely and use function -// resolver instead. Previously, CDC utilized search path as a interface hack to -// do the same thing. This interface makes the concept not relevant to search -// path anymore and also makes the purpose more specific on "Builtin" functions. -// However, it's ideal to get rid of this hack and use function resolver -// instead. One issue need to be addressed is that "WrapFunction" always look at -// builtin functions. So, the FunctionReferenceResolver interface might need to -// be extended to have a specific path for builtin functions. -type CustomBuiltinFunctionWrapper interface { - WrapFunction(name string) (*ResolvedFunctionDefinition, error) -} - // WrapFunction creates a new ResolvableFunctionReference holding a pre-resolved // function from a built-in function name. Helper for grammar rules and // execbuilder.