Skip to content

Commit

Permalink
schemachanger: Implement ALTER TABLE .. VALIDATE CONSTRAINT
Browse files Browse the repository at this point in the history
The main idea is to drop the unvalidated element and add a vanilla
counterpart in the builder state.
  • Loading branch information
Xiang-Gu committed Feb 7, 2023
1 parent f8a7f49 commit 74d4545
Show file tree
Hide file tree
Showing 16 changed files with 880 additions and 7 deletions.
5 changes: 5 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -1571,7 +1571,7 @@ ALTER TABLE unique_without_index ADD CONSTRAINT my_unique_e UNIQUE WITHOUT INDEX
ALTER TABLE unique_without_index ADD CONSTRAINT my_unique_e2 UNIQUE WITHOUT INDEX (e) NOT VALID

# Trying to validate one of the constraints will fail.
statement error pgcode 23505 pq: could not create unique constraint "my_unique_e"\nDETAIL: Key \(e\)=\(1\) is duplicated\.
statement error pgcode 23505 pq: could not create unique constraint ".*"\nDETAIL: Key \(e\)=\(1\) is duplicated\.
ALTER TABLE unique_without_index VALIDATE CONSTRAINT my_unique_e

# But after we delete a row, validation should succeed.
Expand Down Expand Up @@ -1632,7 +1632,7 @@ statement ok
ALTER TABLE unique_without_index_partial ADD CONSTRAINT uniq_a_1 UNIQUE WITHOUT INDEX (a) WHERE b > 0 OR c > 0 NOT VALID

# Trying to validate the constraint will fail.
statement error pgcode 23505 pq: could not create unique constraint "uniq_a_1"\nDETAIL: Key \(a\)=\(1\) is duplicated\.
statement error pgcode 23505 pq: could not create unique constraint ".*"\nDETAIL: Key \(a\)=\(1\) is duplicated\.
ALTER TABLE unique_without_index_partial VALIDATE CONSTRAINT uniq_a_1

# But after we delete a row, validation should succeed.
Expand Down
77 changes: 77 additions & 0 deletions pkg/sql/opt/exec/execbuilder/testdata/unique
Original file line number Diff line number Diff line change
Expand Up @@ -1775,6 +1775,13 @@ ALTER TABLE uniq_overlaps_pk VALIDATE CONSTRAINT unique_a

# Same test as the previous, but now that the constraint has been validated, it
# can be treated as a key. This allows the joins to be more efficient.
#
# The implementation detail for VALIDATE CONSTRAINT is slightly different between
# legacy and declarative schema changer, causing the output of the following
# EXPLAIN UPDATE to be slightly different: the ordering of three same-level
# constraint checks is different. We thus test on the same EXPLAIN UPDATE twice
# with slightly different expected output.
skipif config local
query T
EXPLAIN UPDATE uniq_overlaps_pk SET a = 1, b = 2, c = 3, d = 4 WHERE a = 5
----
Expand Down Expand Up @@ -1844,6 +1851,76 @@ vectorized: true
└── • scan buffer
label: buffer 1

onlyif config local
query T
EXPLAIN UPDATE uniq_overlaps_pk SET a = 1, b = 2, c = 3, d = 4 WHERE a = 5
----
distribution: local
vectorized: true
·
• root
├── • update
│ │ table: uniq_overlaps_pk
│ │ set: a, b, c, d
│ │
│ └── • buffer
│ │ label: buffer 1
│ │
│ └── • render
│ │
│ └── • scan
│ missing stats
│ table: uniq_overlaps_pk@uniq_overlaps_pk_pkey
│ spans: [/5 - /5]
│ locking strength: for update
├── • constraint-check
│ │
│ └── • error if rows
│ │
│ └── • hash join (right semi)
│ │ equality: (b, c) = (b_new, c_new)
│ │ right cols are key
│ │ pred: a_new != a
│ │
│ ├── • scan
│ │ missing stats
│ │ table: uniq_overlaps_pk@uniq_overlaps_pk_pkey
│ │ spans: FULL SCAN
│ │
│ └── • scan buffer
│ label: buffer 1
├── • constraint-check
│ │
│ └── • error if rows
│ │
│ └── • hash join (right semi)
│ │ equality: (c, d) = (c_new, d_new)
│ │ right cols are key
│ │ pred: (a_new != a) OR (b_new != b)
│ │
│ ├── • scan
│ │ missing stats
│ │ table: uniq_overlaps_pk@uniq_overlaps_pk_pkey
│ │ spans: FULL SCAN
│ │
│ └── • scan buffer
│ label: buffer 1
└── • constraint-check
└── • error if rows
└── • lookup join (semi)
│ table: uniq_overlaps_pk@uniq_overlaps_pk_pkey
│ equality: (a_new) = (a)
│ pred: b_new != b
└── • scan buffer
label: buffer 1

# Update with non-constant input.
# No need to add a check for b,c since those columns weren't updated.
# Add inequality filters for the hidden primary key column.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"alter_table_alter_primary_key.go",
"alter_table_drop_column.go",
"alter_table_drop_constraint.go",
"alter_table_validate_constraint.go",
"comment_on.go",
"create_function.go",
"create_index.go",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,8 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{

return false
}},
reflect.TypeOf((*tree.AlterTableDropConstraint)(nil)): {fn: alterTableDropConstraint, on: true, minSupportedClusterVersion: clusterversion.V23_1},
reflect.TypeOf((*tree.AlterTableDropConstraint)(nil)): {fn: alterTableDropConstraint, on: true, minSupportedClusterVersion: clusterversion.V23_1},
reflect.TypeOf((*tree.AlterTableValidateConstraint)(nil)): {fn: alterTableValidateConstraint, on: true, minSupportedClusterVersion: clusterversion.V23_1},
}

// alterTableAddConstraintMinSupportedClusterVersion tracks the minimal supported cluster version
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
// Copyright 2023 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 scbuildstmt

import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)

func alterTableValidateConstraint(
b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableValidateConstraint,
) {
constraintElems := b.ResolveConstraint(tbl.TableID, t.Constraint, ResolveParams{
IsExistenceOptional: false,
RequiredPrivilege: privilege.CREATE,
})

// 1. We can only validate non-index-backed constraints. Panic if not.
_, _, constraintNameElem := scpb.FindConstraintWithoutIndexName(constraintElems)
if constraintNameElem == nil {
panic(pgerror.Newf(pgcode.WrongObjectType,
"constraint %q of relation %q is not a foreign key, check, or unique without index"+
" constraint", tree.ErrString(&t.Constraint), tree.ErrString(tn)))
}

// 2. Return if the constraint is already validated.
constraintID := constraintNameElem.ConstraintID
if !shouldValidateConstraint(b, tbl.TableID, constraintID) {
return
}

// 3. Drop the not-valid constraint and old constraint name element
// Add a new sibling constraint and a new constraint name element.
validateConstraint(b, tbl.TableID, validateConstraintSpec{
constraintNameElem: constraintNameElem,
ckNotValidElem: retrieveCheckConstraintUnvalidatedElem(b, tbl.TableID, constraintID),
uwiNotValidElem: retrieveUniqueWithoutIndexConstraintUnvalidatedElem(b, tbl.TableID, constraintID),
fkNotValidElem: retrieveForeignKeyConstraintUnvalidatedElem(b, tbl.TableID, constraintID),
})
}

func retrieveCheckConstraintUnvalidatedElem(
b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID,
) (CheckConstraintUnvalidatedElem *scpb.CheckConstraintUnvalidated) {
scpb.ForEachCheckConstraintUnvalidated(b.QueryByID(tableID), func(
current scpb.Status, target scpb.TargetStatus, e *scpb.CheckConstraintUnvalidated,
) {
if e.ConstraintID == constraintID {
CheckConstraintUnvalidatedElem = e
}
})
return CheckConstraintUnvalidatedElem
}

func retrieveUniqueWithoutIndexConstraintUnvalidatedElem(
b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID,
) (UniqueWithoutIndexConstraintUnvalidatedElem *scpb.UniqueWithoutIndexConstraintUnvalidated) {
scpb.ForEachUniqueWithoutIndexConstraintUnvalidated(b.QueryByID(tableID), func(
current scpb.Status, target scpb.TargetStatus, e *scpb.UniqueWithoutIndexConstraintUnvalidated,
) {
if e.ConstraintID == constraintID {
UniqueWithoutIndexConstraintUnvalidatedElem = e
}
})
return UniqueWithoutIndexConstraintUnvalidatedElem
}

func retrieveForeignKeyConstraintUnvalidatedElem(
b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID,
) (ForeignKeyConstraintUnvalidatedElem *scpb.ForeignKeyConstraintUnvalidated) {
scpb.ForEachForeignKeyConstraintUnvalidated(b.QueryByID(tableID), func(
current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraintUnvalidated,
) {
if e.ConstraintID == constraintID {
ForeignKeyConstraintUnvalidatedElem = e
}
})
return ForeignKeyConstraintUnvalidatedElem
}

type validateConstraintSpec struct {
constraintNameElem *scpb.ConstraintWithoutIndexName
ckNotValidElem *scpb.CheckConstraintUnvalidated
uwiNotValidElem *scpb.UniqueWithoutIndexConstraintUnvalidated
fkNotValidElem *scpb.ForeignKeyConstraintUnvalidated
}

func validateConstraint(b BuildCtx, tableID catid.DescID, spec validateConstraintSpec) {
nextConstraintID := b.NextTableConstraintID(tableID)
if spec.ckNotValidElem != nil {
b.Drop(spec.ckNotValidElem)
b.Add(&scpb.CheckConstraint{
TableID: tableID,
ConstraintID: nextConstraintID,
ColumnIDs: spec.ckNotValidElem.ColumnIDs,
Expression: spec.ckNotValidElem.Expression,
FromHashShardedColumn: false,
IndexIDForValidation: getIndexIDForValidationForConstraint(b, tableID),
})
}
if spec.uwiNotValidElem != nil {
b.Drop(spec.uwiNotValidElem)
b.Add(&scpb.UniqueWithoutIndexConstraint{
TableID: tableID,
ConstraintID: nextConstraintID,
ColumnIDs: spec.uwiNotValidElem.ColumnIDs,
Predicate: spec.uwiNotValidElem.Predicate,
})
}
if spec.fkNotValidElem != nil {
b.Drop(spec.fkNotValidElem)
b.Add(&scpb.ForeignKeyConstraint{
TableID: tableID,
ConstraintID: nextConstraintID,
ColumnIDs: spec.fkNotValidElem.ColumnIDs,
ReferencedTableID: spec.fkNotValidElem.ReferencedTableID,
ReferencedColumnIDs: spec.fkNotValidElem.ReferencedColumnIDs,
OnUpdateAction: spec.fkNotValidElem.OnUpdateAction,
OnDeleteAction: spec.fkNotValidElem.OnDeleteAction,
CompositeKeyMatchMethod: spec.fkNotValidElem.CompositeKeyMatchMethod,
IndexIDForValidation: getIndexIDForValidationForConstraint(b, tableID),
})
}
b.Drop(spec.constraintNameElem)
b.Add(&scpb.ConstraintWithoutIndexName{
TableID: tableID,
ConstraintID: nextConstraintID,
Name: spec.constraintNameElem.Name,
})
}
17 changes: 17 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -839,3 +839,20 @@ func maybeFailOnCrossDBTypeReference(b BuildCtx, typeID descpb.ID, parentDBID de
typeName.String()))
}
}

// shouldValidateConstraint returns true if the constraint ID identifies
// an unvalidated, non-index-backed constraint, either public or to-be-public.
func shouldValidateConstraint(
b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID,
) (should bool) {
constraintElements(b, tableID, constraintID).Filter(publicTargetFilter).ForEachElementStatus(func(
current scpb.Status, target scpb.TargetStatus, e scpb.Element,
) {
switch e.(type) {
case *scpb.CheckConstraintUnvalidated, *scpb.UniqueWithoutIndexConstraintUnvalidated,
*scpb.ForeignKeyConstraintUnvalidated:
should = true
}
})
return should
}
Original file line number Diff line number Diff line change
Expand Up @@ -84,10 +84,6 @@ unimplemented
ALTER TABLE defaultdb.foo ADD PRIMARY KEY (l);
----

unimplemented
ALTER TABLE defaultdb.foo VALIDATE CONSTRAINT foobar
----

unimplemented
ALTER TABLE defaultdb.foo PARTITION BY NOTHING
----
Expand Down
25 changes: 25 additions & 0 deletions pkg/sql/schemachanger/sctest_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 74d4545

Please sign in to comment.