Skip to content

Commit

Permalink
schemachanger: Enable adding/dropping path of check constraints
Browse files Browse the repository at this point in the history
We enable the adding/dropping path of check constraints in declarative
schema changer (previously it changes between PUBLIC and ABSENT status
with a NotImplemented operation).

Namely, we introduced a new element status `validating` and reused
`validated` such that the status transitions of a check constraint
element is
ABSENT <==> WRITE_ONLY <==> VALIDATED <==> PUBLIC
and it's subject to the 2-version invariant.

Correspondingly, we emit operations for each transition as follows:

Adding path:
 - ABSENT ==> WRITE_ONLY: Add a check constraint (ADD direction) to the mutation slice (Mutation Type op)
 - WRITE_ONLY ==> VALIDATED: validate a check constraint (Validation Type op)
 - VALIDATED ==> PUBLIC: complete the check constraint adding mutation (Mutation Type op)

Dropping path:
 - PUBLIC ==> VALIDATED: Add a check constraint (DROP direction) to the mutation slice (Mutation Type op)
 - VALIDATED ==> WRITE_ONLY: equivalent status
 - (*) WRITE_ONLY ==> ABSENT: complete the check constraint dropping mutation (Mutation Type op)

A check constraint in
 - WRITE_ONLY means it will be enforced for all new UPDATE/INSERT.
 - VALIDATED means validation has succeeded but it's not yet public.

N.B.: * means it's information publishing/destroying stage and hence
should be non-revertible.
  • Loading branch information
Xiang-Gu committed Nov 1, 2022
1 parent c6bc7b4 commit a7f3ae6
Show file tree
Hide file tree
Showing 18 changed files with 644 additions and 248 deletions.
6 changes: 6 additions & 0 deletions pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -655,6 +655,12 @@ func IndexNamePlaceholder(id descpb.IndexID) string {
return fmt.Sprintf("crdb_internal_index_%d_name_placeholder", id)
}

// ConstraintNamePlaceholder constructs a placeholder name for a constraint based
// on its id.
func ConstraintNamePlaceholder(id descpb.ConstraintID) string {
return fmt.Sprintf("crdb_internal_constraint_%d_name_placeholder", id)
}

// RenameColumnInTable will rename the column in tableDesc from oldName to
// newName, including in expressions as well as shard columns.
// The function is recursive because of this, but there should only be one level
Expand Down
29 changes: 28 additions & 1 deletion pkg/sql/schemachanger/scexec/exec_validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,34 @@ func executeValidateUniqueIndex(
func executeValidateCheckConstraint(
ctx context.Context, deps Dependencies, op *scop.ValidateCheckConstraint,
) error {
return errors.Errorf("executeValidateCheckConstraint is not implemented")
descs, err := deps.Catalog().MustReadImmutableDescriptors(ctx, op.TableID)
if err != nil {
return err
}
desc := descs[0]
table, err := catalog.AsTableDescriptor(desc)
if err != nil {
return err
}
constraint, err := table.FindConstraintWithID(op.ConstraintID)
if err != nil {
return err
}
if constraint.CheckConstraint == nil {
return errors.Newf("constraint ID %v does not identify a check constraint in table %v.",
op.ConstraintID, op.TableID)
}

// Execute the validation operation as a root user.
execOverride := sessiondata.InternalExecutorOverride{
User: username.RootUserName(),
}
err = deps.Validator().ValidateCheckConstraint(ctx, table, constraint, execOverride)
if err != nil {
return scerrors.SchemaChangerUserError(err)
}
constraint.CheckConstraint.Validity = descpb.ConstraintValidity_Validated
return nil
}

func executeValidationOps(ctx context.Context, deps Dependencies, ops []scop.Op) (err error) {
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,22 @@ func enqueueDropColumnMutation(tbl *tabledesc.Mutable, col *descpb.ColumnDescrip
return nil
}

func enqueueAddCheckConstraintMutation(
tbl *tabledesc.Mutable, ck *descpb.TableDescriptor_CheckConstraint,
) error {
tbl.AddCheckMutation(ck, descpb.DescriptorMutation_ADD)
tbl.NextMutationID--
return nil
}

func enqueueDropCheckConstraintMutation(
tbl *tabledesc.Mutable, ck *descpb.TableDescriptor_CheckConstraint,
) error {
tbl.AddCheckMutation(ck, descpb.DescriptorMutation_DROP)
tbl.NextMutationID--
return nil
}

func enqueueAddIndexMutation(
tbl *tabledesc.Mutable, idx *descpb.IndexDescriptor, state descpb.DescriptorMutation_State,
) error {
Expand Down
87 changes: 87 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/references.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
Expand Down Expand Up @@ -89,6 +90,92 @@ func (m *visitor) RemoveCheckConstraint(ctx context.Context, op scop.RemoveCheck
return nil
}

func (m *visitor) MakeAbsentCheckConstraintWriteOnly(
ctx context.Context, op scop.MakeAbsentCheckConstraintWriteOnly,
) error {
tbl, err := m.checkOutTable(ctx, op.TableID)
if err != nil || tbl.Dropped() {
return err
}
if op.ConstraintID >= tbl.NextConstraintID {
tbl.NextConstraintID = op.ConstraintID + 1
}

// We should have already validated that the check constraint
// is syntactically valid in the builder, so we just need to
// enqueue it to the descriptor's mutation slice.
ck := &descpb.TableDescriptor_CheckConstraint{
Expr: string(op.Expr),
Name: tabledesc.ConstraintNamePlaceholder(op.ConstraintID),
Validity: descpb.ConstraintValidity_Validating,
ColumnIDs: op.ColumnIDs,
FromHashShardedColumn: op.FromHashShardedColumn,
ConstraintID: op.ConstraintID,
}
return enqueueAddCheckConstraintMutation(tbl, ck)
}

func (m *visitor) MakeValidatedCheckConstraintPublic(
ctx context.Context, op scop.MakeValidatedCheckConstraintPublic,
) error {
tbl, err := m.checkOutTable(ctx, op.TableID)
if err != nil || tbl.Dropped() {
return err
}

var found bool
for idx, mutation := range tbl.Mutations {
if c := mutation.GetConstraint(); c != nil &&
c.ConstraintType == descpb.ConstraintToUpdate_CHECK &&
c.Check.ConstraintID == op.ConstraintID {
tbl.Checks = append(tbl.Checks, &c.Check)

// Remove the mutation from the mutation slice. The `MakeMutationComplete`
// call will also mark the above added check as VALIDATED.
// If this is a rollback of a drop, we are trying to add the check constraint
// back, so swap the direction before making it complete.
mutation.Direction = descpb.DescriptorMutation_ADD
err = tbl.MakeMutationComplete(mutation)
if err != nil {
return err
}
tbl.Mutations = append(tbl.Mutations[:idx], tbl.Mutations[idx+1:]...)

found = true
break
}
}

if !found {
return errors.AssertionFailedf("failed to find check constraint %d in table %q (%d)",
op.ConstraintID, tbl.GetName(), tbl.GetID())
}

if len(tbl.Mutations) == 0 {
tbl.Mutations = nil
}

return nil
}

func (m *visitor) MakePublicCheckConstraintValidated(
ctx context.Context, op scop.MakePublicCheckConstraintValidated,
) error {
tbl, err := m.checkOutTable(ctx, op.TableID)
if err != nil {
return err
}
for i, ck := range tbl.Checks {
if ck.ConstraintID == op.ConstraintID {
tbl.Checks = append(tbl.Checks[:i], tbl.Checks[i+1:]...)
ck.Validity = descpb.ConstraintValidity_Dropping
return enqueueDropCheckConstraintMutation(tbl, ck)
}
}

return errors.AssertionFailedf("failed to find check constraint %d in descriptor %v", op.ConstraintID, tbl)
}

func (m *visitor) RemoveForeignKeyBackReference(
ctx context.Context, op scop.RemoveForeignKeyBackReference,
) error {
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/schemachanger/scop/mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -292,6 +292,33 @@ type RemoveCheckConstraint struct {
ConstraintID descpb.ConstraintID
}

// MakeAbsentCheckConstraintWriteOnly adds a non-existent check constraint
// to the table in the WRITE_ONLY state.
type MakeAbsentCheckConstraintWriteOnly struct {
mutationOp
TableID descpb.ID
ConstraintID descpb.ConstraintID
ColumnIDs []descpb.ColumnID
scpb.Expression
FromHashShardedColumn bool
}

// MakePublicCheckConstraintValidated moves a public
// check constraint to VALIDATED.
type MakePublicCheckConstraintValidated struct {
mutationOp
TableID descpb.ID
ConstraintID descpb.ConstraintID
}

// MakeValidatedCheckConstraintPublic moves a new, validated check
// constraint from mutation to public.
type MakeValidatedCheckConstraintPublic struct {
mutationOp
TableID descpb.ID
ConstraintID descpb.ConstraintID
}

// RemoveForeignKeyConstraint removes a foreign key from the origin table.
type RemoveForeignKeyConstraint struct {
mutationOp
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/schemachanger/scop/mutation_visitor_generated.go

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

2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scpb/scpb.proto
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,12 @@ enum Status {
DROPPED = 5;

// Intermediate states on the column and index dropping and adding paths.
// WRITE_ONLY is also used on constraint adding and dropping paths.
WRITE_ONLY = 6;
DELETE_ONLY = 7;

// Intermediate states on the index dropping and adding paths.
// VALIDATED is also used on constraint adding and dropping paths.
VALIDATED = 8;
MERGED = 9;
MERGE_ONLY = 10;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,64 @@ func init() {
opRegistry.register((*scpb.CheckConstraint)(nil),
toPublic(
scpb.Status_ABSENT,
to(scpb.Status_WRITE_ONLY,
emit(func(this *scpb.CheckConstraint) *scop.MakeAbsentCheckConstraintWriteOnly {
return &scop.MakeAbsentCheckConstraintWriteOnly{
TableID: this.TableID,
ConstraintID: this.ConstraintID,
ColumnIDs: this.ColumnIDs,
Expression: this.Expression,
FromHashShardedColumn: this.FromHashShardedColumn,
}
}),
emit(func(this *scpb.CheckConstraint) *scop.UpdateTableBackReferencesInTypes {
if len(this.UsesTypeIDs) == 0 {
return nil
}
return &scop.UpdateTableBackReferencesInTypes{
TypeIDs: this.UsesTypeIDs,
BackReferencedTableID: this.TableID,
}
}),
emit(func(this *scpb.CheckConstraint) *scop.UpdateBackReferencesInSequences {
if len(this.UsesSequenceIDs) == 0 {
return nil
}
return &scop.UpdateBackReferencesInSequences{
SequenceIDs: this.UsesSequenceIDs,
BackReferencedTableID: this.TableID,
}
}),
),
to(scpb.Status_VALIDATED,
emit(func(this *scpb.CheckConstraint) *scop.ValidateCheckConstraint {
return &scop.ValidateCheckConstraint{
TableID: this.TableID,
ConstraintID: this.ConstraintID,
}
}),
),
to(scpb.Status_PUBLIC,
emit(func(this *scpb.CheckConstraint) *scop.NotImplemented {
return notImplemented(this)
emit(func(this *scpb.CheckConstraint) *scop.MakeValidatedCheckConstraintPublic {
return &scop.MakeValidatedCheckConstraintPublic{
TableID: this.TableID,
ConstraintID: this.ConstraintID,
}
}),
),
),
toAbsent(
scpb.Status_PUBLIC,
to(scpb.Status_VALIDATED,
emit(func(this *scpb.CheckConstraint) *scop.MakePublicCheckConstraintValidated {
return &scop.MakePublicCheckConstraintValidated{
TableID: this.TableID,
ConstraintID: this.ConstraintID,
}
}),
),
equiv(scpb.Status_WRITE_ONLY),
to(scpb.Status_ABSENT,
// TODO(postamar): remove revertibility constraint when possible
revertible(false),
emit(func(this *scpb.CheckConstraint) *scop.RemoveCheckConstraint {
return &scop.RemoveCheckConstraint{
Expand Down
25 changes: 14 additions & 11 deletions pkg/sql/schemachanger/scplan/internal/rules/dep_drop_constraint.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,31 +18,34 @@ import (

// These rules ensure that constraint-dependent elements, like an constraint's
// name, etc. disappear once the constraint reaches a suitable state.
// TODO (xiang): The dep rules here are not complete, as they are aimed specifically
// for check constraints only. Complete them when we properly support the
// other constraints: UniqueWithoutIndex, ForeignKey, Unique, and Not Null.
func init() {

registerDepRuleForDrop(
"constraint dependent absent right before constraint",
scgraph.SameStagePrecedence,
"dependent", "constraint",
scpb.Status_ABSENT, scpb.Status_ABSENT,
"constraint no longer public before dependents",
scgraph.Precedence,
"constraint", "dependent",
scpb.Status_VALIDATED, scpb.Status_ABSENT,
func(from, to nodeVars) rel.Clauses {
return rel.Clauses{
from.typeFilter(isConstraintDependent),
to.typeFilter(isConstraint, not(isIndex)),
from.typeFilter(isSupportedNonIndexBackedConstraint),
to.typeFilter(isConstraintDependent),
joinOnConstraintID(from, to, "table-id", "constraint-id"),
}
},
)

registerDepRuleForDrop(
"constraint dependent absent right before constraint",
scgraph.SameStagePrecedence,
"dependent", "constraint",
scpb.Status_VALIDATED, scpb.Status_ABSENT,
"dependents removed before constraint",
scgraph.Precedence,
"dependents", "constraint",
scpb.Status_ABSENT, scpb.Status_ABSENT,
func(from, to nodeVars) rel.Clauses {
return rel.Clauses{
from.typeFilter(isConstraintDependent),
to.typeFilter(isConstraint, isIndex),
to.typeFilter(isSupportedNonIndexBackedConstraint),
joinOnConstraintID(from, to, "table-id", "constraint-id"),
}
},
Expand Down
Loading

0 comments on commit a7f3ae6

Please sign in to comment.