From ec0a2c4c16dc133e8456eb86db1a7a6db75866e2 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Thu, 15 Dec 2022 15:19:33 -0500 Subject: [PATCH 1/5] sql: Added logic to validate unique without index constraint --- pkg/sql/backfill.go | 13 +++++++++-- pkg/sql/check.go | 36 ++++++++++++++++++++++++++---- pkg/sql/scrub_unique_constraint.go | 3 ++- 3 files changed, 45 insertions(+), 7 deletions(-) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 02ae4c0f22f0..5062fd67578d 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -1505,8 +1505,6 @@ func (e InvalidIndexesError) Error() string { // ValidateConstraint validates the constraint against all rows // in `tbl`. -// -// TODO (xiang): Support validating UNIQUE_WITHOUT_INDEX constraint in this function. func ValidateConstraint( ctx context.Context, tableDesc catalog.TableDescriptor, @@ -1558,6 +1556,15 @@ func ValidateConstraint( indexIDForValidation, txn, ie) }, ) + case catconstants.ConstraintTypeUniqueWithoutIndex: + uwi := constraint.AsUniqueWithoutIndex() + return ie.WithSyntheticDescriptors( + []catalog.Descriptor{tableDesc}, + func() error { + return validateUniqueConstraint(ctx, tableDesc, uwi.GetName(), uwi.CollectKeyColumnIDs().Ordered(), + uwi.GetPredicate(), indexIDForValidation, ie, txn, sessionData.User(), false) + }, + ) default: return errors.AssertionFailedf("validation of unsupported constraint type") } @@ -2049,6 +2056,7 @@ func countIndexRowsAndMaybeCheckUniqueness( idx.GetName(), idx.IndexDesc().KeyColumnIDs[idx.ImplicitPartitioningColumnCount():], idx.GetPredicate(), + 0, /* indexIDForValidation */ ie, txn, username.NodeUserName(), @@ -2737,6 +2745,7 @@ func validateUniqueWithoutIndexConstraintInTxn( uc.Name, uc.ColumnIDs, uc.Predicate, + 0, /* indexIDForValidation */ ie, txn, user, diff --git a/pkg/sql/check.go b/pkg/sql/check.go index 9b285771eed2..fcddb6e10565 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -368,8 +368,15 @@ func validateForeignKey( // The pred argument is a partial unique constraint predicate, which filters the // subset of rows that are guaranteed unique by the constraint. If the unique // constraint is not partial, pred should be empty. +// +// `indexIDForValidation`, if non-zero, will be used to force the sql query to +// use this particular index by hinting the query. func duplicateRowQuery( - srcTbl catalog.TableDescriptor, columnIDs []descpb.ColumnID, pred string, limitResults bool, + srcTbl catalog.TableDescriptor, + columnIDs []descpb.ColumnID, + pred string, + indexIDForValidation descpb.IndexID, + limitResults bool, ) (sql string, colNames []string, _ error) { colNames, err := srcTbl.NamesForColumnIDs(columnIDs) if err != nil { @@ -397,13 +404,24 @@ func duplicateRowQuery( if limitResults { limit = " LIMIT 1" } - return fmt.Sprintf( + query := fmt.Sprintf( `SELECT %[1]s FROM [%[2]d AS tbl] WHERE %[3]s GROUP BY %[1]s HAVING count(*) > 1 %[4]s`, strings.Join(srcCols, ", "), // 1 srcTbl.GetID(), // 2 strings.Join(srcWhere, " AND "), // 3 limit, // 4 - ), colNames, nil + ) + if indexIDForValidation != 0 { + query = fmt.Sprintf( + `SELECT %[1]s FROM [%[2]d AS tbl]@[%[3]d] WHERE %[4]s GROUP BY %[1]s HAVING count(*) > 1 %[5]s`, + strings.Join(srcCols, ", "), // 1 + srcTbl.GetID(), // 2 + indexIDForValidation, // 3 + strings.Join(srcWhere, " AND "), // 4 + limit, // 5 + ) + } + return query, colNames, nil } // RevalidateUniqueConstraintsInCurrentDB verifies that all unique constraints @@ -473,6 +491,7 @@ func (p *planner) RevalidateUniqueConstraint( index.GetName(), index.IndexDesc().KeyColumnIDs[index.ImplicitPartitioningColumnCount():], index.GetPredicate(), + 0, /* indexIDForValidation */ p.ExecCfg().InternalExecutor, p.Txn(), p.User(), @@ -493,6 +512,7 @@ func (p *planner) RevalidateUniqueConstraint( uc.GetName(), uc.CollectKeyColumnIDs().Ordered(), uc.GetPredicate(), + 0, /* indexIDForValidation */ p.ExecCfg().InternalExecutor, p.Txn(), p.User(), @@ -557,6 +577,7 @@ func RevalidateUniqueConstraintsInTable( index.GetName(), index.IndexDesc().KeyColumnIDs[index.ImplicitPartitioningColumnCount():], index.GetPredicate(), + 0, /* indexIDForValidation */ ie, txn, user, @@ -577,6 +598,7 @@ func RevalidateUniqueConstraintsInTable( uc.GetName(), uc.CollectKeyColumnIDs().Ordered(), uc.GetPredicate(), + 0, /* indexIDForValidation */ ie, txn, user, @@ -595,6 +617,11 @@ func RevalidateUniqueConstraintsInTable( // validateUniqueConstraint verifies that all the rows in the srcTable // have unique values for the given columns. // +// `indexIDForValidation`, if non-zero, will be used to force validation +// against this particular index. This is used to facilitate the declarative +// schema changer when the validation should be against a yet non-public +// primary index. +// // It operates entirely on the current goroutine and is thus able to // reuse an existing kv.Txn safely. // @@ -606,13 +633,14 @@ func validateUniqueConstraint( constraintName string, columnIDs []descpb.ColumnID, pred string, + indexIDForValidation descpb.IndexID, ie sqlutil.InternalExecutor, txn *kv.Txn, user username.SQLUsername, preExisting bool, ) error { query, colNames, err := duplicateRowQuery( - srcTable, columnIDs, pred, true, /* limitResults */ + srcTable, columnIDs, pred, indexIDForValidation, true, /* limitResults */ ) if err != nil { return err diff --git a/pkg/sql/scrub_unique_constraint.go b/pkg/sql/scrub_unique_constraint.go index ed85eec8a3d7..bb930ce4a4ba 100644 --- a/pkg/sql/scrub_unique_constraint.go +++ b/pkg/sql/scrub_unique_constraint.go @@ -124,7 +124,8 @@ func (o *sqlUniqueConstraintCheckOperation) Start(params runParams) error { asOf = fmt.Sprintf("AS OF SYSTEM TIME '%s'", o.asOf.AsOfSystemTime()) } tableName := fmt.Sprintf("%s.%s", o.tableName.Catalog(), o.tableName.Table()) - dup, _, err := duplicateRowQuery(o.tableDesc, o.cols, o.predicate, false /* limitResults */) + dup, _, err := duplicateRowQuery(o.tableDesc, o.cols, o.predicate, + 0 /* indexIDForValidation */, false /* limitResults */) if err != nil { return err } From c0231f35a2eb239e3b35b565080a9a32eab341f4 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Thu, 15 Dec 2022 16:03:13 -0500 Subject: [PATCH 2/5] opgen: support adding/dropping path of UniqueWithoutIndex element --- .../scexec/scmutationexec/constraint.go | 131 ++++++++++++++++++ .../scexec/scmutationexec/helpers.go | 16 +++ pkg/sql/schemachanger/scop/mutation.go | 33 +++++ .../scop/mutation_visitor_generated.go | 24 ++++ pkg/sql/schemachanger/scpb/elements.proto | 4 + pkg/sql/schemachanger/scpb/uml/table.puml | 1 + .../opgen_unique_without_index_constraint.go | 60 +++++++- 7 files changed, 265 insertions(+), 4 deletions(-) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go index 09348d53ab0f..34880529d12e 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go @@ -199,6 +199,43 @@ func (m *visitor) RemoveForeignKeyConstraint( return nil } +func (m *visitor) RemoveUniqueWithoutIndexConstraint( + ctx context.Context, op scop.RemoveUniqueWithoutIndexConstraint, +) error { + tbl, err := m.checkOutTable(ctx, op.TableID) + if err != nil || tbl.Dropped() { + return err + } + var found bool + for i, c := range tbl.UniqueWithoutIndexConstraints { + if c.ConstraintID == op.ConstraintID { + tbl.UniqueWithoutIndexConstraints = append(tbl.UniqueWithoutIndexConstraints[:i], tbl.UniqueWithoutIndexConstraints[i+1:]...) + if len(tbl.UniqueWithoutIndexConstraints) == 0 { + tbl.UniqueWithoutIndexConstraints = nil + } + found = true + break + } + } + for i, m := range tbl.Mutations { + if c := m.GetConstraint(); c != nil && + c.ConstraintType == descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX && + c.UniqueWithoutIndexConstraint.ConstraintID == op.ConstraintID { + tbl.Mutations = append(tbl.Mutations[:i], tbl.Mutations[i+1:]...) + if len(tbl.Mutations) == 0 { + tbl.Mutations = nil + } + found = true + break + } + } + if !found { + return errors.AssertionFailedf("failed to find unnique_without_index constraint %d in table %q (%d)", + op.ConstraintID, tbl.GetName(), tbl.GetID()) + } + return nil +} + func (m *visitor) MakeAbsentForeignKeyConstraintWriteOnly( ctx context.Context, op scop.MakeAbsentForeignKeyConstraintWriteOnly, ) error { @@ -297,3 +334,97 @@ func (m *visitor) MakePublicForeignKeyConstraintValidated( return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, tbl) } + +func (m *visitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly( + ctx context.Context, op scop.MakeAbsentUniqueWithoutIndexConstraintWriteOnly, +) 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 + } + + uwi := &descpb.UniqueWithoutIndexConstraint{ + TableID: op.TableID, + ColumnIDs: op.ColumnIDs, + Name: tabledesc.ConstraintNamePlaceholder(op.ConstraintID), + Validity: descpb.ConstraintValidity_Validating, + ConstraintID: op.ConstraintID, + } + if op.Predicate != nil { + uwi.Predicate = string(op.Predicate.Expr) + } + if err = enqueueAddUniqueWithoutIndexConstraintMutation(tbl, uwi); err != nil { + return err + } + // Fast-forward the mutation state to WRITE_ONLY because this constraint + // is now considered as enforced. + tbl.Mutations[len(tbl.Mutations)-1].State = descpb.DescriptorMutation_WRITE_ONLY + return nil +} + +func (m *visitor) MakeValidatedUniqueWithoutIndexConstraintPublic( + ctx context.Context, op scop.MakeValidatedUniqueWithoutIndexConstraintPublic, +) 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_UNIQUE_WITHOUT_INDEX && + c.UniqueWithoutIndexConstraint.ConstraintID == op.ConstraintID { + tbl.UniqueWithoutIndexConstraints = append(tbl.UniqueWithoutIndexConstraints, c.UniqueWithoutIndexConstraint) + + // Remove the mutation from the mutation slice. The `MakeMutationComplete` + // call will also mark the above added unique_without_index as VALIDATED. + // If this is a rollback of a drop, we are trying to add the + // unique_without_index 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:]...) + if len(tbl.Mutations) == 0 { + tbl.Mutations = nil + } + + found = true + break + } + } + + if !found { + return errors.AssertionFailedf("failed to find unique_without_index constraint %d in table %q (%d)", + op.ConstraintID, tbl.GetName(), tbl.GetID()) + } + + return nil +} + +func (m *visitor) MakePublicUniqueWithoutIndexConstraintValidated( + ctx context.Context, op scop.MakePublicUniqueWithoutIndexConstraintValidated, +) error { + tbl, err := m.checkOutTable(ctx, op.TableID) + if err != nil { + return err + } + for i, uwi := range tbl.UniqueWithoutIndexConstraints { + if uwi.ConstraintID == op.ConstraintID { + tbl.UniqueWithoutIndexConstraints = append(tbl.UniqueWithoutIndexConstraints[:i], tbl.UniqueWithoutIndexConstraints[i+1:]...) + if len(tbl.UniqueWithoutIndexConstraints) == 0 { + tbl.UniqueWithoutIndexConstraints = nil + } + uwi.Validity = descpb.ConstraintValidity_Dropping + return enqueueDropUniqueWithoutIndexConstraintMutation(tbl, &uwi) + } + } + + return errors.AssertionFailedf("failed to find unique_without_index constraint %d in descriptor %v", op.ConstraintID, tbl) +} diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index b48cd255e350..6867b3323074 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -219,6 +219,14 @@ func enqueueAddCheckConstraintMutation( return nil } +func enqueueAddUniqueWithoutIndexConstraintMutation( + tbl *tabledesc.Mutable, uwi *descpb.UniqueWithoutIndexConstraint, +) error { + tbl.AddUniqueWithoutIndexMutation(uwi, descpb.DescriptorMutation_ADD) + tbl.NextMutationID-- + return nil +} + func enqueueDropCheckConstraintMutation( tbl *tabledesc.Mutable, ck *descpb.TableDescriptor_CheckConstraint, ) error { @@ -227,6 +235,14 @@ func enqueueDropCheckConstraintMutation( return nil } +func enqueueDropUniqueWithoutIndexConstraintMutation( + tbl *tabledesc.Mutable, uwi *descpb.UniqueWithoutIndexConstraint, +) error { + tbl.AddUniqueWithoutIndexMutation(uwi, descpb.DescriptorMutation_DROP) + tbl.NextMutationID-- + return nil +} + func enqueueAddForeignKeyConstraintMutation( tbl *tabledesc.Mutable, fk *descpb.ForeignKeyConstraint, ) error { diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go index dc22a4af5f53..9c8adbd15ce0 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -368,6 +368,39 @@ type RemoveForeignKeyBackReference struct { OriginConstraintID descpb.ConstraintID } +// MakeAbsentUniqueWithoutIndexConstraintWriteOnly adds a non-existent +// unique_without_index constraint to the table in the WRITE_ONLY state. +type MakeAbsentUniqueWithoutIndexConstraintWriteOnly struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID + ColumnIDs []descpb.ColumnID + Predicate *scpb.Expression +} + +// MakeValidatedUniqueWithoutIndexConstraintPublic moves a new, validated unique_without_index +// constraint from mutation to public. +type MakeValidatedUniqueWithoutIndexConstraintPublic struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID +} + +// MakePublicUniqueWithoutIndexConstraintValidated moves a public +// unique_without_index constraint to VALIDATED. +type MakePublicUniqueWithoutIndexConstraintValidated struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID +} + +// RemoveUniqueWithoutIndexConstraint removes a unique_without_index from the origin table. +type RemoveUniqueWithoutIndexConstraint struct { + mutationOp + TableID descpb.ID + ConstraintID descpb.ConstraintID +} + // RemoveSchemaParent removes the schema - parent database relationship. type RemoveSchemaParent struct { mutationOp diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go index a947168976ed..1d150f6da3ee 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -63,6 +63,10 @@ type MutationVisitor interface { MakePublicForeignKeyConstraintValidated(context.Context, MakePublicForeignKeyConstraintValidated) error RemoveForeignKeyConstraint(context.Context, RemoveForeignKeyConstraint) error RemoveForeignKeyBackReference(context.Context, RemoveForeignKeyBackReference) error + MakeAbsentUniqueWithoutIndexConstraintWriteOnly(context.Context, MakeAbsentUniqueWithoutIndexConstraintWriteOnly) error + MakeValidatedUniqueWithoutIndexConstraintPublic(context.Context, MakeValidatedUniqueWithoutIndexConstraintPublic) error + MakePublicUniqueWithoutIndexConstraintValidated(context.Context, MakePublicUniqueWithoutIndexConstraintValidated) error + RemoveUniqueWithoutIndexConstraint(context.Context, RemoveUniqueWithoutIndexConstraint) error RemoveSchemaParent(context.Context, RemoveSchemaParent) error AddIndexPartitionInfo(context.Context, AddIndexPartitionInfo) error LogEvent(context.Context, LogEvent) error @@ -310,6 +314,26 @@ func (op RemoveForeignKeyBackReference) Visit(ctx context.Context, v MutationVis return v.RemoveForeignKeyBackReference(ctx, op) } +// Visit is part of the MutationOp interface. +func (op MakeAbsentUniqueWithoutIndexConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeAbsentUniqueWithoutIndexConstraintWriteOnly(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakeValidatedUniqueWithoutIndexConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakeValidatedUniqueWithoutIndexConstraintPublic(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op MakePublicUniqueWithoutIndexConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error { + return v.MakePublicUniqueWithoutIndexConstraintValidated(ctx, op) +} + +// Visit is part of the MutationOp interface. +func (op RemoveUniqueWithoutIndexConstraint) Visit(ctx context.Context, v MutationVisitor) error { + return v.RemoveUniqueWithoutIndexConstraint(ctx, op) +} + // Visit is part of the MutationOp interface. func (op RemoveSchemaParent) Visit(ctx context.Context, v MutationVisitor) error { return v.RemoveSchemaParent(ctx, op) diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index e989b88a7db6..9c01b205aac5 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -328,6 +328,10 @@ message UniqueWithoutIndexConstraint { repeated uint32 column_ids = 3 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; // Predicate, if non-nil, means a partial uniqueness constraint. Expression predicate = 4 [(gogoproto.customname) = "Predicate"]; + // IndexIDForValidation is the index id to hint to the unique_without_index + // constraint validation SQL query about which index to validate against. + // It is used exclusively by sql.validateUniqueConstraint. + uint32 index_id_for_validation = 5 [(gogoproto.customname) = "IndexIDForValidation", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.IndexID"]; } message CheckConstraint { diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index d9e5205dfaa5..1963a5aee765 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -81,6 +81,7 @@ UniqueWithoutIndexConstraint : TableID UniqueWithoutIndexConstraint : ConstraintID UniqueWithoutIndexConstraint : []ColumnIDs UniqueWithoutIndexConstraint : Predicate +UniqueWithoutIndexConstraint : IndexIDForValidation object CheckConstraint diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go index 085943adf809..5a822e52cc87 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_unique_without_index_constraint.go @@ -19,19 +19,71 @@ func init() { opRegistry.register((*scpb.UniqueWithoutIndexConstraint)(nil), toPublic( scpb.Status_ABSENT, + to(scpb.Status_WRITE_ONLY, + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.MakeAbsentUniqueWithoutIndexConstraintWriteOnly { + return &scop.MakeAbsentUniqueWithoutIndexConstraintWriteOnly{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + ColumnIDs: this.ColumnIDs, + Predicate: this.Predicate, + } + }), + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.UpdateTableBackReferencesInTypes { + if this.Predicate == nil || this.Predicate.UsesTypeIDs == nil || len(this.Predicate.UsesTypeIDs) == 0 { + return nil + } + return &scop.UpdateTableBackReferencesInTypes{ + TypeIDs: this.Predicate.UsesTypeIDs, + BackReferencedTableID: this.TableID, + } + }), + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.UpdateBackReferencesInSequences { + if this.Predicate == nil || this.Predicate.UsesSequenceIDs == nil || len(this.Predicate.UsesSequenceIDs) == 0 { + return nil + } + return &scop.UpdateBackReferencesInSequences{ + SequenceIDs: this.Predicate.UsesSequenceIDs, + BackReferencedTableID: this.TableID, + } + }), + ), + to(scpb.Status_VALIDATED, + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.ValidateConstraint { + return &scop.ValidateConstraint{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + IndexIDForValidation: this.IndexIDForValidation, + } + }), + ), to(scpb.Status_PUBLIC, - emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.NotImplemented { - return notImplemented(this) + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.MakeValidatedUniqueWithoutIndexConstraintPublic { + return &scop.MakeValidatedUniqueWithoutIndexConstraintPublic{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + } }), ), ), toAbsent( scpb.Status_PUBLIC, + to(scpb.Status_VALIDATED, + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.MakePublicUniqueWithoutIndexConstraintValidated { + return &scop.MakePublicUniqueWithoutIndexConstraintValidated{ + 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.UniqueWithoutIndexConstraint) *scop.NotImplemented { - return notImplemented(this) + emit(func(this *scpb.UniqueWithoutIndexConstraint) *scop.RemoveUniqueWithoutIndexConstraint { + return &scop.RemoveUniqueWithoutIndexConstraint{ + TableID: this.TableID, + ConstraintID: this.ConstraintID, + } }), ), ), From c7b299369d921854bbbad5ee6d6c7aa1a70e35e2 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Thu, 15 Dec 2022 17:05:29 -0500 Subject: [PATCH 3/5] scbuildstmt: Implement `ADD UNIQUE WITHOUT INDEX` in the builder --- .../scbuildstmt/alter_table_add_constraint.go | 112 +++++++++++++++++- 1 file changed, 108 insertions(+), 4 deletions(-) diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go index a97b6b581cc5..059ad0f33894 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_add_constraint.go @@ -41,6 +41,8 @@ func alterTableAddConstraint( case *tree.UniqueConstraintTableDef: if d.PrimaryKey && t.ValidationBehavior == tree.ValidationDefault { alterTableAddPrimaryKey(b, tn, tbl, t) + } else if d.WithoutIndex && t.ValidationBehavior == tree.ValidationDefault { + alterTableAddUniqueWithoutIndex(b, tn, tbl, t) } case *tree.CheckConstraintTableDef: if t.ValidationBehavior == tree.ValidationDefault { @@ -373,9 +375,110 @@ func alterTableAddForeignKey( }) } -// getFullyResolvedColNames returns fully resolved column names. -// For each name in `colNames`, its fully resolved name will be "db.sc.tbl.col". -// The order of column names in the return is in syc with the input `colNames`. +// alterTableAddUniqueWithoutIndex contains logic for building ALTER TABLE ... ADD CONSTRAINT ... UNIQUE WITHOUT INDEX. +// It assumes `t` is such a command. +func alterTableAddUniqueWithoutIndex( + b BuildCtx, tn *tree.TableName, tbl *scpb.Table, t *tree.AlterTableAddConstraint, +) { + d := t.ConstraintDef.(*tree.UniqueConstraintTableDef) + + // 1. A bunch of checks. + if !b.SessionData().EnableUniqueWithoutIndexConstraints { + panic(pgerror.New(pgcode.FeatureNotSupported, + "unique constraints without an index are not yet supported", + )) + } + if len(d.Storing) > 0 { + panic(pgerror.New(pgcode.FeatureNotSupported, + "unique constraints without an index cannot store columns", + )) + } + if d.PartitionByIndex.ContainsPartitions() { + panic(pgerror.New(pgcode.FeatureNotSupported, + "partitioned unique constraints without an index are not supported", + )) + } + if d.NotVisible { + // Theoretically, this should never happen because this is not supported by + // the parser. This is just a safe check. + panic(pgerror.Newf(pgcode.FeatureNotSupported, + "creating a unique constraint using UNIQUE WITH NOT VISIBLE INDEX is not supported", + )) + } + + // 2. Check that columns that we want to have uniqueness should have no duplicate. + var colSet catalog.TableColSet + var colIDs []catid.ColumnID + var colNames []string + for _, col := range d.Columns { + colID := getColumnIDFromColumnName(b, tbl.TableID, col.Column) + if colSet.Contains(colID) { + panic(pgerror.Newf(pgcode.DuplicateColumn, + "column %q appears twice in unique constraint", col.Column)) + } + colSet.Add(colID) + colIDs = append(colIDs, colID) + colNames = append(colNames, string(col.Column)) + } + + // 3. If a name is provided, check that this name is not used; Otherwise, generate + // a unique name for it. + if skip, err := validateConstraintNameIsNotUsed(b, tn, tbl, t); err != nil { + panic(err) + } else if skip { + return + } + if d.Name == "" { + d.Name = tree.Name(tabledesc.GenerateUniqueName( + fmt.Sprintf("unique_%s", strings.Join(colNames, "_")), + func(name string) bool { + return constraintNameInUse(b, tbl.TableID, name) + }, + )) + } + + // 4. If there is a predicate, validate it. + if d.Predicate != nil { + predicate, _, _, err := schemaexpr.DequalifyAndValidateExprImpl(b, d.Predicate, types.Bool, "unique without index predicate", b.SemaCtx(), volatility.Immutable, tn, + func() colinfo.ResultColumns { + return getNonDropResultColumns(b, tbl.TableID) + }, + func(columnName tree.Name) (exists bool, accessible bool, id catid.ColumnID, typ *types.T) { + return columnLookupFn(b, tbl.TableID, columnName) + }, + ) + if err != nil { + panic(err) + } + typedPredicate, err := parser.ParseExpr(predicate) + if err != nil { + panic(err) + } + d.Predicate = typedPredicate + } + + // 5. (Finally!) Add a UniqueWithoutIndex, ConstraintName element to builder state. + constraintID := b.NextTableConstraintID(tbl.TableID) + uwi := &scpb.UniqueWithoutIndexConstraint{ + TableID: tbl.TableID, + ConstraintID: constraintID, + ColumnIDs: colIDs, + IndexIDForValidation: getIndexIDForValidationForConstraint(b, tbl.TableID), + } + if d.Predicate != nil { + uwi.Predicate = b.WrapExpression(tbl.TableID, d.Predicate) + } + b.Add(uwi) + b.Add(&scpb.ConstraintWithoutIndexName{ + TableID: tbl.TableID, + ConstraintID: constraintID, + Name: string(d.Name), + }) +} + +// getFullyResolvedColNames returns fully resolved column names for `colNames`. +// For each column name in `colNames`, its fully resolved name will be "db.sc.tbl.col". +// The order of column names in the return is in syc with that in the input `colNames`. func getFullyResolvedColNames( b BuildCtx, tableID catid.DescID, colNames tree.NameList, ) (ret tree.NameList) { @@ -433,7 +536,8 @@ func validateConstraintNameIsNotUsed( name = d.Name ifNotExists = d.IfNotExists case *tree.UniqueConstraintTableDef: - panic(scerrors.NotImplementedErrorf(t, "UNIQUE constraint %v not yet implemented", d.Name)) + name = d.Name + ifNotExists = d.IfNotExists default: return false, errors.AssertionFailedf( "unsupported constraint: %T", t.ConstraintDef) From 82d4c7cdffb80d637c8975451343bcea04e11f47 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Mon, 19 Dec 2022 14:55:13 -0500 Subject: [PATCH 4/5] rewrite: Rewrite `UniqueWithoutIndex` on mutation slice during restore --- pkg/sql/catalog/rewrite/rewrite.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pkg/sql/catalog/rewrite/rewrite.go b/pkg/sql/catalog/rewrite/rewrite.go index a495b193841c..6e6b529a4b1c 100644 --- a/pkg/sql/catalog/rewrite/rewrite.go +++ b/pkg/sql/catalog/rewrite/rewrite.go @@ -173,6 +173,8 @@ func TableDescs( } } + // Rewrite unique_without_index in both `UniqueWithoutIndexConstraints` + // and `Mutations` slice. origUniqueWithoutIndexConstraints := table.UniqueWithoutIndexConstraints table.UniqueWithoutIndexConstraints = nil for _, unique := range origUniqueWithoutIndexConstraints { @@ -186,6 +188,18 @@ func TableDescs( "UniqueWithoutIndexConstraint %d was not found", table.Name, unique.TableID) } } + for idx := range table.Mutations { + if c := table.Mutations[idx].GetConstraint(); c != nil && + c.ConstraintType == descpb.ConstraintToUpdate_UNIQUE_WITHOUT_INDEX { + uwi := &c.UniqueWithoutIndexConstraint + if rewrite, ok := descriptorRewrites[uwi.TableID]; ok { + uwi.TableID = rewrite.ID + } else { + return errors.AssertionFailedf("cannot restore %q because referenced table ID in "+ + "UniqueWithoutIndexConstraint %d was not found", table.Name, uwi.TableID) + } + } + } if table.IsSequence() && table.SequenceOpts.HasOwner() { if ownerRewrite, ok := descriptorRewrites[table.SequenceOpts.SequenceOwner.OwnerTableID]; ok { From edad297aa6523d948d4f98cf9355f59a883f43e2 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Fri, 16 Dec 2022 17:05:52 -0500 Subject: [PATCH 5/5] schemachanger: Turn on `ADD UNIQUE WITHOUT INDEX` by default and add tests --- .../backup_base_generated_test.go | 5 + .../logictest/testdata/logic_test/alter_table | 9 +- pkg/sql/schema_changer_test.go | 1 + .../internal/scbuildstmt/alter_table.go | 11 +- .../scplan/internal/rules/helpers.go | 2 +- .../scplan/internal/rules/testdata/deprules | 225 +++++++++++++----- pkg/sql/schemachanger/sctest/end_to_end.go | 1 + .../schemachanger/sctest_generated_test.go | 25 ++ .../alter_table_add_unique_without_index | 146 ++++++++++++ .../alter_table_add_unique_without_index | 34 +++ ...e_add_unique_without_index.rollback_1_of_2 | 17 ++ ...e_add_unique_without_index.rollback_2_of_2 | 17 ++ .../alter_table_add_unique_without_index | 107 +++++++++ ...e_add_unique_without_index.rollback_1_of_2 | 41 ++++ ...e_add_unique_without_index.rollback_2_of_2 | 41 ++++ 15 files changed, 612 insertions(+), 70 deletions(-) create mode 100644 pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_1_of_2 create mode 100644 pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_2_of_2 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 create mode 100644 pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 diff --git a/pkg/ccl/schemachangerccl/backup_base_generated_test.go b/pkg/ccl/schemachangerccl/backup_base_generated_test.go index b66b7c6bf0a5..bc5052700eb5 100644 --- a/pkg/ccl/schemachangerccl/backup_base_generated_test.go +++ b/pkg/ccl/schemachangerccl/backup_base_generated_test.go @@ -58,6 +58,11 @@ func TestBackupbase_alter_table_add_primary_key_drop_rowid(t *testing.T) { defer log.Scope(t).Close(t) sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid", newCluster) } +func TestBackupbase_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Backup(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", newCluster) +} func TestBackupbase_alter_table_alter_primary_key_drop_rowid(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index c40eb0ce7da5..823f5b79054e 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -1560,7 +1560,9 @@ statement ok INSERT INTO unique_without_index (e, f) VALUES (1, 1), (1, 2) # But trying to add a unique constraint now fails. -statement error pgcode 23505 pq: could not create unique constraint "my_unique_e"\nDETAIL: Key \(e\)=\(1\) is duplicated\. +# Note that we omit the constraint name in the expected error message because if the declarative schema changer is used, +# the constraint name, at the time of validation failure, is still a place-holder name. +statement error pgcode 23505 pq: could not create unique constraint ".*"\nDETAIL: Key \(e\)=\(1\) is duplicated\. ALTER TABLE unique_without_index ADD CONSTRAINT my_unique_e UNIQUE WITHOUT INDEX (e) # We can create not-valid constraints, however. @@ -1603,11 +1605,14 @@ SELECT usage_count WHERE feature_name = 'sql.schema_changer.errors.constraint_violation'; # Trying to add a partial unique constraint fails when there are duplicates. -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 ADD CONSTRAINT uniq_a_1 UNIQUE WITHOUT INDEX (a) WHERE b > 0 OR c > 0 # Sanity: Check the number of user errors and # database errors in the test. +# We only do this check for the legacy schema changer because the declarative schema changer does not increment +# exactly the same counters. +onlyif config local-legacy-schema-changer query B SELECT usage_count > $constraint_violations_before FROM crdb_internal.feature_usage diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index d537004d8214..2b55d8505a65 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -5627,6 +5627,7 @@ func TestTableValidityWhileAddingUniqueConstraint(t *testing.T) { if _, err := sqlDB.Exec(` CREATE DATABASE t; CREATE TABLE t.tab (a INT PRIMARY KEY, b INT, c INT); +SET use_declarative_schema_changer = off; `); err != nil { t.Fatal(err) } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 644499dfcac7..959a85b3f4ec 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -54,6 +54,8 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ // Support ALTER TABLE ... ADD PRIMARY KEY if d, ok := t.ConstraintDef.(*tree.UniqueConstraintTableDef); ok && d.PrimaryKey && t.ValidationBehavior == tree.ValidationDefault { return true + } else if ok && d.WithoutIndex && t.ValidationBehavior == tree.ValidationDefault { + return true } // Support ALTER TABLE ... ADD CONSTRAINT CHECK @@ -75,9 +77,10 @@ var supportedAlterTableStatements = map[reflect.Type]supportedAlterTableCommand{ // They key is constructed as "ADD" + constraint type + validation behavior, joined with "_". // E.g. "ADD_PRIMARY_KEY_DEFAULT", "ADD_CHECK_SKIP", "ADD_FOREIGN_KEY_DEFAULT", etc. var alterTableAddConstraintMinSupportedClusterVersion = map[string]clusterversion.Key{ - "ADD_PRIMARY_KEY_DEFAULT": clusterversion.V22_2Start, - "ADD_CHECK_DEFAULT": clusterversion.V23_1Start, - "ADD_FOREIGN_KEY_DEFAULT": clusterversion.V23_1Start, + "ADD_PRIMARY_KEY_DEFAULT": clusterversion.V22_2Start, + "ADD_CHECK_DEFAULT": clusterversion.V23_1Start, + "ADD_FOREIGN_KEY_DEFAULT": clusterversion.V23_1Start, + "ADD_UNIQUE_WITHOUT_INDEX_DEFAULT": clusterversion.V23_1Start, } func init() { @@ -166,6 +169,8 @@ func alterTableAddConstraintSupportedInCurrentClusterVersion( case *tree.UniqueConstraintTableDef: if d.PrimaryKey { cmdKey = "ADD_PRIMARY_KEY" + } else if d.WithoutIndex { + cmdKey = "ADD_UNIQUE_WITHOUT_INDEX" } case *tree.CheckConstraintTableDef: cmdKey = "ADD_CHECK" diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 82fa65e55bb4..01e7523ba26d 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -375,7 +375,7 @@ func isIndexDependent(e scpb.Element) bool { // when we properly support adding/dropping them in the new schema changer. func isSupportedNonIndexBackedConstraint(e scpb.Element) bool { switch e.(type) { - case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint: + case *scpb.CheckConstraint, *scpb.ForeignKeyConstraint, *scpb.UniqueWithoutIndexConstraint: return true } return false diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index 6644c695a33d..e67794556d3e 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -1446,6 +1446,103 @@ deprules - descriptorIsNotBeingDropped($prev) - joinTargetNode($prev, $prev-target, $prev-node) - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = PUBLIC + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = VALIDATED + - $next-node[CurrentStatus] = ABSENT + - descriptorIsNotBeingDropped($prev) + - nodeNotExistsWithStatusIn_WRITE_ONLY($prev-target) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = ABSENT + - $prev-node[CurrentStatus] = WRITE_ONLY + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = ABSENT + - $next-node[CurrentStatus] = WRITE_ONLY + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = VALIDATED + - $next-node[CurrentStatus] = PUBLIC + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) +- name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' + from: prev-node + kind: PreviousTransactionPrecedence + to: next-node + query: + - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $next[Type] = '*scpb.UniqueWithoutIndexConstraint' + - $prev[DescID] = $_ + - $prev[Self] = $next + - $prev-target[Self] = $next-target + - $prev-target[TargetStatus] = PUBLIC + - $prev-node[CurrentStatus] = WRITE_ONLY + - $next-node[CurrentStatus] = VALIDATED + - descriptorIsNotBeingDropped($prev) + - joinTargetNode($prev, $prev-target, $prev-node) + - joinTargetNode($next, $next-target, $next-node) - name: column dependents exist before column becomes public from: dependent-node kind: Precedence @@ -1626,11 +1723,12 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - transient($constraint-target, $dependent-target) - - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED + - $constraint-target[TargetStatus] = ABSENT + - $constraint-node[CurrentStatus] = VALIDATED + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1639,12 +1737,11 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - $constraint-target[TargetStatus] = TRANSIENT_ABSENT - - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED - - $dependent-target[TargetStatus] = ABSENT + - toAbsent($constraint-target, $dependent-target) + - $constraint-node[CurrentStatus] = VALIDATED - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) @@ -1653,12 +1750,12 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - toAbsent($constraint-target, $dependent-target) - - $constraint-node[CurrentStatus] = VALIDATED - - $dependent-node[CurrentStatus] = ABSENT + - transient($constraint-target, $dependent-target) + - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: constraint no longer public before dependents @@ -1666,13 +1763,13 @@ deprules kind: Precedence to: dependent-node query: - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - $dependent[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - joinOnConstraintID($constraint, $dependent, $table-id, $constraint-id) - - $constraint-target[TargetStatus] = ABSENT - - $constraint-node[CurrentStatus] = VALIDATED - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $constraint-target[TargetStatus] = TRANSIENT_ABSENT + - $constraint-node[CurrentStatus] = TRANSIENT_VALIDATED + - $dependent-target[TargetStatus] = ABSENT + - $dependent-node[CurrentStatus] = ABSENT - joinTargetNode($constraint, $constraint-target, $constraint-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: dependent view absent before secondary index @@ -1683,10 +1780,10 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = ABSENT - - $view-node[CurrentStatus] = ABSENT - - $index-target[TargetStatus] = TRANSIENT_ABSENT - - $index-node[CurrentStatus] = TRANSIENT_ABSENT + - $view-target[TargetStatus] = TRANSIENT_ABSENT + - $view-node[CurrentStatus] = TRANSIENT_ABSENT + - $index-target[TargetStatus] = ABSENT + - $index-node[CurrentStatus] = ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view absent before secondary index @@ -1697,10 +1794,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = TRANSIENT_ABSENT + - transient($view-target, $index-target) - $view-node[CurrentStatus] = TRANSIENT_ABSENT - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = ABSENT + - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view absent before secondary index @@ -1724,8 +1820,9 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - transient($view-target, $index-target) - - $view-node[CurrentStatus] = TRANSIENT_ABSENT + - $view-target[TargetStatus] = ABSENT + - $view-node[CurrentStatus] = ABSENT + - $index-target[TargetStatus] = TRANSIENT_ABSENT - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) @@ -1737,10 +1834,10 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = ABSENT - - $view-node[CurrentStatus] = DROPPED - - $index-target[TargetStatus] = TRANSIENT_ABSENT - - $index-node[CurrentStatus] = TRANSIENT_VALIDATED + - $view-target[TargetStatus] = TRANSIENT_ABSENT + - $view-node[CurrentStatus] = TRANSIENT_DROPPED + - $index-target[TargetStatus] = ABSENT + - $index-node[CurrentStatus] = VALIDATED - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependent view no longer public before secondary index @@ -1777,10 +1874,10 @@ deprules - $view[Type] = '*scpb.View' - $index[Type] = '*scpb.SecondaryIndex' - viewReferencesIndex(*scpb.View, *scpb.SecondaryIndex)($view, $index) - - $view-target[TargetStatus] = TRANSIENT_ABSENT - - $view-node[CurrentStatus] = TRANSIENT_DROPPED - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = VALIDATED + - $view-target[TargetStatus] = ABSENT + - $view-node[CurrentStatus] = DROPPED + - $index-target[TargetStatus] = TRANSIENT_ABSENT + - $index-node[CurrentStatus] = TRANSIENT_VALIDATED - joinTargetNode($view, $view-target, $view-node) - joinTargetNode($index, $index-target, $index-node) - name: dependents removed before column @@ -1791,8 +1888,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - transient($dependent-target, $column-target) - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $dependent-target[TargetStatus] = ABSENT + - $dependent-node[CurrentStatus] = ABSENT + - $column-target[TargetStatus] = TRANSIENT_ABSENT - $column-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) @@ -1804,8 +1902,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - toAbsent($dependent-target, $column-target) - - $dependent-node[CurrentStatus] = ABSENT + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $column-target[TargetStatus] = ABSENT - $column-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) @@ -1817,10 +1916,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - transient($dependent-target, $column-target) - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $column-target[TargetStatus] = ABSENT - - $column-node[CurrentStatus] = ABSENT + - $column-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before column @@ -1831,10 +1929,9 @@ deprules - $dependent[Type] IN ['*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexColumn'] - $column[Type] = '*scpb.Column' - joinOnColumnID($dependent, $column, $table-id, $col-id) - - $dependent-target[TargetStatus] = ABSENT + - toAbsent($dependent-target, $column-target) - $dependent-node[CurrentStatus] = ABSENT - - $column-target[TargetStatus] = TRANSIENT_ABSENT - - $column-node[CurrentStatus] = TRANSIENT_ABSENT + - $column-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($column, $column-target, $column-node) - name: dependents removed before constraint @@ -1843,11 +1940,11 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - - toAbsent($dependents-target, $constraint-target) - - $dependents-node[CurrentStatus] = ABSENT - - $constraint-node[CurrentStatus] = ABSENT + - transient($dependents-target, $constraint-target) + - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT + - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependents, $dependents-target, $dependents-node) - joinTargetNode($constraint, $constraint-target, $constraint-node) - name: dependents removed before constraint @@ -1856,11 +1953,11 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - - transient($dependents-target, $constraint-target) - - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT - - $constraint-node[CurrentStatus] = TRANSIENT_ABSENT + - toAbsent($dependents-target, $constraint-target) + - $dependents-node[CurrentStatus] = ABSENT + - $constraint-node[CurrentStatus] = ABSENT - joinTargetNode($dependents, $dependents-target, $dependents-node) - joinTargetNode($constraint, $constraint-target, $constraint-node) - name: dependents removed before constraint @@ -1869,7 +1966,7 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - $dependents-target[TargetStatus] = TRANSIENT_ABSENT - $dependents-node[CurrentStatus] = TRANSIENT_ABSENT @@ -1883,7 +1980,7 @@ deprules to: constraint-node query: - $dependents[Type] IN ['*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment'] - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnConstraintID($dependents, $constraint, $table-id, $constraint-id) - $dependents-target[TargetStatus] = ABSENT - $dependents-node[CurrentStatus] = ABSENT @@ -1899,10 +1996,10 @@ deprules - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] - joinOnIndexID($dependent, $index, $table-id, $index-id) - - $dependent-target[TargetStatus] = TRANSIENT_ABSENT - - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT - - $index-target[TargetStatus] = ABSENT - - $index-node[CurrentStatus] = ABSENT + - $dependent-target[TargetStatus] = ABSENT + - $dependent-node[CurrentStatus] = ABSENT + - $index-target[TargetStatus] = TRANSIENT_ABSENT + - $index-node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($index, $index-target, $index-node) - name: dependents removed before index @@ -1939,10 +2036,10 @@ deprules - $dependent[Type] IN ['*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn'] - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] - joinOnIndexID($dependent, $index, $table-id, $index-id) - - $dependent-target[TargetStatus] = ABSENT - - $dependent-node[CurrentStatus] = ABSENT - - $index-target[TargetStatus] = TRANSIENT_ABSENT - - $index-node[CurrentStatus] = TRANSIENT_ABSENT + - $dependent-target[TargetStatus] = TRANSIENT_ABSENT + - $dependent-node[CurrentStatus] = TRANSIENT_ABSENT + - $index-target[TargetStatus] = ABSENT + - $index-node[CurrentStatus] = ABSENT - joinTargetNode($dependent, $dependent-target, $dependent-node) - joinTargetNode($index, $index-target, $index-node) - name: descriptor DROPPED in transaction before removal @@ -1977,7 +2074,7 @@ deprules to: dependent-node query: - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] - joinOnDescID($descriptor, $dependent, $desc-id) - toAbsent($descriptor-target, $dependent-target) - $descriptor-node[CurrentStatus] = DROPPED @@ -1991,7 +2088,7 @@ deprules to: referencing-via-attr-node query: - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.UniqueWithoutIndexConstraint', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - $referencing-via-attr[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] - joinReferencedDescID($referencing-via-attr, $referenced-descriptor, $desc-id) - toAbsent($referenced-descriptor-target, $referencing-via-attr-target) - $referenced-descriptor-node[CurrentStatus] = DROPPED @@ -2099,7 +2196,7 @@ deprules to: constraint-node query: - $index[Type] = '*scpb.PrimaryIndex' - - $constraint[Type] IN ['*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] - joinOnDescID($index, $constraint, $table-id) - $index[IndexID] = $index-id-for-validation - $constraint[IndexID] = $index-id-for-validation diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 39124144f136..1fd1d6cc8aee 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -184,6 +184,7 @@ func EndToEndSideEffects(t *testing.T, relPath string, newCluster NewClusterFunc // changer will allow non-fully implemented operations. sd.NewSchemaChangerMode = sessiondatapb.UseNewSchemaChangerUnsafe sd.ApplicationName = "" + sd.EnableUniqueWithoutIndexConstraints = true // this allows `ADD UNIQUE WITHOUT INDEX` in the testing suite. })), sctestdeps.WithTestingKnobs(&scexec.TestingKnobs{ BeforeStage: func(p scplan.Plan, stageIdx int) error { diff --git a/pkg/sql/schemachanger/sctest_generated_test.go b/pkg/sql/schemachanger/sctest_generated_test.go index 236392efb499..90d219265fc6 100644 --- a/pkg/sql/schemachanger/sctest_generated_test.go +++ b/pkg/sql/schemachanger/sctest_generated_test.go @@ -220,6 +220,31 @@ func TestRollback_alter_table_add_primary_key_drop_rowid(t *testing.T) { defer log.Scope(t).Close(t) sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid", sctest.SingleNodeCluster) } +func TestEndToEndSideEffects_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.EndToEndSideEffects(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", sctest.SingleNodeCluster) +} +func TestExecuteWithDMLInjection_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.ExecuteWithDMLInjection(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", sctest.SingleNodeCluster) +} +func TestGenerateSchemaChangeCorpus_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.GenerateSchemaChangeCorpus(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", sctest.SingleNodeCluster) +} +func TestPause_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Pause(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", sctest.SingleNodeCluster) +} +func TestRollback_alter_table_add_unique_without_index(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + sctest.Rollback(t, "pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index", sctest.SingleNodeCluster) +} func TestEndToEndSideEffects_alter_table_alter_primary_key_drop_rowid(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index new file mode 100644 index 000000000000..8e3e93ca513e --- /dev/null +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index @@ -0,0 +1,146 @@ +setup +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; +---- +... ++object {100 101 t} -> 104 + +test +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +---- +begin transaction #1 +# begin StatementPhase +checking for feature: ALTER TABLE +increment telemetry for sql.schema.alter_table +increment telemetry for sql.schema.alter_table.add_constraint +## StatementPhase stage 1 of 1 with 1 MutationType op +upsert descriptor #104 + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: UNIQUE_WITHOUT_INDEX + + foreignKey: {} + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: + + columnIds: + + - 2 + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + tableId: 104 + + validity: Validating + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +# end StatementPhase +# begin PreCommitPhase +## PreCommitPhase stage 1 of 1 with 2 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + + declarativeSchemaChangerState: + + authorization: + + userName: root + + currentStatuses: + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONSTRAINT ‹unique_j› + + UNIQUE WITHOUT INDEX (‹j›) + + statement: ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j) + + statementTag: ALTER TABLE + + revertible: true + + targetRanks: + + targets: + families: + - columnIds: + ... + time: {} + unexposedParentSchemaId: 101 + - version: "1" + + version: "2" +create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ADD CONSTRAINT unique_j UNIQUE WITHOUT INDEX (j)" + descriptor IDs: [104] +# end PreCommitPhase +commit transaction #1 +notified job registry to adopt jobs: [1] +# begin PostCommitPhase +begin transaction #2 +commit transaction #2 +begin transaction #3 +## PostCommitPhase stage 1 of 2 with 1 ValidationType op +validate UNIQUE WITHOUT INDEX constraint crdb_internal_constraint_2_name_placeholder in table #104 +commit transaction #3 +begin transaction #4 +## PostCommitPhase stage 2 of 2 with 4 MutationType ops +upsert descriptor #104 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - declarativeSchemaChangerState: + - authorization: + - userName: root + - currentStatuses: + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONSTRAINT ‹unique_j› + - UNIQUE WITHOUT INDEX (‹j›) + - statement: ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j) + - statementTag: ALTER TABLE + - revertible: true + - targetRanks: + - targets: + families: + - columnIds: + ... + id: 104 + modificationTime: {} + - mutations: + - - constraint: + - check: {} + - constraintType: UNIQUE_WITHOUT_INDEX + - foreignKey: {} + - name: crdb_internal_constraint_2_name_placeholder + - uniqueWithoutIndexConstraint: + - columnIds: + - - 2 + - constraintId: 2 + - name: crdb_internal_constraint_2_name_placeholder + - tableId: 104 + - validity: Validating + - direction: ADD + - mutationId: 1 + - state: WRITE_ONLY + name: t + nextColumnId: 3 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "2" + + uniqueWithoutIndexConstraints: + + - columnIds: + + - 2 + + constraintId: 2 + + name: unique_j + + tableId: 104 + + version: "3" +update progress of schema change job #1: "all stages completed" +set schema change job #1 to non-cancellable +updated schema change job #1 descriptor IDs to [] +commit transaction #4 +# end PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index new file mode 100644 index 000000000000..7a9dc00094c2 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index @@ -0,0 +1,34 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +EXPLAIN (ddl) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +---- +Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ └── 1 Mutation operation + │ └── MakeAbsentUniqueWithoutIndexConstraintWriteOnly {"ConstraintID":2,"TableID":104} + ├── PreCommitPhase + │ └── Stage 1 of 1 in PreCommitPhase + │ └── 2 Mutation operations + │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} + │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} + └── PostCommitPhase + ├── Stage 1 of 2 in PostCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── WRITE_ONLY → VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ └── 1 Validation operation + │ └── ValidateConstraint {"ConstraintID":2,"TableID":104} + └── Stage 2 of 2 in PostCommitPhase + ├── 2 elements transitioning toward PUBLIC + │ ├── VALIDATED → PUBLIC UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + └── 4 Mutation operations + ├── SetConstraintName {"ConstraintID":2,"Name":"unique_j","TableID":104} + ├── MakeValidatedUniqueWithoutIndexConstraintPublic {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_1_of_2 new file mode 100644 index 000000000000..aaef8a063504 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_1_of_2 @@ -0,0 +1,17 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +EXPLAIN (ddl) rollback at post-commit stage 1 of 2; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── WRITE_ONLY → ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + └── 3 Mutation operations + ├── RemoveUniqueWithoutIndexConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_2_of_2 new file mode 100644 index 000000000000..92066dcd8338 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index.rollback_2_of_2 @@ -0,0 +1,17 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +EXPLAIN (ddl) rollback at post-commit stage 2 of 2; +---- +Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); + └── PostCommitNonRevertiblePhase + └── Stage 1 of 1 in PostCommitNonRevertiblePhase + ├── 1 element transitioning toward ABSENT + │ └── WRITE_ONLY → ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + └── 3 Mutation operations + ├── RemoveUniqueWithoutIndexConstraint {"ConstraintID":2,"TableID":104} + ├── RemoveJobStateFromDescriptor {"DescriptorID":104} + └── UpdateSchemaChangerJob {"IsNonCancelable":true,"RunningStatus":"all stages compl..."} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index new file mode 100644 index 000000000000..cd5de74ba90e --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index @@ -0,0 +1,107 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +EXPLAIN (ddl, verbose) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +---- +• Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); +│ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ │ +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousTransactionPrecedence dependency from ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" +│ │ +│ └── • 1 Mutation operation +│ │ +│ └── • MakeAbsentUniqueWithoutIndexConstraintWriteOnly +│ ColumnIDs: +│ - 2 +│ ConstraintID: 2 +│ TableID: 104 +│ +├── • PreCommitPhase +│ │ +│ └── • Stage 1 of 1 in PreCommitPhase +│ │ +│ └── • 2 Mutation operations +│ │ +│ ├── • SetJobStateOnDescriptor +│ │ DescriptorID: 104 +│ │ Initialize: true +│ │ +│ └── • CreateSchemaChangerJob +│ Authorization: +│ UserName: root +│ DescriptorIDs: +│ - 104 +│ JobID: 1 +│ RunningStatus: PostCommitPhase stage 1 of 2 with 1 ValidationType op pending +│ Statements: +│ - statement: ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j) +│ redactedstatement: ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONSTRAINT ‹unique_j› +│ UNIQUE WITHOUT INDEX (‹j›) +│ statementtag: ALTER TABLE +│ +└── • PostCommitPhase + │ + ├── • Stage 1 of 2 in PostCommitPhase + │ │ + │ ├── • 1 element transitioning toward PUBLIC + │ │ │ + │ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ │ WRITE_ONLY → VALIDATED + │ │ │ + │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • 1 Validation operation + │ │ + │ └── • ValidateConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + └── • Stage 2 of 2 in PostCommitPhase + │ + ├── • 2 elements transitioning toward PUBLIC + │ │ + │ ├── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ │ VALIDATED → PUBLIC + │ │ │ + │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" + │ │ │ + │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ │ rule: "constraint dependent public right before constraint" + │ │ + │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ ABSENT → PUBLIC + │ + └── • 4 Mutation operations + │ + ├── • SetConstraintName + │ ConstraintID: 2 + │ Name: unique_j + │ TableID: 104 + │ + ├── • MakeValidatedUniqueWithoutIndexConstraintPublic + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 new file mode 100644 index 000000000000..730d7667f052 --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 @@ -0,0 +1,41 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ WRITE_ONLY → ABSENT + │ │ + │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 3 Mutation operations + │ + ├── • RemoveUniqueWithoutIndexConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 new file mode 100644 index 000000000000..3ca04f084fff --- /dev/null +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 @@ -0,0 +1,41 @@ +/* setup */ +CREATE TABLE t (i INT PRIMARY KEY, j INT); +SET experimental_enable_unique_without_index_constraints = true; + +/* test */ +ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); +EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; +---- +• Schema change plan for rolling back ALTER TABLE ‹defaultdb›.public.‹t› ADD CONSTRAINT ‹unique_j› UNIQUE WITHOUT INDEX (‹j›); +│ +└── • PostCommitNonRevertiblePhase + │ + └── • Stage 1 of 1 in PostCommitNonRevertiblePhase + │ + ├── • 1 element transitioning toward ABSENT + │ │ + │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ WRITE_ONLY → ABSENT + │ │ + │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" + │ │ + │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} + │ rule: "dependents removed before constraint" + │ + └── • 3 Mutation operations + │ + ├── • RemoveUniqueWithoutIndexConstraint + │ ConstraintID: 2 + │ TableID: 104 + │ + ├── • RemoveJobStateFromDescriptor + │ DescriptorID: 104 + │ JobID: 1 + │ + └── • UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + RunningStatus: all stages completed