From f5c3d6eebda7399ad86c742245327219be6c9082 Mon Sep 17 00:00:00 2001 From: Rebecca Taft Date: Mon, 16 Nov 2020 19:47:18 -0600 Subject: [PATCH] opt: add insertion checks for unique constraints This commit adds checks for unique constraints when planning insertions in the optimizer. This does not yet impact anything outside of the optimizer tests, since UNIQUE WITHOUT INDEX is still not supported outside of the optimizer test catalog. Informs #41535 Release note: None --- pkg/sql/opt/exec/execbuilder/mutation.go | 39 +- pkg/sql/opt/memo/expr_format.go | 18 +- pkg/sql/opt/memo/interner.go | 18 + pkg/sql/opt/norm/prune_cols_funcs.go | 10 +- pkg/sql/opt/norm/rules/prune_cols.opt | 28 +- pkg/sql/opt/norm/testdata/rules/prune_cols | 3 + pkg/sql/opt/ops/mutation.opt | 41 +- pkg/sql/opt/optbuilder/BUILD.bazel | 1 + pkg/sql/opt/optbuilder/delete.go | 4 +- pkg/sql/opt/optbuilder/insert.go | 10 +- pkg/sql/opt/optbuilder/mutation_builder.go | 70 ++- pkg/sql/opt/optbuilder/mutation_builder_fk.go | 87 +-- .../opt/optbuilder/mutation_builder_unique.go | 196 ++++++ .../optbuilder/testdata/unique-checks-insert | 563 ++++++++++++++++++ pkg/sql/opt/optbuilder/update.go | 4 +- pkg/sql/opt/xform/testdata/rules/groupby | 134 +++-- pkg/sql/sqltelemetry/planning.go | 4 + 17 files changed, 1054 insertions(+), 176 deletions(-) create mode 100644 pkg/sql/opt/optbuilder/mutation_builder_unique.go create mode 100644 pkg/sql/opt/optbuilder/testdata/unique-checks-insert diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index cf0a2a722d74..4caa9c4c8a4a 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -45,8 +45,8 @@ func (b *Builder) buildMutationInput( } if p.WithID != 0 { - // The input might have extra columns that are used only by FK checks; make - // sure we don't project them away. + // The input might have extra columns that are used only by FK or unique + // checks; make sure we don't project them away. cols := inputExpr.Relational().OutputCols.Copy() for _, c := range colList { cols.Remove(c) @@ -101,7 +101,8 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (execPlan, error) { insertOrds, returnOrds, checkOrds, - b.allowAutoCommit && len(ins.Checks) == 0 && len(ins.FKCascades) == 0, + b.allowAutoCommit && len(ins.UniqueChecks) == 0 && + len(ins.FKChecks) == 0 && len(ins.FKCascades) == 0, ) if err != nil { return execPlan{}, err @@ -112,7 +113,9 @@ func (b *Builder) buildInsert(ins *memo.InsertExpr) (execPlan, error) { ep.outputCols = mutationOutputColMap(ins) } - if err := b.buildFKChecks(ins.Checks); err != nil { + // TODO(rytaft): build unique checks. + + if err := b.buildFKChecks(ins.FKChecks); err != nil { return execPlan{}, err } @@ -148,9 +151,9 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b // - there are no self-referencing foreign keys; // - all FK checks can be performed using direct lookups into unique indexes. - fkChecks := make([]exec.InsertFastPathFKCheck, len(ins.Checks)) - for i := range ins.Checks { - c := &ins.Checks[i] + fkChecks := make([]exec.InsertFastPathFKCheck, len(ins.FKChecks)) + for i := range ins.FKChecks { + c := &ins.FKChecks[i] if md.Table(c.ReferencedTable).ID() == md.Table(ins.Table).ID() { // Self-referencing FK. return execPlan{}, false, nil @@ -325,13 +328,16 @@ func (b *Builder) buildUpdate(upd *memo.UpdateExpr) (execPlan, error) { returnColOrds, checkOrds, passthroughCols, - b.allowAutoCommit && len(upd.Checks) == 0 && len(upd.FKCascades) == 0, + b.allowAutoCommit && len(upd.UniqueChecks) == 0 && + len(upd.FKChecks) == 0 && len(upd.FKCascades) == 0, ) if err != nil { return execPlan{}, err } - if err := b.buildFKChecks(upd.Checks); err != nil { + // TODO(rytaft): build unique checks. + + if err := b.buildFKChecks(upd.FKChecks); err != nil { return execPlan{}, err } @@ -406,13 +412,16 @@ func (b *Builder) buildUpsert(ups *memo.UpsertExpr) (execPlan, error) { updateColOrds, returnColOrds, checkOrds, - b.allowAutoCommit && len(ups.Checks) == 0 && len(ups.FKCascades) == 0, + b.allowAutoCommit && len(ups.UniqueChecks) == 0 && + len(ups.FKChecks) == 0 && len(ups.FKCascades) == 0, ) if err != nil { return execPlan{}, err } - if err := b.buildFKChecks(ups.Checks); err != nil { + // TODO(rytaft): build unique checks. + + if err := b.buildFKChecks(ups.FKChecks); err != nil { return execPlan{}, err } @@ -460,13 +469,13 @@ func (b *Builder) buildDelete(del *memo.DeleteExpr) (execPlan, error) { tab, fetchColOrds, returnColOrds, - b.allowAutoCommit && len(del.Checks) == 0 && len(del.FKCascades) == 0, + b.allowAutoCommit && len(del.FKChecks) == 0 && len(del.FKCascades) == 0, ) if err != nil { return execPlan{}, err } - if err := b.buildFKChecks(del.Checks); err != nil { + if err := b.buildFKChecks(del.FKChecks); err != nil { return execPlan{}, err } @@ -528,7 +537,7 @@ func (b *Builder) tryBuildDeleteRange(del *memo.DeleteExpr) (_ execPlan, ok bool if err != nil { return execPlan{}, false, err } - if err := b.buildFKChecks(del.Checks); err != nil { + if err := b.buildFKChecks(del.FKChecks); err != nil { return execPlan{}, false, err } if err := b.buildFKCascades(del.WithID, del.FKCascades); err != nil { @@ -677,7 +686,7 @@ func (b *Builder) buildDeleteRange( autoCommit = true } } - if len(del.Checks) > 0 || len(del.FKCascades) > 0 { + if len(del.FKChecks) > 0 || len(del.FKCascades) > 0 { // Do not allow autocommit if we have checks or cascades. This does not // apply for the interleaved case, where we decided that the delete // range takes care of all the FKs as well. diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index deb1bb39ae16..a746668575ad 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -785,7 +785,7 @@ func (f *ExprFmtCtx) formatScalarWithLabel( f.Buffer.WriteString(": ") } switch scalar.Op() { - case opt.ProjectionsOp, opt.AggregationsOp, opt.FKChecksOp, opt.KVOptionsOp: + case opt.ProjectionsOp, opt.AggregationsOp, opt.UniqueChecksOp, opt.FKChecksOp, opt.KVOptionsOp: // Omit empty lists (except filters). if scalar.ChildCount() == 0 { return @@ -912,7 +912,8 @@ func (f *ExprFmtCtx) formatScalarWithLabel( func (f *ExprFmtCtx) scalarPropsStrings(scalar opt.ScalarExpr) []string { typ := scalar.DataType() if typ == nil { - if scalar.Op() == opt.FKChecksItemOp || scalar.Op() == opt.KVOptionsItemOp { + if scalar.Op() == opt.UniqueChecksItemOp || scalar.Op() == opt.FKChecksItemOp || + scalar.Op() == opt.KVOptionsItemOp { // These are not true scalars and have no properties. return nil } @@ -999,6 +1000,19 @@ func (f *ExprFmtCtx) formatScalarPrivate(scalar opt.ScalarExpr) { case *KVOptionsItem: fmt.Fprintf(f.Buffer, " %s", t.Key) + case *UniqueChecksItem: + tab := f.Memo.metadata.TableMeta(t.Table) + constraint := tab.Table.Unique(t.CheckOrdinal) + fmt.Fprintf(f.Buffer, ": %s(", tab.Alias.ObjectName) + for i := 0; i < constraint.ColumnCount(); i++ { + if i > 0 { + f.Buffer.WriteByte(',') + } + col := tab.Table.Column(constraint.ColumnOrdinal(tab.Table, i)) + f.Buffer.WriteString(string(col.ColName())) + } + f.Buffer.WriteByte(')') + case *FKChecksItem: origin := f.Memo.metadata.TableMeta(t.OriginTable) referenced := f.Memo.metadata.TableMeta(t.ReferencedTable) diff --git a/pkg/sql/opt/memo/interner.go b/pkg/sql/opt/memo/interner.go index 7f3110d1a926..533ba9e3403e 100644 --- a/pkg/sql/opt/memo/interner.go +++ b/pkg/sql/opt/memo/interner.go @@ -645,6 +645,12 @@ func (h *hasher) HashFKChecksExpr(val FKChecksExpr) { } } +func (h *hasher) HashUniqueChecksExpr(val UniqueChecksExpr) { + for i := range val { + h.HashRelExpr(val[i].Check) + } +} + func (h *hasher) HashKVOptionsExpr(val KVOptionsExpr) { for i := range val { h.HashString(val[i].Key) @@ -1050,6 +1056,18 @@ func (h *hasher) IsFKChecksExprEqual(l, r FKChecksExpr) bool { return true } +func (h *hasher) IsUniqueChecksExprEqual(l, r UniqueChecksExpr) bool { + if len(l) != len(r) { + return false + } + for i := range l { + if l[i].Check != r[i].Check { + return false + } + } + return true +} + func (h *hasher) IsKVOptionsExprEqual(l, r KVOptionsExpr) bool { if len(l) != len(r) { return false diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index 25f6e739b2d7..2605ae22730e 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -43,7 +43,7 @@ func (c *CustomFuncs) NeededExplainCols(private *memo.ExplainPrivate) opt.ColSet // referenced by it. Other rules filter the FetchCols, CheckCols, etc. and can // in turn trigger the PruneMutationInputCols rule. func (c *CustomFuncs) NeededMutationCols( - private *memo.MutationPrivate, checks memo.FKChecksExpr, + private *memo.MutationPrivate, uniqueChecks memo.UniqueChecksExpr, fkChecks memo.FKChecksExpr, ) opt.ColSet { var cols opt.ColSet @@ -69,8 +69,12 @@ func (c *CustomFuncs) NeededMutationCols( } if private.WithID != 0 { - for i := range checks { - withUses := memo.WithUses(checks[i].Check) + for i := range uniqueChecks { + withUses := memo.WithUses(uniqueChecks[i].Check) + cols.UnionWith(withUses[private.WithID].UsedCols) + } + for i := range fkChecks { + withUses := memo.WithUses(fkChecks[i].Check) cols.UnionWith(withUses[private.WithID].UsedCols) } } diff --git a/pkg/sql/opt/norm/rules/prune_cols.opt b/pkg/sql/opt/norm/rules/prune_cols.opt index b932601647ea..d920613a4bf2 100644 --- a/pkg/sql/opt/norm/rules/prune_cols.opt +++ b/pkg/sql/opt/norm/rules/prune_cols.opt @@ -415,7 +415,8 @@ [PruneMutationFetchCols, Normalize] (Update | Upsert | Delete $input:* - $checks:* + $uniqueChecks:* + $fkChecks:* $mutationPrivate:* & (CanPruneMutationFetchCols $mutationPrivate @@ -428,7 +429,8 @@ => ((OpName) $input - $checks + $uniqueChecks + $fkChecks (PruneMutationFetchCols $mutationPrivate $needed) ) @@ -437,15 +439,25 @@ [PruneMutationInputCols, Normalize] (Insert | Update | Upsert | Delete $input:* - $checks:* + $uniqueChecks:* + $fkChecks:* $mutationPrivate:* & (CanPruneCols $input - $needed:(NeededMutationCols $mutationPrivate $checks) + $needed:(NeededMutationCols + $mutationPrivate + $uniqueChecks + $fkChecks + ) ) ) => -((OpName) (PruneCols $input $needed) $checks $mutationPrivate) +((OpName) + (PruneCols $input $needed) + $uniqueChecks + $fkChecks + $mutationPrivate +) # PruneReturningCols removes columns from the mutation operator's ReturnCols # set if they are not used in the RETURNING clause of the mutation. @@ -457,7 +469,8 @@ (Project $input:(Insert | Update | Upsert | Delete $innerInput:* - $checks:* + $uniqueChecks:* + $fkChecks:* $mutationPrivate:* ) $projections:* @@ -475,7 +488,8 @@ (Project ((OpName $input) $innerInput - $checks + $uniqueChecks + $fkChecks (PruneMutationReturnCols $mutationPrivate $needed) ) $projections diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index 47e18a8cc053..379b6e3bcc33 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -2585,6 +2585,9 @@ upsert checks ├── upsert_b:16 > 10 [as=check2:18, outer=(16)] └── column2:7 > upsert_b:16 [as=check3:19, outer=(7,16)] +# TODO(rytaft): test that columns needed for unique checks are not pruned +# from updates. + # ------------------------------------------------------------------------------ # PruneMutationReturnCols # ------------------------------------------------------------------------------ diff --git a/pkg/sql/opt/ops/mutation.opt b/pkg/sql/opt/ops/mutation.opt index 2f1843990f0a..f81319c9e56c 100644 --- a/pkg/sql/opt/ops/mutation.opt +++ b/pkg/sql/opt/ops/mutation.opt @@ -20,7 +20,8 @@ [Relational, Mutation, WithBinding] define Insert { Input RelExpr - Checks FKChecksExpr + UniqueChecks UniqueChecksExpr + FKChecks FKChecksExpr _ MutationPrivate } @@ -182,7 +183,8 @@ define MutationPrivate { [Relational, Mutation, WithBinding] define Update { Input RelExpr - Checks FKChecksExpr + UniqueChecks UniqueChecksExpr + FKChecks FKChecksExpr _ MutationPrivate } @@ -205,7 +207,8 @@ define Update { [Relational, Mutation, WithBinding] define Upsert { Input RelExpr - Checks FKChecksExpr + UniqueChecks UniqueChecksExpr + FKChecks FKChecksExpr _ MutationPrivate } @@ -217,7 +220,8 @@ define Upsert { [Relational, Mutation, WithBinding] define Delete { Input RelExpr - Checks FKChecksExpr + UniqueChecks UniqueChecksExpr + FKChecks FKChecksExpr _ MutationPrivate } @@ -257,3 +261,32 @@ define FKChecksItemPrivate { # OpName is the name that should be used for this check in error messages. OpName string } + +# UniqueChecks is a list of uniqueness check queries, to be run after the main +# query. +[Scalar, List] +define UniqueChecks { +} + +# UniqueChecksItem is a unique check query, to be run after the main query. +# An execution error will be generated if the query returns any results. +[Scalar, ListItem] +define UniqueChecksItem { + Check RelExpr + _ UniqueChecksItemPrivate +} + +[Private] +define UniqueChecksItemPrivate { + Table TableID + + # This is the ordinal of the check in the table's unique constraints. + CheckOrdinal int + + # KeyCols are the columns in the Check query that form the value tuple shown + # in the error message. + KeyCols ColList + + # OpName is the name that should be used for this check in error messages. + OpName string +} diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index 612d9c469f30..bc2e613fa6a5 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "misc_statements.go", "mutation_builder.go", "mutation_builder_fk.go", + "mutation_builder_unique.go", "opaque.go", "orderby.go", "partial_index.go", diff --git a/pkg/sql/opt/optbuilder/delete.go b/pkg/sql/opt/optbuilder/delete.go index 8b86009d108f..6b28ed589374 100644 --- a/pkg/sql/opt/optbuilder/delete.go +++ b/pkg/sql/opt/optbuilder/delete.go @@ -77,7 +77,9 @@ func (mb *mutationBuilder) buildDelete(returning tree.ReturningExprs) { mb.buildFKChecksAndCascadesForDelete() private := mb.makeMutationPrivate(returning != nil) - mb.outScope.expr = mb.b.factory.ConstructDelete(mb.outScope.expr, mb.checks, private) + mb.outScope.expr = mb.b.factory.ConstructDelete( + mb.outScope.expr, mb.uniqueChecks, mb.fkChecks, private, + ) mb.buildReturning(returning) } diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 6c1e33b44c70..d13a0e675d2e 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -652,10 +652,14 @@ func (mb *mutationBuilder) buildInsert(returning tree.ReturningExprs) { // Add any partial index put boolean columns to the input. mb.projectPartialIndexPutCols(preCheckScope) + mb.buildUniqueChecksForInsert() + mb.buildFKChecksForInsert() private := mb.makeMutationPrivate(returning != nil) - mb.outScope.expr = mb.b.factory.ConstructInsert(mb.outScope.expr, mb.checks, private) + mb.outScope.expr = mb.b.factory.ConstructInsert( + mb.outScope.expr, mb.uniqueChecks, mb.fkChecks, private, + ) mb.buildReturning(returning) } @@ -1089,7 +1093,9 @@ func (mb *mutationBuilder) buildUpsert(returning tree.ReturningExprs) { mb.buildFKChecksForUpsert() private := mb.makeMutationPrivate(returning != nil) - mb.outScope.expr = mb.b.factory.ConstructUpsert(mb.outScope.expr, mb.checks, private) + mb.outScope.expr = mb.b.factory.ConstructUpsert( + mb.outScope.expr, mb.uniqueChecks, mb.fkChecks, private, + ) mb.buildReturning(returning) } diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index f578ab239dab..22eb54a6c8d5 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -150,8 +150,11 @@ type mutationBuilder struct { // reuse. parsedIndexExprs []tree.Expr - // checks contains foreign key check queries; see buildFK* methods. - checks memo.FKChecksExpr + // uniqueChecks contains unique check queries; see buildUnique* methods. + uniqueChecks memo.UniqueChecksExpr + + // fkChecks contains foreign key check queries; see buildFK* methods. + fkChecks memo.FKChecksExpr // cascades contains foreign key check cascades; see buildFK* methods. cascades memo.FKCascades @@ -167,6 +170,9 @@ type mutationBuilder struct { // fkCheckHelper is used to prevent allocating the helper separately. fkCheckHelper fkCheckHelper + + // uniqueCheckHelper is used to prevent allocating the helper separately. + uniqueCheckHelper uniqueCheckHelper } func (mb *mutationBuilder) init(b *Builder, opName string, tab cat.Table, alias tree.TableName) { @@ -893,7 +899,7 @@ func (mb *mutationBuilder) makeMutationPrivate(needResults bool) *memo.MutationP } // If we didn't actually plan any checks or cascades, don't buffer the input. - if len(mb.checks) > 0 || len(mb.cascades) > 0 { + if len(mb.uniqueChecks) > 0 || len(mb.fkChecks) > 0 || len(mb.cascades) > 0 { private.WithID = mb.withID } @@ -1144,3 +1150,61 @@ func partialIndexCount(tab cat.Table) int { } return count } + +type checkInputScanType uint8 + +const ( + checkInputScanNewVals checkInputScanType = iota + checkInputScanFetchedVals +) + +// makeCheckInputScan constructs a WithScan that iterates over the input to the +// mutation operator. Used in expressions that generate rows for checking for FK +// and uniqueness violations. +// +// The WithScan expression will scan either the new values or the fetched values +// for the given table ordinals (which correspond to FK or unique columns). +// +// Returns the output columns from the WithScan, which map 1-to-1 to +// tabOrdinals. Also returns the subset of these columns that can be assumed +// to be not null (either because they are not null in the mutation input or +// because they are non-nullable table columns). +// +func (mb *mutationBuilder) makeCheckInputScan( + typ checkInputScanType, tabOrdinals []int, +) (scan memo.RelExpr, outCols opt.ColList, notNullOutCols opt.ColSet) { + // inputCols are the column IDs from the mutation input that we are scanning. + inputCols := make(opt.ColList, len(tabOrdinals)) + // outCols will store the newly synthesized output columns for WithScan. + outCols = make(opt.ColList, len(inputCols)) + for i, tabOrd := range tabOrdinals { + if typ == checkInputScanNewVals { + inputCols[i] = mb.mapToReturnColID(tabOrd) + } else { + inputCols[i] = mb.fetchColIDs[tabOrd] + } + if inputCols[i] == 0 { + panic(errors.AssertionFailedf("no value for FK column (tabOrd=%d)", tabOrd)) + } + + // Synthesize new column. + c := mb.b.factory.Metadata().ColumnMeta(inputCols[i]) + outCols[i] = mb.md.AddColumn(c.Alias, c.Type) + + // If a table column is not nullable, NULLs cannot be inserted (the + // mutation will fail). So for the purposes of checks, we can treat + // these columns as not null. + if mb.outScope.expr.Relational().NotNullCols.Contains(inputCols[i]) || + !mb.tab.Column(tabOrd).IsNullable() { + notNullOutCols.Add(outCols[i]) + } + } + + scan = mb.b.factory.ConstructWithScan(&memo.WithScanPrivate{ + With: mb.withID, + InCols: inputCols, + OutCols: outCols, + ID: mb.b.factory.Metadata().NextUniqueID(), + }) + return scan, outCols, notNullOutCols +} diff --git a/pkg/sql/opt/optbuilder/mutation_builder_fk.go b/pkg/sql/opt/optbuilder/mutation_builder_fk.go index f735414b0e90..213e30fce5df 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder_fk.go +++ b/pkg/sql/opt/optbuilder/mutation_builder_fk.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/errors" ) -// This file contains methods that populate mutationBuilder.checks and cascades. +// This file contains methods that populate mutationBuilder.fkChecks and cascades. // // -- Checks -- // @@ -80,7 +80,7 @@ func (mb *mutationBuilder) buildFKChecksForInsert() { h := &mb.fkCheckHelper for i, n := 0, mb.tab.OutboundForeignKeyCount(); i < n; i++ { if h.initWithOutboundFK(mb, i) { - mb.checks = append(mb.checks, h.buildInsertionCheck()) + mb.fkChecks = append(mb.fkChecks, h.buildInsertionCheck()) } } telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter) @@ -171,8 +171,8 @@ func (mb *mutationBuilder) buildFKChecksAndCascadesForDelete() { } mb.ensureWithID() - fkInput, withScanCols, _ := h.makeFKInputScan(fkInputScanFetchedVals) - mb.checks = append(mb.checks, h.buildDeletionCheck(fkInput, withScanCols)) + fkInput, withScanCols, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals) + mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(fkInput, withScanCols)) } telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter) } @@ -269,7 +269,7 @@ func (mb *mutationBuilder) buildFKChecksForUpdate() { // Verify that at least one FK column is actually updated. if mb.outboundFKColsUpdated(i) { if h.initWithOutboundFK(mb, i) { - mb.checks = append(mb.checks, h.buildInsertionCheck()) + mb.fkChecks = append(mb.fkChecks, h.buildInsertionCheck()) } } } @@ -336,8 +336,8 @@ func (mb *mutationBuilder) buildFKChecksForUpdate() { // performance either: we would be incurring extra cost (more complicated // expressions, scanning the input buffer twice) for a rare case. - oldRows, colsForOldRow, _ := h.makeFKInputScan(fkInputScanFetchedVals) - newRows, colsForNewRow, _ := h.makeFKInputScan(fkInputScanNewVals) + oldRows, colsForOldRow, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals) + newRows, colsForNewRow, _ := mb.makeCheckInputScan(checkInputScanNewVals, h.tabOrdinals) // The rows that no longer exist are the ones that were "deleted" by virtue // of being updated _from_, minus the ones that were "added" by virtue of @@ -352,7 +352,7 @@ func (mb *mutationBuilder) buildFKChecksForUpdate() { }, ) - mb.checks = append(mb.checks, h.buildDeletionCheck(deletedRows, colsForOldRow)) + mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(deletedRows, colsForOldRow)) } telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter) } @@ -387,7 +387,7 @@ func (mb *mutationBuilder) buildFKChecksForUpsert() { h := &mb.fkCheckHelper for i := 0; i < numOutbound; i++ { if h.initWithOutboundFK(mb, i) { - mb.checks = append(mb.checks, h.buildInsertionCheck()) + mb.fkChecks = append(mb.fkChecks, h.buildInsertionCheck()) } } @@ -439,8 +439,8 @@ func (mb *mutationBuilder) buildFKChecksForUpsert() { // insertions (using a "canaryCol IS NOT NULL" condition). But the rows we // would filter out have all-null fetched values anyway and will never match // in the semi join. - oldRows, colsForOldRow, _ := h.makeFKInputScan(fkInputScanFetchedVals) - newRows, colsForNewRow, _ := h.makeFKInputScan(fkInputScanNewVals) + oldRows, colsForOldRow, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals) + newRows, colsForNewRow, _ := mb.makeCheckInputScan(checkInputScanNewVals, h.tabOrdinals) // The rows that no longer exist are the ones that were "deleted" by virtue // of being updated _from_, minus the ones that were "added" by virtue of @@ -454,7 +454,7 @@ func (mb *mutationBuilder) buildFKChecksForUpsert() { OutCols: colsForOldRow, }, ) - mb.checks = append(mb.checks, h.buildDeletionCheck(deletedRows, colsForOldRow)) + mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(deletedRows, colsForOldRow)) } telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter) } @@ -612,65 +612,6 @@ func resolveTable(ctx context.Context, catalog cat.Catalog, id cat.StableID) cat return ref.(cat.Table) } -type fkInputScanType uint8 - -const ( - fkInputScanNewVals fkInputScanType = iota - fkInputScanFetchedVals -) - -// makeFKInputScan constructs a WithScan that iterates over the input to the -// mutation operator. Used in expressions that generate rows for checking for FK -// violations. -// -// The WithScan expression will scan either the new values or the fetched values -// for the given table ordinals (which correspond to FK columns). -// -// Returns the output columns from the WithScan, which map 1-to-1 to -// h.tabOrdinals. Also returns the subset of these columns that can be assumed -// to be not null (either because they are not null in the mutation input or -// because they are non-nullable table columns). -// -func (h *fkCheckHelper) makeFKInputScan( - typ fkInputScanType, -) (scan memo.RelExpr, outCols opt.ColList, notNullOutCols opt.ColSet) { - mb := h.mb - // inputCols are the column IDs from the mutation input that we are scanning. - inputCols := make(opt.ColList, len(h.tabOrdinals)) - // outCols will store the newly synthesized output columns for WithScan. - outCols = make(opt.ColList, len(inputCols)) - for i, tabOrd := range h.tabOrdinals { - if typ == fkInputScanNewVals { - inputCols[i] = mb.mapToReturnColID(tabOrd) - } else { - inputCols[i] = mb.fetchColIDs[tabOrd] - } - if inputCols[i] == 0 { - panic(errors.AssertionFailedf("no value for FK column (tabOrd=%d)", tabOrd)) - } - - // Synthesize new column. - c := mb.b.factory.Metadata().ColumnMeta(inputCols[i]) - outCols[i] = mb.md.AddColumn(c.Alias, c.Type) - - // If a table column is not nullable, NULLs cannot be inserted (the - // mutation will fail). So for the purposes of FK checks, we can treat - // these columns as not null. - if mb.outScope.expr.Relational().NotNullCols.Contains(inputCols[i]) || - !mb.tab.Column(tabOrd).IsNullable() { - notNullOutCols.Add(outCols[i]) - } - } - - scan = mb.b.factory.ConstructWithScan(&memo.WithScanPrivate{ - With: mb.withID, - InCols: inputCols, - OutCols: outCols, - ID: mb.b.factory.Metadata().NextUniqueID(), - }) - return scan, outCols, notNullOutCols -} - // buildOtherTableScan builds a Scan of the "other" table. func (h *fkCheckHelper) buildOtherTableScan() (outScope *scope, tabMeta *opt.TableMeta) { otherTabMeta := h.mb.b.addTable(h.otherTab, tree.NewUnqualifiedTableName(h.otherTab.Name())) @@ -693,7 +634,9 @@ func (h *fkCheckHelper) allocOrdinals(numCols int) { // The input to the insertion check will be produced from the input to the // mutation operator. func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem { - fkInput, withScanCols, notNullWithScanCols := h.makeFKInputScan(fkInputScanNewVals) + fkInput, withScanCols, notNullWithScanCols := h.mb.makeCheckInputScan( + checkInputScanNewVals, h.tabOrdinals, + ) numCols := len(withScanCols) f := h.mb.b.factory diff --git a/pkg/sql/opt/optbuilder/mutation_builder_unique.go b/pkg/sql/opt/optbuilder/mutation_builder_unique.go new file mode 100644 index 000000000000..a23e6bafa5e5 --- /dev/null +++ b/pkg/sql/opt/optbuilder/mutation_builder_unique.go @@ -0,0 +1,196 @@ +// Copyright 2020 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 optbuilder + +import ( + "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/util" +) + +// buildUniqueChecksForInsert builds uniqueness check queries for an insert. +func (mb *mutationBuilder) buildUniqueChecksForInsert() { + uniqueCount := mb.tab.UniqueCount() + if uniqueCount == 0 { + // No relevant unique checks. + return + } + + // We only need to build unique checks if there is at least one unique + // constraint without an index. + needChecks := false + i := 0 + for ; i < uniqueCount; i++ { + if mb.tab.Unique(i).WithoutIndex() { + needChecks = true + break + } + } + if !needChecks { + return + } + + mb.ensureWithID() + h := &mb.uniqueCheckHelper + + // i is already set to the index of the first uniqueness check without an + // index, so start iterating from there. + for ; i < uniqueCount; i++ { + // If this constraint is already enforced by an index we don't need to plan + // a check. + if mb.tab.Unique(i).WithoutIndex() && h.init(mb, i) { + mb.uniqueChecks = append(mb.uniqueChecks, h.buildInsertionCheck()) + } + } + telemetry.Inc(sqltelemetry.UniqueChecksUseCounter) +} + +// uniqueCheckHelper is a type associated with a single unique constraint and +// is used to build the "leaves" of a unique check expression, namely the +// WithScan of the mutation input and the Scan of the table. +type uniqueCheckHelper struct { + mb *mutationBuilder + + unique cat.UniqueConstraint + uniqueOrdinal int + + // uniqueOrdinals are the table ordinals of the unique columns in the table + // that is being mutated. They correspond 1-to-1 to the columns in the + // UniqueConstraint. + uniqueOrdinals []int + + // uniqueAndPrimaryKeyOrdinals includes all the ordinals from uniqueOrdinals, + // plus the ordinals from any primary key columns that are not already + // included in uniqueOrdinals. + uniqueAndPrimaryKeyOrdinals []int +} + +// init initializes the helper with a unique constraint. +// +// Returns false if the constraint should be ignored (e.g. because the new +// values for the unique columns are known to be always NULL). +func (h *uniqueCheckHelper) init(mb *mutationBuilder, uniqueOrdinal int) bool { + *h = uniqueCheckHelper{ + mb: mb, + unique: mb.tab.Unique(uniqueOrdinal), + uniqueOrdinal: uniqueOrdinal, + } + + uniqueCount := h.unique.ColumnCount() + + var uniqueOrds util.FastIntSet + for i := 0; i < uniqueCount; i++ { + uniqueOrds.Add(h.unique.ColumnOrdinal(mb.tab, i)) + } + + // Find the primary key columns that are not part of the unique constraint. + // If there aren't any, we don't need a check. + primaryOrds := getIndexLaxKeyOrdinals(mb.tab.Index(cat.PrimaryIndex)) + primaryOrds.DifferenceWith(uniqueOrds) + if primaryOrds.Empty() { + // The primary key columns are a subset of the unique columns; unique check + // not needed. + return false + } + + h.uniqueAndPrimaryKeyOrdinals = append(uniqueOrds.Ordered(), primaryOrds.Ordered()...) + h.uniqueOrdinals = h.uniqueAndPrimaryKeyOrdinals[:uniqueCount] + + // Check if we are setting NULL values for the unique columns, like when this + // mutation is the result of a SET NULL cascade action. + numNullCols := 0 + for _, tabOrd := range h.uniqueOrdinals { + colID := mb.mapToReturnColID(tabOrd) + if memo.OutputColumnIsAlwaysNull(mb.outScope.expr, colID) { + numNullCols++ + } + } + + // If at least one unique column is getting a NULL value, unique check not + // needed. + return numNullCols == 0 +} + +// buildInsertionCheck creates a unique check for rows which are added to a +// table. The input to the insertion check will be produced from the input to +// the mutation operator. +func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem { + checkInput, withScanCols, _ := h.mb.makeCheckInputScan( + checkInputScanNewVals, h.uniqueAndPrimaryKeyOrdinals, + ) + + numCols := len(withScanCols) + f := h.mb.b.factory + + // Build a self semi-join, with the new values on the left and the + // existing values on the right. + + scanScope, _ := h.buildTableScan() + + // Build the join filters: + // (new_a = existing_a) AND (new_b = existing_b) AND ... + // + // Set the capacity to len(h.uniqueOrdinals)+1 since we'll have an equality + // condition for each column in the unique constraint, plus one additional + // condition to prevent rows from matching themselves (see below). + semiJoinFilters := make(memo.FiltersExpr, 0, len(h.uniqueOrdinals)+1) + for i := 0; i < len(h.uniqueOrdinals); i++ { + semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem( + f.ConstructEq( + f.ConstructVariable(withScanCols[i]), + f.ConstructVariable(scanScope.cols[i].id), + ), + )) + } + + // We need to prevent rows from matching themselves in the semi join. We can + // do this by adding another filter that uses the primary keys to check if + // two rows are identical: + // (new_pk1 != existing_pk1) OR (new_pk2 != existing_pk2) OR ... + var pkFilter opt.ScalarExpr + for i := len(h.uniqueOrdinals); i < numCols; i++ { + pkFilterLocal := f.ConstructNe( + f.ConstructVariable(withScanCols[i]), + f.ConstructVariable(scanScope.cols[i].id), + ) + if pkFilter == nil { + pkFilter = pkFilterLocal + } else { + pkFilter = f.ConstructOr(pkFilter, pkFilterLocal) + } + } + semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(pkFilter)) + + semiJoin := f.ConstructSemiJoin(checkInput, scanScope.expr, semiJoinFilters, &memo.JoinPrivate{}) + + return f.ConstructUniqueChecksItem(semiJoin, &memo.UniqueChecksItemPrivate{ + Table: h.mb.tabID, + CheckOrdinal: h.uniqueOrdinal, + KeyCols: withScanCols, + OpName: h.mb.opName, + }) +} + +// buildTableScan builds a Scan of the table. +func (h *uniqueCheckHelper) buildTableScan() (outScope *scope, tabMeta *opt.TableMeta) { + tabMeta = h.mb.b.addTable(h.mb.tab, tree.NewUnqualifiedTableName(h.mb.tab.Name())) + return h.mb.b.buildScan( + tabMeta, + h.uniqueAndPrimaryKeyOrdinals, + nil, /* indexFlags */ + noRowLocking, + h.mb.b.allocScope(), + ), tabMeta +} diff --git a/pkg/sql/opt/optbuilder/testdata/unique-checks-insert b/pkg/sql/opt/optbuilder/testdata/unique-checks-insert new file mode 100644 index 000000000000..6901bed26ca5 --- /dev/null +++ b/pkg/sql/opt/optbuilder/testdata/unique-checks-insert @@ -0,0 +1,563 @@ +exec-ddl +CREATE TABLE uniq ( + k INT PRIMARY KEY, + v INT UNIQUE, + w INT UNIQUE WITHOUT INDEX, + x INT, + y INT, + UNIQUE WITHOUT INDEX (x, y) +) +---- + +# None of the inserted values have nulls. +build +INSERT INTO uniq VALUES (1, 1, 1, 1, 1), (2, 2, 2, 2, 2) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + ├── input binding: &1 + ├── values + │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column5:11!null + │ ├── (1, 1, 1, 1, 1) + │ └── (2, 2, 2, 2, 2) + └── unique-checks + ├── unique-checks-item: uniq(w) + │ └── semi-join (hash) + │ ├── columns: column3:12!null column1:13!null + │ ├── with-scan &1 + │ │ ├── columns: column3:12!null column1:13!null + │ │ └── mapping: + │ │ ├── column3:9 => column3:12 + │ │ └── column1:7 => column1:13 + │ ├── scan uniq + │ │ └── columns: k:14!null w:16 + │ └── filters + │ ├── column3:12 = w:16 + │ └── column1:13 != k:14 + └── unique-checks-item: uniq(x,y) + └── semi-join (hash) + ├── columns: column4:20!null column5:21!null column1:22!null + ├── with-scan &1 + │ ├── columns: column4:20!null column5:21!null column1:22!null + │ └── mapping: + │ ├── column4:10 => column4:20 + │ ├── column5:11 => column5:21 + │ └── column1:7 => column1:22 + ├── scan uniq + │ └── columns: k:23!null x:26 y:27 + └── filters + ├── column4:20 = x:26 + ├── column5:21 = y:27 + └── column1:22 != k:23 + +# Some of the inserted values have nulls. +build +INSERT INTO uniq VALUES (1, 1, 1, 1, 1), (2, 2, 2, 2, 2), (3, NULL, NULL, NULL, 3) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + ├── input binding: &1 + ├── values + │ ├── columns: column1:7!null column2:8 column3:9 column4:10 column5:11!null + │ ├── (1, 1, 1, 1, 1) + │ ├── (2, 2, 2, 2, 2) + │ └── (3, NULL::INT8, NULL::INT8, NULL::INT8, 3) + └── unique-checks + ├── unique-checks-item: uniq(w) + │ └── semi-join (hash) + │ ├── columns: column3:12 column1:13!null + │ ├── with-scan &1 + │ │ ├── columns: column3:12 column1:13!null + │ │ └── mapping: + │ │ ├── column3:9 => column3:12 + │ │ └── column1:7 => column1:13 + │ ├── scan uniq + │ │ └── columns: k:14!null w:16 + │ └── filters + │ ├── column3:12 = w:16 + │ └── column1:13 != k:14 + └── unique-checks-item: uniq(x,y) + └── semi-join (hash) + ├── columns: column4:20 column5:21!null column1:22!null + ├── with-scan &1 + │ ├── columns: column4:20 column5:21!null column1:22!null + │ └── mapping: + │ ├── column4:10 => column4:20 + │ ├── column5:11 => column5:21 + │ └── column1:7 => column1:22 + ├── scan uniq + │ └── columns: k:23!null x:26 y:27 + └── filters + ├── column4:20 = x:26 + ├── column5:21 = y:27 + └── column1:22 != k:23 + +# No need to plan checks for w since it's aways null. +build +INSERT INTO uniq VALUES (1, 1, NULL, 1, 1), (2, 2, NULL, 2, 2) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + ├── input binding: &1 + ├── values + │ ├── columns: column1:7!null column2:8!null column3:9 column4:10!null column5:11!null + │ ├── (1, 1, NULL::INT8, 1, 1) + │ └── (2, 2, NULL::INT8, 2, 2) + └── unique-checks + └── unique-checks-item: uniq(x,y) + └── semi-join (hash) + ├── columns: column4:12!null column5:13!null column1:14!null + ├── with-scan &1 + │ ├── columns: column4:12!null column5:13!null column1:14!null + │ └── mapping: + │ ├── column4:10 => column4:12 + │ ├── column5:11 => column5:13 + │ └── column1:7 => column1:14 + ├── scan uniq + │ └── columns: k:15!null x:18 y:19 + └── filters + ├── column4:12 = x:18 + ├── column5:13 = y:19 + └── column1:14 != k:15 + +# No need to plan checks for x,y since x is aways null. +build +INSERT INTO uniq VALUES (1, 1, 1, NULL, 1), (2, 2, NULL, NULL, 2) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + ├── input binding: &1 + ├── values + │ ├── columns: column1:7!null column2:8!null column3:9 column4:10 column5:11!null + │ ├── (1, 1, 1, NULL::INT8, 1) + │ └── (2, 2, NULL::INT8, NULL::INT8, 2) + └── unique-checks + └── unique-checks-item: uniq(w) + └── semi-join (hash) + ├── columns: column3:12 column1:13!null + ├── with-scan &1 + │ ├── columns: column3:12 column1:13!null + │ └── mapping: + │ ├── column3:9 => column3:12 + │ └── column1:7 => column1:13 + ├── scan uniq + │ └── columns: k:14!null w:16 + └── filters + ├── column3:12 = w:16 + └── column1:13 != k:14 + +# No need to plan checks for x,y since y is aways null. +build +INSERT INTO uniq VALUES (1, 1, 1, 1, NULL), (2, 2, 2, 2, NULL) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + ├── input binding: &1 + ├── values + │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column5:11 + │ ├── (1, 1, 1, 1, NULL::INT8) + │ └── (2, 2, 2, 2, NULL::INT8) + └── unique-checks + └── unique-checks-item: uniq(w) + └── semi-join (hash) + ├── columns: column3:12!null column1:13!null + ├── with-scan &1 + │ ├── columns: column3:12!null column1:13!null + │ └── mapping: + │ ├── column3:9 => column3:12 + │ └── column1:7 => column1:13 + ├── scan uniq + │ └── columns: k:14!null w:16 + └── filters + ├── column3:12 = w:16 + └── column1:13 != k:14 + +# No need to plan any checks, since w, x and y are aways null. +build +INSERT INTO uniq VALUES (1, 1, NULL, NULL, NULL), (2, 2, NULL, NULL, NULL) +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => k:1 + │ ├── column2:8 => v:2 + │ ├── column3:9 => w:3 + │ ├── column4:10 => x:4 + │ └── column5:11 => y:5 + └── values + ├── columns: column1:7!null column2:8!null column3:9 column4:10 column5:11 + ├── (1, 1, NULL::INT8, NULL::INT8, NULL::INT8) + └── (2, 2, NULL::INT8, NULL::INT8, NULL::INT8) + +exec-ddl +CREATE TABLE other (k INT, v INT, w INT NOT NULL, x INT, y INT) +---- + +# Insert with non-constant input. +build +INSERT INTO uniq SELECT k, v, w, x, y FROM other +---- +insert uniq + ├── columns: + ├── insert-mapping: + │ ├── other.k:7 => uniq.k:1 + │ ├── other.v:8 => uniq.v:2 + │ ├── other.w:9 => uniq.w:3 + │ ├── other.x:10 => uniq.x:4 + │ └── other.y:11 => uniq.y:5 + ├── input binding: &1 + ├── project + │ ├── columns: other.k:7 other.v:8 other.w:9!null other.x:10 other.y:11 + │ └── scan other + │ └── columns: other.k:7 other.v:8 other.w:9!null other.x:10 other.y:11 rowid:12!null other.crdb_internal_mvcc_timestamp:13 + └── unique-checks + ├── unique-checks-item: uniq(w) + │ └── semi-join (hash) + │ ├── columns: w:14!null k:15 + │ ├── with-scan &1 + │ │ ├── columns: w:14!null k:15 + │ │ └── mapping: + │ │ ├── other.w:9 => w:14 + │ │ └── other.k:7 => k:15 + │ ├── scan uniq + │ │ └── columns: uniq.k:16!null uniq.w:18 + │ └── filters + │ ├── w:14 = uniq.w:18 + │ └── k:15 != uniq.k:16 + └── unique-checks-item: uniq(x,y) + └── semi-join (hash) + ├── columns: x:22 y:23 k:24 + ├── with-scan &1 + │ ├── columns: x:22 y:23 k:24 + │ └── mapping: + │ ├── other.x:10 => x:22 + │ ├── other.y:11 => y:23 + │ └── other.k:7 => k:24 + ├── scan uniq + │ └── columns: uniq.k:25!null uniq.x:28 uniq.y:29 + └── filters + ├── x:22 = uniq.x:28 + ├── y:23 = uniq.y:29 + └── k:24 != uniq.k:25 + +exec-ddl +CREATE TABLE uniq_overlaps_pk ( + a INT, + b INT, + c INT, + d INT, + PRIMARY KEY (a, b), + UNIQUE WITHOUT INDEX (b, c), + UNIQUE WITHOUT INDEX (a, b, d), + UNIQUE WITHOUT INDEX (a), + UNIQUE WITHOUT INDEX (c, d) +) +---- + +# Insert with constant input. +# Add inequality filters for the primary key columns that are not part of each +# unique constraint to prevent rows from matching themselves in the semi join. +build +INSERT INTO uniq_overlaps_pk VALUES (1, 1, 1, 1), (2, 2, 2, 2) +---- +insert uniq_overlaps_pk + ├── columns: + ├── insert-mapping: + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column3:8 => c:3 + │ └── column4:9 => d:4 + ├── input binding: &1 + ├── values + │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ ├── (1, 1, 1, 1) + │ └── (2, 2, 2, 2) + └── unique-checks + ├── unique-checks-item: uniq_overlaps_pk(b,c) + │ └── semi-join (hash) + │ ├── columns: column2:10!null column3:11!null column1:12!null + │ ├── with-scan &1 + │ │ ├── columns: column2:10!null column3:11!null column1:12!null + │ │ └── mapping: + │ │ ├── column2:7 => column2:10 + │ │ ├── column3:8 => column3:11 + │ │ └── column1:6 => column1:12 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: a:13!null b:14!null c:15 + │ └── filters + │ ├── column2:10 = b:14 + │ ├── column3:11 = c:15 + │ └── column1:12 != a:13 + ├── unique-checks-item: uniq_overlaps_pk(a) + │ └── semi-join (hash) + │ ├── columns: column1:18!null column2:19!null + │ ├── with-scan &1 + │ │ ├── columns: column1:18!null column2:19!null + │ │ └── mapping: + │ │ ├── column1:6 => column1:18 + │ │ └── column2:7 => column2:19 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: a:20!null b:21!null + │ └── filters + │ ├── column1:18 = a:20 + │ └── column2:19 != b:21 + └── unique-checks-item: uniq_overlaps_pk(c,d) + └── semi-join (hash) + ├── columns: column3:25!null column4:26!null column1:27!null column2:28!null + ├── with-scan &1 + │ ├── columns: column3:25!null column4:26!null column1:27!null column2:28!null + │ └── mapping: + │ ├── column3:8 => column3:25 + │ ├── column4:9 => column4:26 + │ ├── column1:6 => column1:27 + │ └── column2:7 => column2:28 + ├── scan uniq_overlaps_pk + │ └── columns: a:29!null b:30!null c:31 d:32 + └── filters + ├── column3:25 = c:31 + ├── column4:26 = d:32 + └── (column1:27 != a:29) OR (column2:28 != b:30) + +# Insert with non-constant input. +# Add inequality filters for the primary key columns that are not part of each +# unique constraint to prevent rows from matching themselves in the semi join. +build +INSERT INTO uniq_overlaps_pk SELECT k, v, x, y FROM other +---- +insert uniq_overlaps_pk + ├── columns: + ├── insert-mapping: + │ ├── other.k:6 => a:1 + │ ├── other.v:7 => b:2 + │ ├── other.x:9 => c:3 + │ └── other.y:10 => d:4 + ├── input binding: &1 + ├── project + │ ├── columns: other.k:6 other.v:7 other.x:9 other.y:10 + │ └── scan other + │ └── columns: other.k:6 other.v:7 w:8!null other.x:9 other.y:10 rowid:11!null other.crdb_internal_mvcc_timestamp:12 + └── unique-checks + ├── unique-checks-item: uniq_overlaps_pk(b,c) + │ └── semi-join (hash) + │ ├── columns: v:13 x:14 k:15 + │ ├── with-scan &1 + │ │ ├── columns: v:13 x:14 k:15 + │ │ └── mapping: + │ │ ├── other.v:7 => v:13 + │ │ ├── other.x:9 => x:14 + │ │ └── other.k:6 => k:15 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: a:16!null b:17!null c:18 + │ └── filters + │ ├── v:13 = b:17 + │ ├── x:14 = c:18 + │ └── k:15 != a:16 + ├── unique-checks-item: uniq_overlaps_pk(a) + │ └── semi-join (hash) + │ ├── columns: k:21 v:22 + │ ├── with-scan &1 + │ │ ├── columns: k:21 v:22 + │ │ └── mapping: + │ │ ├── other.k:6 => k:21 + │ │ └── other.v:7 => v:22 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: a:23!null b:24!null + │ └── filters + │ ├── k:21 = a:23 + │ └── v:22 != b:24 + └── unique-checks-item: uniq_overlaps_pk(c,d) + └── semi-join (hash) + ├── columns: x:28 y:29 k:30 v:31 + ├── with-scan &1 + │ ├── columns: x:28 y:29 k:30 v:31 + │ └── mapping: + │ ├── other.x:9 => x:28 + │ ├── other.y:10 => y:29 + │ ├── other.k:6 => k:30 + │ └── other.v:7 => v:31 + ├── scan uniq_overlaps_pk + │ └── columns: a:32!null b:33!null c:34 d:35 + └── filters + ├── x:28 = c:34 + ├── y:29 = d:35 + └── (k:30 != a:32) OR (v:31 != b:33) + +exec-ddl +CREATE TABLE uniq_hidden_pk ( + a INT, + b INT, + c INT, + d INT, + UNIQUE WITHOUT INDEX (b, c), + UNIQUE WITHOUT INDEX (a, b, d), + UNIQUE WITHOUT INDEX (a) +) +---- + +# Insert with constant input. +# Add inequality filters for the hidden primary key column. +build +INSERT INTO uniq_hidden_pk VALUES (1, 1, 1, 1), (2, 2, 2, 2) +---- +insert uniq_hidden_pk + ├── columns: + ├── insert-mapping: + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column3:9 => c:3 + │ ├── column4:10 => d:4 + │ └── column11:11 => rowid:5 + ├── input binding: &1 + ├── project + │ ├── columns: column11:11 column1:7!null column2:8!null column3:9!null column4:10!null + │ ├── values + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null + │ │ ├── (1, 1, 1, 1) + │ │ └── (2, 2, 2, 2) + │ └── projections + │ └── unique_rowid() [as=column11:11] + └── unique-checks + ├── unique-checks-item: uniq_hidden_pk(b,c) + │ └── semi-join (hash) + │ ├── columns: column2:12!null column3:13!null column11:14 + │ ├── with-scan &1 + │ │ ├── columns: column2:12!null column3:13!null column11:14 + │ │ └── mapping: + │ │ ├── column2:8 => column2:12 + │ │ ├── column3:9 => column3:13 + │ │ └── column11:11 => column11:14 + │ ├── scan uniq_hidden_pk + │ │ └── columns: b:16 c:17 rowid:19!null + │ └── filters + │ ├── column2:12 = b:16 + │ ├── column3:13 = c:17 + │ └── column11:14 != rowid:19 + ├── unique-checks-item: uniq_hidden_pk(a,b,d) + │ └── semi-join (hash) + │ ├── columns: column1:21!null column2:22!null column4:23!null column11:24 + │ ├── with-scan &1 + │ │ ├── columns: column1:21!null column2:22!null column4:23!null column11:24 + │ │ └── mapping: + │ │ ├── column1:7 => column1:21 + │ │ ├── column2:8 => column2:22 + │ │ ├── column4:10 => column4:23 + │ │ └── column11:11 => column11:24 + │ ├── scan uniq_hidden_pk + │ │ └── columns: a:25 b:26 d:28 rowid:29!null + │ └── filters + │ ├── column1:21 = a:25 + │ ├── column2:22 = b:26 + │ ├── column4:23 = d:28 + │ └── column11:24 != rowid:29 + └── unique-checks-item: uniq_hidden_pk(a) + └── semi-join (hash) + ├── columns: column1:31!null column11:32 + ├── with-scan &1 + │ ├── columns: column1:31!null column11:32 + │ └── mapping: + │ ├── column1:7 => column1:31 + │ └── column11:11 => column11:32 + ├── scan uniq_hidden_pk + │ └── columns: a:33 rowid:37!null + └── filters + ├── column1:31 = a:33 + └── column11:32 != rowid:37 + +# Insert with non-constant input. +# Add inequality filters for the hidden primary key column. +build +INSERT INTO uniq_hidden_pk SELECT k, v, x, y FROM other +---- +insert uniq_hidden_pk + ├── columns: + ├── insert-mapping: + │ ├── other.k:7 => a:1 + │ ├── other.v:8 => b:2 + │ ├── other.x:10 => c:3 + │ ├── other.y:11 => d:4 + │ └── column14:14 => uniq_hidden_pk.rowid:5 + ├── input binding: &1 + ├── project + │ ├── columns: column14:14 other.k:7 other.v:8 other.x:10 other.y:11 + │ ├── project + │ │ ├── columns: other.k:7 other.v:8 other.x:10 other.y:11 + │ │ └── scan other + │ │ └── columns: other.k:7 other.v:8 w:9!null other.x:10 other.y:11 other.rowid:12!null other.crdb_internal_mvcc_timestamp:13 + │ └── projections + │ └── unique_rowid() [as=column14:14] + └── unique-checks + ├── unique-checks-item: uniq_hidden_pk(b,c) + │ └── semi-join (hash) + │ ├── columns: v:15 x:16 column14:17 + │ ├── with-scan &1 + │ │ ├── columns: v:15 x:16 column14:17 + │ │ └── mapping: + │ │ ├── other.v:8 => v:15 + │ │ ├── other.x:10 => x:16 + │ │ └── column14:14 => column14:17 + │ ├── scan uniq_hidden_pk + │ │ └── columns: b:19 c:20 uniq_hidden_pk.rowid:22!null + │ └── filters + │ ├── v:15 = b:19 + │ ├── x:16 = c:20 + │ └── column14:17 != uniq_hidden_pk.rowid:22 + ├── unique-checks-item: uniq_hidden_pk(a,b,d) + │ └── semi-join (hash) + │ ├── columns: k:24 v:25 y:26 column14:27 + │ ├── with-scan &1 + │ │ ├── columns: k:24 v:25 y:26 column14:27 + │ │ └── mapping: + │ │ ├── other.k:7 => k:24 + │ │ ├── other.v:8 => v:25 + │ │ ├── other.y:11 => y:26 + │ │ └── column14:14 => column14:27 + │ ├── scan uniq_hidden_pk + │ │ └── columns: a:28 b:29 d:31 uniq_hidden_pk.rowid:32!null + │ └── filters + │ ├── k:24 = a:28 + │ ├── v:25 = b:29 + │ ├── y:26 = d:31 + │ └── column14:27 != uniq_hidden_pk.rowid:32 + └── unique-checks-item: uniq_hidden_pk(a) + └── semi-join (hash) + ├── columns: k:34 column14:35 + ├── with-scan &1 + │ ├── columns: k:34 column14:35 + │ └── mapping: + │ ├── other.k:7 => k:34 + │ └── column14:14 => column14:35 + ├── scan uniq_hidden_pk + │ └── columns: a:36 uniq_hidden_pk.rowid:40!null + └── filters + ├── k:34 = a:36 + └── column14:35 != uniq_hidden_pk.rowid:40 diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index 5f43b6a4671f..b0ffefaad98d 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -341,6 +341,8 @@ func (mb *mutationBuilder) buildUpdate(returning tree.ReturningExprs) { private.PassthroughCols = append(private.PassthroughCols, col.id) } } - mb.outScope.expr = mb.b.factory.ConstructUpdate(mb.outScope.expr, mb.checks, private) + mb.outScope.expr = mb.b.factory.ConstructUpdate( + mb.outScope.expr, mb.uniqueChecks, mb.fkChecks, private, + ) mb.buildReturning(returning) } diff --git a/pkg/sql/opt/xform/testdata/rules/groupby b/pkg/sql/opt/xform/testdata/rules/groupby index 1d6b13b167c6..5c3641043285 100644 --- a/pkg/sql/opt/xform/testdata/rules/groupby +++ b/pkg/sql/opt/xform/testdata/rules/groupby @@ -1795,133 +1795,135 @@ memo INSERT INTO xyz SELECT v, w, 1.0 FROM kuvw ON CONFLICT (x) DO NOTHING ---- memo (optimized, ~19KB, required=[]) - ├── G1: (insert G2 G3 xyz) + ├── G1: (insert G2 G3 G4 xyz) │ └── [] - │ ├── best: (insert G2 G3 xyz) + │ ├── best: (insert G2 G3 G4 xyz) │ └── cost: 2158.51 - ├── G2: (upsert-distinct-on G4 G5 cols=(7)) (upsert-distinct-on G4 G5 cols=(7),ordering=+7 opt(10,11)) + ├── G2: (upsert-distinct-on G5 G6 cols=(7)) (upsert-distinct-on G5 G6 cols=(7),ordering=+7 opt(10,11)) │ └── [] - │ ├── best: (upsert-distinct-on G4="[ordering: +7 opt(10,11)]" G5 cols=(7),ordering=+7 opt(10,11)) + │ ├── best: (upsert-distinct-on G5="[ordering: +7 opt(10,11)]" G6 cols=(7),ordering=+7 opt(10,11)) │ └── cost: 2158.50 - ├── G3: (f-k-checks) - ├── G4: (select G6 G7) + ├── G3: (unique-checks) + ├── G4: (f-k-checks) + ├── G5: (select G7 G8) │ ├── [ordering: +7 opt(10,11)] - │ │ ├── best: (select G6="[ordering: +7 opt(10,11)]" G7) + │ │ ├── best: (select G7="[ordering: +7 opt(10,11)]" G8) │ │ └── cost: 2158.08 │ └── [] - │ ├── best: (select G6 G7) + │ ├── best: (select G7 G8) │ └── cost: 2158.08 - ├── G5: (aggregations G8 G9) - ├── G6: (left-join G10 G11 G12) (right-join G11 G10 G12) (merge-join G10 G11 G13 left-join,+7,+11) (lookup-join G10 G13 xyz,keyCols=[7],outCols=(7,8,10,11)) (lookup-join G10 G13 xyz@xy,keyCols=[7],outCols=(7,8,10,11)) (merge-join G11 G10 G13 right-join,+11,+7) + ├── G6: (aggregations G9 G10) + ├── G7: (left-join G11 G12 G13) (right-join G12 G11 G13) (merge-join G11 G12 G14 left-join,+7,+11) (lookup-join G11 G14 xyz,keyCols=[7],outCols=(7,8,10,11)) (lookup-join G11 G14 xyz@xy,keyCols=[7],outCols=(7,8,10,11)) (merge-join G12 G11 G14 right-join,+11,+7) │ ├── [ordering: +7 opt(10,11)] - │ │ ├── best: (merge-join G10="[ordering: +7 opt(10)]" G11="[ordering: +11]" G13 left-join,+7,+11) + │ │ ├── best: (merge-join G11="[ordering: +7 opt(10)]" G12="[ordering: +11]" G14 left-join,+7,+11) │ │ └── cost: 2148.06 │ └── [] - │ ├── best: (merge-join G10="[ordering: +7 opt(10)]" G11="[ordering: +11]" G13 left-join,+7,+11) + │ ├── best: (merge-join G11="[ordering: +7 opt(10)]" G12="[ordering: +11]" G14 left-join,+7,+11) │ └── cost: 2148.06 - ├── G7: (filters G14) - ├── G8: (first-agg G15) + ├── G8: (filters G15) ├── G9: (first-agg G16) - ├── G10: (project G17 G18 v w) + ├── G10: (first-agg G17) + ├── G11: (project G18 G19 v w) │ ├── [ordering: +7 opt(10)] - │ │ ├── best: (project G17="[ordering: +7]" G18 v w) + │ │ ├── best: (project G18="[ordering: +7]" G19 v w) │ │ └── cost: 1084.03 │ └── [] - │ ├── best: (project G17 G18 v w) + │ ├── best: (project G18 G19 v w) │ └── cost: 1084.03 - ├── G11: (scan xyz,cols=(11)) (scan xyz@xy,cols=(11)) (scan xyz@zyx,cols=(11)) (scan xyz@yy,cols=(11)) + ├── G12: (scan xyz,cols=(11)) (scan xyz@xy,cols=(11)) (scan xyz@zyx,cols=(11)) (scan xyz@yy,cols=(11)) │ ├── [ordering: +11] │ │ ├── best: (scan xyz@xy,cols=(11)) │ │ └── cost: 1034.02 │ └── [] │ ├── best: (scan xyz@xy,cols=(11)) │ └── cost: 1034.02 - ├── G12: (filters G19) - ├── G13: (filters) - ├── G14: (is G20 G21) - ├── G15: (variable w) - ├── G16: (variable "?column?") - ├── G17: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) + ├── G13: (filters G20) + ├── G14: (filters) + ├── G15: (is G21 G22) + ├── G16: (variable w) + ├── G17: (variable "?column?") + ├── G18: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) │ ├── [ordering: +7] │ │ ├── best: (scan kuvw@vw,cols=(7,8)) │ │ └── cost: 1064.02 │ └── [] │ ├── best: (scan kuvw,cols=(7,8)) │ └── cost: 1064.02 - ├── G18: (projections G22) - ├── G19: (eq G23 G20) - ├── G20: (variable x) - ├── G21: (null) - ├── G22: (const 1.0) - └── G23: (variable v) + ├── G19: (projections G23) + ├── G20: (eq G24 G21) + ├── G21: (variable x) + ├── G22: (null) + ├── G23: (const 1.0) + └── G24: (variable v) # Ensure that streaming ensure-upsert-distinct-on will be used. memo INSERT INTO xyz SELECT v, w, 1.0 FROM kuvw ON CONFLICT (x) DO UPDATE SET z=2.0 ---- memo (optimized, ~22KB, required=[]) - ├── G1: (upsert G2 G3 xyz) + ├── G1: (upsert G2 G3 G4 xyz) │ └── [] - │ ├── best: (upsert G2 G3 xyz) + │ ├── best: (upsert G2 G3 G4 xyz) │ └── cost: 2238.10 - ├── G2: (project G4 G5 v w ?column? x y z) + ├── G2: (project G5 G6 v w ?column? x y z) │ └── [] - │ ├── best: (project G4 G5 v w ?column? x y z) + │ ├── best: (project G5 G6 v w ?column? x y z) │ └── cost: 2238.09 - ├── G3: (f-k-checks) - ├── G4: (left-join G6 G7 G8) (right-join G7 G6 G8) (lookup-join G6 G9 xyz,keyCols=[7],outCols=(7,8,10-13)) (lookup-join G10 G9 xyz,keyCols=[11],outCols=(7,8,10-13)) (merge-join G7 G6 G9 right-join,+11,+7) + ├── G3: (unique-checks) + ├── G4: (f-k-checks) + ├── G5: (left-join G7 G8 G9) (right-join G8 G7 G9) (lookup-join G7 G10 xyz,keyCols=[7],outCols=(7,8,10-13)) (lookup-join G11 G10 xyz,keyCols=[11],outCols=(7,8,10-13)) (merge-join G8 G7 G10 right-join,+11,+7) │ └── [] - │ ├── best: (merge-join G7="[ordering: +11]" G6="[ordering: +7 opt(10)]" G9 right-join,+11,+7) + │ ├── best: (merge-join G8="[ordering: +11]" G7="[ordering: +7 opt(10)]" G10 right-join,+11,+7) │ └── cost: 2218.08 - ├── G5: (projections G11) - ├── G6: (ensure-upsert-distinct-on G12 G13 cols=(7)) (ensure-upsert-distinct-on G12 G13 cols=(7),ordering=+7 opt(10)) + ├── G6: (projections G12) + ├── G7: (ensure-upsert-distinct-on G13 G14 cols=(7)) (ensure-upsert-distinct-on G13 G14 cols=(7),ordering=+7 opt(10)) │ ├── [ordering: +7 opt(10)] - │ │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +7 opt(10)]" G13 cols=(7)) + │ │ ├── best: (ensure-upsert-distinct-on G13="[ordering: +7 opt(10)]" G14 cols=(7)) │ │ └── cost: 1124.05 │ └── [] - │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +7 opt(10)]" G13 cols=(7),ordering=+7 opt(10)) + │ ├── best: (ensure-upsert-distinct-on G13="[ordering: +7 opt(10)]" G14 cols=(7),ordering=+7 opt(10)) │ └── cost: 1124.05 - ├── G7: (scan xyz,cols=(11-13)) (scan xyz@zyx,cols=(11-13)) + ├── G8: (scan xyz,cols=(11-13)) (scan xyz@zyx,cols=(11-13)) │ ├── [ordering: +11] │ │ ├── best: (scan xyz,cols=(11-13)) │ │ └── cost: 1064.02 │ └── [] │ ├── best: (scan xyz,cols=(11-13)) │ └── cost: 1064.02 - ├── G8: (filters G14) - ├── G9: (filters) - ├── G10: (lookup-join G6 G9 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) + ├── G9: (filters G15) + ├── G10: (filters) + ├── G11: (lookup-join G7 G10 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) │ └── [] - │ ├── best: (lookup-join G6 G9 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) + │ ├── best: (lookup-join G7 G10 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) │ └── cost: 7174.06 - ├── G11: (case G15 G16 G17) - ├── G12: (project G18 G19 v w) + ├── G12: (case G16 G17 G18) + ├── G13: (project G19 G20 v w) │ ├── [ordering: +7 opt(10)] - │ │ ├── best: (project G18="[ordering: +7]" G19 v w) + │ │ ├── best: (project G19="[ordering: +7]" G20 v w) │ │ └── cost: 1084.03 │ └── [] - │ ├── best: (project G18 G19 v w) + │ ├── best: (project G19 G20 v w) │ └── cost: 1084.03 - ├── G13: (aggregations G20 G21) - ├── G14: (eq G22 G23) - ├── G15: (true) - ├── G16: (scalar-list G24) - ├── G17: (const 2.0) - ├── G18: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) + ├── G14: (aggregations G21 G22) + ├── G15: (eq G23 G24) + ├── G16: (true) + ├── G17: (scalar-list G25) + ├── G18: (const 2.0) + ├── G19: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) │ ├── [ordering: +7] │ │ ├── best: (scan kuvw@vw,cols=(7,8)) │ │ └── cost: 1064.02 │ └── [] │ ├── best: (scan kuvw,cols=(7,8)) │ └── cost: 1064.02 - ├── G19: (projections G25) - ├── G20: (first-agg G26) + ├── G20: (projections G26) ├── G21: (first-agg G27) - ├── G22: (variable v) - ├── G23: (variable x) - ├── G24: (when G28 G27) - ├── G25: (const 1.0) - ├── G26: (variable w) - ├── G27: (variable "?column?") - ├── G28: (is G23 G29) - └── G29: (null) + ├── G22: (first-agg G28) + ├── G23: (variable v) + ├── G24: (variable x) + ├── G25: (when G29 G28) + ├── G26: (const 1.0) + ├── G27: (variable w) + ├── G28: (variable "?column?") + ├── G29: (is G24 G30) + └── G30: (null) diff --git a/pkg/sql/sqltelemetry/planning.go b/pkg/sql/sqltelemetry/planning.go index 245d820aa12c..d902e12ec130 100644 --- a/pkg/sql/sqltelemetry/planning.go +++ b/pkg/sql/sqltelemetry/planning.go @@ -33,6 +33,10 @@ var SubqueryUseCounter = telemetry.GetCounterOnce("sql.plan.subquery") // correlated subquery has been processed during planning. var CorrelatedSubqueryUseCounter = telemetry.GetCounterOnce("sql.plan.subquery.correlated") +// UniqueChecksUseCounter is to be incremented every time a mutation has +// unique checks and the checks are planned by the optimizer. +var UniqueChecksUseCounter = telemetry.GetCounterOnce("sql.plan.unique.checks") + // ForeignKeyChecksUseCounter is to be incremented every time a mutation has // foreign key checks and the checks are planned by the optimizer. var ForeignKeyChecksUseCounter = telemetry.GetCounterOnce("sql.plan.fk.checks")