From e54ddc07bc371d568533b0a8e9508f052028d119 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Mon, 27 Feb 2023 13:34:37 +0000 Subject: [PATCH 1/3] backupccl: move vmodule read out of loop Release note: none. Epic: none. --- pkg/ccl/backupccl/restore_data_processor.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index ec93faae17b9..e98a10bc0c02 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -459,6 +459,9 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( } defer batcher.Close(ctx) + // Read log.V once first to avoid the vmodule mutex in the tight loop below. + verbose := log.V(5) + var keyScratch, valueScratch []byte startKeyMVCC, endKeyMVCC := storage.MVCCKey{Key: entry.Span.Key}, @@ -499,7 +502,7 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( if !ok { // If the key rewriter didn't match this key, it's not data for the // table(s) we're interested in. - if log.V(5) { + if verbose { log.Infof(ctx, "skipping %s %s", key.Key, value.PrettyPrint()) } continue @@ -509,7 +512,7 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( value.ClearChecksum() value.InitChecksum(key.Key) - if log.V(5) { + if verbose { log.Infof(ctx, "Put %s -> %s", key.Key, value.PrettyPrint()) } if err := batcher.AddMVCCKey(ctx, key, value.RawBytes); err != nil { From 8c6cf287788eb1c4d5789e56d65db3c33a491efb Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Thu, 23 Feb 2023 11:20:15 -0500 Subject: [PATCH 2/3] sql, schemachanger: Support dropping UWI with dependent inbound FKs Previously, when a UWI constraint is used to serve an inbound FK and we drop the UWI constraint, the FK is not dropped. This causes a corrupt state where a FK constraint exists but the referenced table does not ensure uniqueness on the referenced columns. This commit fixes this issue in both legacy and declarative schema changer. --- pkg/sql/alter_table.go | 7 + .../logictest/testdata/logic_test/alter_table | 57 +++++++ .../alter_table_drop_constraint.go | 27 ++++ .../alter_table_validate_constraint.go | 13 ++ .../internal/scbuildstmt/drop_index.go | 147 +++++++++--------- 5 files changed, 180 insertions(+), 71 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index ec2072d5a4f2..97c9a5eb2aae 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -508,6 +508,13 @@ func (n *alterTableNode) startExec(params runParams) error { } return sqlerrors.NewUndefinedConstraintError(string(t.Constraint), n.tableDesc.Name) } + if uwoi := c.AsUniqueWithoutIndex(); uwoi != nil { + if err := params.p.tryRemoveFKBackReferences( + params.ctx, n.tableDesc, uwoi, t.DropBehavior, true, + ); err != nil { + return err + } + } if err := n.tableDesc.DropConstraint( c, func(backRef catalog.ForeignKeyConstraint) error { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 6c87228e43ae..e646d8890d6b 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -2940,3 +2940,60 @@ ALTER TABLE t_96648 DROP CONSTRAINT check_i, DROP CONSTRAINT check_i1; statement ok DROP TABLE t_96648 + +# This subtests ensures that dropping unique and unique without index +# constraints with dependent FKs works properly. +subtest drop_constraint_with_dependent_FKs_97538 + +statement ok +CREATE TABLE t_97538_2 (i INT PRIMARY KEY, j INT); + +statement ok +SET experimental_enable_unique_without_index_constraints = true + +statement ok +ALTER TABLE t_97538_2 ADD UNIQUE WITHOUT INDEX (j); + +statement ok +CREATE UNIQUE INDEX t_97538_2_j_key on t_97538_2(j); + +statement ok +CREATE TABLE t_97538_1 (i INT PRIMARY KEY REFERENCES t_97538_2(j)); + +statement ok +DROP INDEX t_97538_2_j_key; + +# Ensure that only the unique index is dropped but not the FK +# because we have replacement (the other unique without index constraint). +query TTTTB colnames +SHOW CONSTRAINTS FROM t_97538_1 +---- +table_name constraint_name constraint_type details validated +t_97538_1 t_97538_1_i_fkey FOREIGN KEY FOREIGN KEY (i) REFERENCES t_97538_2(j) true +t_97538_1 t_97538_1_pkey PRIMARY KEY PRIMARY KEY (i ASC) true + +# Add back the unique index, and drop the unique without index constraint (without CASCADE) this time. +statement ok +CREATE UNIQUE INDEX t_97538_2_j_key on t_97538_2(j); + +statement ok +ALTER TABLE t_97538_2 DROP CONSTRAINT unique_j; + +# Similarly, ensure that only the constraint is dropped but not the FK +# because we have replacement (the other unique index). +query TTTTB colnames +SHOW CONSTRAINTS FROM t_97538_1 +---- +table_name constraint_name constraint_type details validated +t_97538_1 t_97538_1_i_fkey FOREIGN KEY FOREIGN KEY (i) REFERENCES t_97538_2(j) true +t_97538_1 t_97538_1_pkey PRIMARY KEY PRIMARY KEY (i ASC) true + +statement ok +DROP INDEX t_97538_2_j_key CASCADE; + +# Now ensure that the FK is dropped as a result of the cascade. +query TTTTB colnames +SHOW CONSTRAINTS FROM t_97538_1 +---- +table_name constraint_name constraint_type details validated +t_97538_1 t_97538_1_pkey PRIMARY KEY PRIMARY KEY (i ASC) true diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go index ebe98c861ca0..82b5af6d9828 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_constraint.go @@ -11,10 +11,12 @@ package scbuildstmt import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" @@ -43,11 +45,36 @@ func alterTableDropConstraint( // Dropping UNIQUE constraint: error out as not implemented. droppingUniqueConstraintNotImplemented(constraintElems, t) + _, _, constraintNameElem := scpb.FindConstraintWithoutIndexName(constraintElems) + constraintID := constraintNameElem.ConstraintID constraintElems.ForEachElementStatus(func( _ scpb.Status, _ scpb.TargetStatus, e scpb.Element, ) { b.Drop(e) }) + + // UniqueWithoutIndex constraints can serve inbound FKs, and hence we might + // need to drop those dependent FKs if cascade. + maybeDropAdditionallyForUniqueWithoutIndexConstraint(b, tbl.TableID, constraintID, + constraintNameElem.Name, t.DropBehavior) +} + +func maybeDropAdditionallyForUniqueWithoutIndexConstraint( + b BuildCtx, + tableID catid.DescID, + maybeUWIConstraintID catid.ConstraintID, + constraintName string, + behavior tree.DropBehavior, +) { + uwiElem := retrieveUniqueWithoutIndexConstraintElem(b, tableID, maybeUWIConstraintID) + if uwiElem == nil { + return + } + maybeDropDependentFKConstraints(b, tableID, uwiElem.ConstraintID, constraintName, behavior, + func(fkReferencedColIDs []catid.ColumnID) bool { + return uwiElem.Predicate == nil && + descpb.ColumnIDs(uwiElem.ColumnIDs).PermutationOf(fkReferencedColIDs) + }) } func fallBackIfDroppingPrimaryKey( diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_validate_constraint.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_validate_constraint.go index 9342fdff1538..f010c6811635 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_validate_constraint.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_validate_constraint.go @@ -80,6 +80,19 @@ func retrieveUniqueWithoutIndexConstraintUnvalidatedElem( return UniqueWithoutIndexConstraintUnvalidatedElem } +func retrieveUniqueWithoutIndexConstraintElem( + b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID, +) (UniqueWithoutIndexConstraintElem *scpb.UniqueWithoutIndexConstraint) { + scpb.ForEachUniqueWithoutIndexConstraint(b.QueryByID(tableID), func( + current scpb.Status, target scpb.TargetStatus, e *scpb.UniqueWithoutIndexConstraint, + ) { + if e.ConstraintID == constraintID { + UniqueWithoutIndexConstraintElem = e + } + }) + return UniqueWithoutIndexConstraintElem +} + func retrieveForeignKeyConstraintUnvalidatedElem( b BuildCtx, tableID catid.DescID, constraintID catid.ConstraintID, ) (ForeignKeyConstraintUnvalidatedElem *scpb.ForeignKeyConstraintUnvalidated) { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go index 82080bfc8e66..6bd976f75cb1 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go @@ -215,53 +215,63 @@ func maybeDropDependentFunctions( }) } -// maybeDropDependentFKConstraints attempts to drop all FK constraints -// that depend on the to be dropped index if CASCADE. -// A FK constraint can only exist if there is `PRIMARY KEY`, `UNIQUE`, -// or `UNIQUE WITHOUT INDEX` constraint on the referenced columns in the child table. -// This is relevant if we're dropping a unique index whose `UNIQUE` -// constraints serves some FK constraints from other tables. In this case, -// we attempt to find a replacement constraint to serve this FK constraint. -// If we can, then we can proceed to drop the index. Otherwise, we will need -// to drop the FK constraint as well (if CASCADE of course). -// Panic if there is a dependent FK constraint, no replacement is found -// but drop behavior is not CASCADE. +// maybeDropDependentFKConstraints attempts to drop all inbound, dependent FK constraints, +// as a result of dropping a uniqueness-providing constraint. +// +// If we find a uniqueness-providing replacement, then we don't drop those FKs. +// If we don't and `behavior` is not CASCADE, then we panic. func maybeDropDependentFKConstraints( b BuildCtx, - toBeDroppedIndex *scpb.SecondaryIndex, - toBeDroppedIndexName *tree.TableIndexName, - dropBehavior tree.DropBehavior, + tableID catid.DescID, + toBeDroppedConstraintID catid.ConstraintID, + toBeDroppedConstraintName string, + behavior tree.DropBehavior, + canToBeDroppedConstraintServeFK func([]catid.ColumnID) bool, ) { - scpb.ForEachForeignKeyConstraint(b.BackReferences(toBeDroppedIndex.TableID), func( - current scpb.Status, target scpb.TargetStatus, e *scpb.ForeignKeyConstraint, - ) { - if !isIndexUniqueAndCanServeFK(b, &toBeDroppedIndex.Index, e.ReferencedColumnIDs) || - uniqueConstraintHasReplacementCandidate(b, toBeDroppedIndex.TableID, - toBeDroppedIndex, e.ReferencedColumnIDs) { - return - } + // shouldDropFK returns true if it is a dependent FK and no uniqueness-providing + // replacement can be found. + shouldDropFK := func(fkReferencedColIDs []catid.ColumnID) bool { + return canToBeDroppedConstraintServeFK(fkReferencedColIDs) && + !hasColsUniquenessConstraintOtherThan(b, tableID, fkReferencedColIDs, toBeDroppedConstraintID) + } - // This foreign key constraint references the table that the - // to-be-dropped index belongs, and such a FK constraint is - // served by this to-be-dropped index (i.e. the to-be-dropped - // index is a unique index that provides a unique constraint on - // the FK referenced columns). - // We also tried but cannot find a "replacement" index to serve - // this dependent FK (i.e. the primary index or another unique - // secondary index that also covers the same columns this FK - // references), so, we will need to remove the dependent FK - // constraint as well. - if dropBehavior != tree.DropCascade { - _, _, ns := scpb.FindNamespace(b.QueryByID(e.TableID)) - panic(fmt.Errorf("%q is referenced by foreign key from table %q", toBeDroppedIndexName.Index, ns.Name)) + // ensureCascadeBehavior panics if behavior is not cascade. + ensureCascadeBehavior := func(fkOriginTableID catid.DescID) { + if behavior != tree.DropCascade { + _, _, fkOriginTableName := scpb.FindNamespace(b.QueryByID(fkOriginTableID)) + panic(sqlerrors.NewUniqueConstraintReferencedByForeignKeyError( + toBeDroppedConstraintName, fkOriginTableName.Name)) } + } - b.BackReferences(toBeDroppedIndex.TableID).Filter(hasConstraintIDAttrFilter(e.ConstraintID)). + // dropDependentFKConstraint is a helper function that drops a dependent + // FK constraint with ID `fkConstraintID`. + dropDependentFKConstraint := func(fkConstraintID catid.ConstraintID) { + b.BackReferences(tableID).Filter(hasConstraintIDAttrFilter(fkConstraintID)). ForEachElementStatus(func( current scpb.Status, target scpb.TargetStatus, e scpb.Element, ) { b.Drop(e) }) + } + + b.BackReferences(tableID).ForEachElementStatus(func( + current scpb.Status, target scpb.TargetStatus, e scpb.Element, + ) { + switch t := e.(type) { + case *scpb.ForeignKeyConstraint: + if !shouldDropFK(t.ReferencedColumnIDs) { + return + } + ensureCascadeBehavior(t.TableID) + dropDependentFKConstraint(t.ConstraintID) + case *scpb.ForeignKeyConstraintUnvalidated: + if !shouldDropFK(t.ReferencedColumnIDs) { + return + } + ensureCascadeBehavior(t.TableID) + dropDependentFKConstraint(t.ConstraintID) + } }) } @@ -473,44 +483,39 @@ func isIndexUniqueAndCanServeFK( allKeyColIDsWithoutShardCol.PermutationOf(fkReferencedColIDs) } -// uniqueConstraintHasReplacementCandidate returns true if table ensures -// uniqueness on columns `referencedColumnIDs` through something other -// than the unique secondary index `sie`. +// hasColsUniquenessConstraintOtherThan returns true if the table ensures +// uniqueness on `columnIDs` through a constraint other than `otherThan`. +// Uniqueness can be ensured through PK, UNIQUE, or UNIQUE WITHOUT INDEX. // -// Uniqueness can be ensured through a unique primary index, -// a unique secondary index, or a unique without index constraint. -func uniqueConstraintHasReplacementCandidate( - b BuildCtx, tableID descpb.ID, sie *scpb.SecondaryIndex, referencedColumnIDs []descpb.ColumnID, -) bool { - result := false - - // Check all indexes (both primary and secondary) to see if we can find a replacement candidate. - // Also check all unique_without_indexes to see if we can find a replace - b.QueryByID(tableID).ForEachElementStatus(func( - current scpb.Status, target scpb.TargetStatus, e scpb.Element, - ) { - if result { - return - } - - switch t := e.(type) { - case *scpb.SecondaryIndex: - if t.IndexID != sie.IndexID && isIndexUniqueAndCanServeFK(b, &t.Index, referencedColumnIDs) { - result = true - } - case *scpb.PrimaryIndex: - if isIndexUniqueAndCanServeFK(b, &t.Index, referencedColumnIDs) { - result = true +// This function can be used to determine, e.g., whether a certain +// uniqueness-providing constraint has a replacement when we want to drop it; +// If yes, we don't need to drop any dependent inbound FKs if not cascade. +func hasColsUniquenessConstraintOtherThan( + b BuildCtx, tableID descpb.ID, columnIDs []descpb.ColumnID, otherThan descpb.ConstraintID, +) (ret bool) { + b.QueryByID(tableID).Filter(publicTargetFilter).Filter(statusPublicFilter). + ForEachElementStatus(func( + current scpb.Status, target scpb.TargetStatus, e scpb.Element, + ) { + if ret { + return } - case *scpb.UniqueWithoutIndexConstraint: - if descpb.ColumnIDs(t.ColumnIDs).PermutationOf(referencedColumnIDs) { - result = true + switch t := e.(type) { + case *scpb.PrimaryIndex: + if t.ConstraintID != otherThan && isIndexUniqueAndCanServeFK(b, &t.Index, columnIDs) { + ret = true + } + case *scpb.SecondaryIndex: + if t.ConstraintID != otherThan && isIndexUniqueAndCanServeFK(b, &t.Index, columnIDs) { + ret = true + } + case *scpb.UniqueWithoutIndexConstraint: + if t.ConstraintID != otherThan && descpb.ColumnIDs(t.ColumnIDs).PermutationOf(columnIDs) { + ret = true + } } - - } - }) - - return result + }) + return ret } func isExpressionIndexColumn(b BuildCtx, ce *scpb.Column) bool { From f84bd02aef41c99f9426dcb46ce2bc6287107233 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Thu, 23 Feb 2023 11:56:28 -0500 Subject: [PATCH 3/3] schemachanger: Support drop column with dependent constraint Previously, we fallback to legacy schema changer if we drop a column that is referenced in a constraint. This commit enables this behavior in the declarative schema changer. --- .../logictest/testdata/logic_test/alter_table | 67 +++++++++++++++++++ pkg/sql/pgwire/testdata/pgtest/notice | 2 +- .../scbuildstmt/alter_table_drop_column.go | 54 ++++++++------- .../internal/scbuildstmt/drop_index.go | 6 +- .../testdata/unimplemented_alter_table | 16 ----- .../internal/rules/current/dep_drop_column.go | 2 +- .../internal/rules/current/testdata/deprules | 16 ++--- 7 files changed, 112 insertions(+), 51 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index e646d8890d6b..6fd5474b587d 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -2997,3 +2997,70 @@ SHOW CONSTRAINTS FROM t_97538_1 ---- table_name constraint_name constraint_type details validated t_97538_1 t_97538_1_pkey PRIMARY KEY PRIMARY KEY (i ASC) true + +# This subtest ensures that dropping a column that is referenced +# by constraints works properly. +subtest drop_column_with_dependent_FKs_96727 + +statement ok +CREATE TABLE t_96727_2 (i INT PRIMARY KEY, j INT UNIQUE, FAMILY (i, j)); + +statement ok +SET experimental_enable_unique_without_index_constraints = true + +statement ok +ALTER TABLE t_96727_2 ADD UNIQUE WITHOUT INDEX (j); + +statement ok +CREATE TABLE t_96727_1 (i INT PRIMARY KEY REFERENCES t_96727_2(j)); + +# Here, either the unique index or the unique without index constraint +# is used for the output of error message is okay, so we used ".*". +statement error pq: ".*" is referenced by foreign key from table "t_96727_1" +ALTER TABLE t_96727_2 DROP COLUMN j; + +statement ok +ALTER TABLE t_96727_2 DROP COLUMN j CASCADE; + +# Ensure that the unique index, unique without index constraint, as well +# as the FK constraint are all dropped as a result of dropping the column. +query TT +SHOW CREATE TABLE t_96727_2 +---- +t_96727_2 CREATE TABLE public.t_96727_2 ( + i INT8 NOT NULL, + CONSTRAINT t_96727_2_pkey PRIMARY KEY (i ASC), + FAMILY fam_0_i_j (i) + ) + +query TT +SHOW CREATE TABLE t_96727_1 +---- +t_96727_1 CREATE TABLE public.t_96727_1 ( + i INT8 NOT NULL, + CONSTRAINT t_96727_1_pkey PRIMARY KEY (i ASC) + ) + +# More testing when the column is referenced in other constraints. +statement ok +ALTER TABLE t_96727_2 ADD COLUMN j INT; + +statement ok +ALTER TABLE t_96727_2 ADD CHECK (j > 0), ADD CHECK (j < 10) NOT VALID, ADD UNIQUE WITHOUT INDEX (j); + +statement ok +CREATE UNIQUE INDEX idx ON t_96727_2(j); + +statement ok +ALTER TABLE t_96727_2 DROP COLUMN j; + +# Ensure dropping the column, even without CASCADE, will also +# drop all the constraints. +query TT +SHOW CREATE TABLE t_96727_2 +---- +t_96727_2 CREATE TABLE public.t_96727_2 ( + i INT8 NOT NULL, + CONSTRAINT t_96727_2_pkey PRIMARY KEY (i ASC), + FAMILY fam_0_i_j (i) + ) diff --git a/pkg/sql/pgwire/testdata/pgtest/notice b/pkg/sql/pgwire/testdata/pgtest/notice index b5541f8e87c4..e60704db2a91 100644 --- a/pkg/sql/pgwire/testdata/pgtest/notice +++ b/pkg/sql/pgwire/testdata/pgtest/notice @@ -55,7 +55,7 @@ Query {"String": "DROP INDEX t_x_idx"} until crdb_only CommandComplete ---- -{"Severity":"NOTICE","SeverityUnlocalized":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":65,"Routine":"DropIndex","UnknownFields":null} +{"Severity":"NOTICE","SeverityUnlocalized":"NOTICE","Code":"00000","Message":"the data for dropped indexes is reclaimed asynchronously","Detail":"","Hint":"The reclamation delay can be customized in the zone configuration for the table.","Position":0,"InternalPosition":0,"InternalQuery":"","Where":"","SchemaName":"","TableName":"","ColumnName":"","DataTypeName":"","ConstraintName":"","File":"drop_index.go","Line":66,"Routine":"DropIndex","UnknownFields":null} {"Type":"CommandComplete","CommandTag":"DROP INDEX"} until noncrdb_only diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go index ab20dfb6d22d..77931bc83b85 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go @@ -212,25 +212,6 @@ func dropColumn( Index: tree.UnrestrictedName(indexName.Name), } dropSecondaryIndex(b, &name, behavior, e) - case *scpb.UniqueWithoutIndexConstraint: - // TODO(ajwerner): Support dropping UNIQUE WITHOUT INDEX constraints. - panic(errors.Wrap(scerrors.NotImplementedError(n), - "dropping of UNIQUE WITHOUT INDEX constraints not supported")) - case *scpb.CheckConstraint: - // TODO(ajwerner): Support dropping CHECK constraints. - // We might need to extend and add check constraint to dep-rule - // "column constraint removed right before column reaches delete only" - // in addition to just `b.Drop(e)`. Read its comment for more details. - panic(errors.Wrap(scerrors.NotImplementedError(n), - "dropping of CHECK constraints not supported")) - case *scpb.ForeignKeyConstraint: - if e.TableID != col.TableID && behavior != tree.DropCascade { - panic(pgerror.Newf(pgcode.DependentObjectsStillExist, - "cannot drop column %s because other objects depend on it", cn.Name)) - } - // TODO(ajwerner): Support dropping FOREIGN KEY constraints. - panic(errors.Wrap(scerrors.NotImplementedError(n), - "dropping of FOREIGN KEY constraints not supported")) case *scpb.View: if behavior != tree.DropCascade { _, _, ns := scpb.FindNamespace(b.QueryByID(col.TableID)) @@ -273,6 +254,22 @@ func dropColumn( ) } dropCascadeDescriptor(b, e.FunctionID) + case *scpb.UniqueWithoutIndexConstraint: + constraintElems := b.QueryByID(e.TableID).Filter(hasConstraintIDAttrFilter(e.ConstraintID)) + _, _, constraintName := scpb.FindConstraintWithoutIndexName(constraintElems.Filter(publicTargetFilter)) + alterTableDropConstraint(b, tn, tbl, &tree.AlterTableDropConstraint{ + IfExists: false, + Constraint: tree.Name(constraintName.Name), + DropBehavior: behavior, + }) + case *scpb.UniqueWithoutIndexConstraintUnvalidated: + constraintElems := b.QueryByID(e.TableID).Filter(hasConstraintIDAttrFilter(e.ConstraintID)) + _, _, constraintName := scpb.FindConstraintWithoutIndexName(constraintElems.Filter(publicTargetFilter)) + alterTableDropConstraint(b, tn, tbl, &tree.AlterTableDropConstraint{ + IfExists: false, + Constraint: tree.Name(constraintName.Name), + DropBehavior: behavior, + }) default: b.Drop(e) } @@ -312,11 +309,10 @@ func walkDropColumnDependencies(b BuildCtx, col *scpb.Column, fn func(e scpb.Ele Filter(referencesColumnIDFilter(col.ColumnID)). ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) { switch elt := e.(type) { - case *scpb.Column, *scpb.ColumnName, *scpb.ColumnComment, *scpb.ColumnNotNull: - fn(e) - case *scpb.ColumnDefaultExpression, *scpb.ColumnOnUpdateExpression: - fn(e) - case *scpb.UniqueWithoutIndexConstraint, *scpb.CheckConstraint: + case *scpb.Column, *scpb.ColumnName, *scpb.ColumnComment, *scpb.ColumnNotNull, + *scpb.ColumnDefaultExpression, *scpb.ColumnOnUpdateExpression, + *scpb.UniqueWithoutIndexConstraint, *scpb.CheckConstraint, + *scpb.UniqueWithoutIndexConstraintUnvalidated, *scpb.CheckConstraintUnvalidated: fn(e) case *scpb.ColumnType: if elt.ColumnID == col.ColumnID { @@ -341,6 +337,16 @@ func walkDropColumnDependencies(b BuildCtx, col *scpb.Column, fn func(e scpb.Ele catalog.MakeTableColSet(elt.ReferencedColumnIDs...).Contains(col.ColumnID) { fn(e) } + case *scpb.ForeignKeyConstraintUnvalidated: + if elt.TableID == col.TableID && + catalog.MakeTableColSet(elt.ColumnIDs...).Contains(col.ColumnID) { + fn(e) + } else if elt.ReferencedTableID == col.TableID && + catalog.MakeTableColSet(elt.ReferencedColumnIDs...).Contains(col.ColumnID) { + fn(e) + } + default: + panic(errors.AssertionFailedf("unknown column-dependent element type %T", elt)) } }) tblElts.ForEachElementStatus(func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) { diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go index 6bd976f75cb1..05a6956b1dea 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/errors" @@ -138,7 +139,10 @@ func dropSecondaryIndex( // In this case, if we were to drop 'ui' and no other unique constraint can be // found to replace 'uc' (to continue to serve 'fk'), we will require CASCADE //and drop 'fk' as well. - maybeDropDependentFKConstraints(next, sie, indexName, dropBehavior) + maybeDropDependentFKConstraints(b, sie.TableID, sie.ConstraintID, string(indexName.Index), dropBehavior, + func(fkReferencedColIDs []catid.ColumnID) bool { + return isIndexUniqueAndCanServeFK(b, &sie.Index, fkReferencedColIDs) + }) // If shard index, also drop the shard column and all check constraints that // uses this shard column if no other index uses the shard column. diff --git a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table index ce3df8f60b28..022d17f0a6ae 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/unimplemented_alter_table @@ -20,22 +20,6 @@ unimplemented ALTER TABLE defaultdb.foo ALTER COLUMN i SET DATA TYPE STRING ---- -unimplemented -ALTER TABLE defaultdb.foo DROP COLUMN k ----- - -unimplemented -ALTER TABLE defaultdb.foo DROP COLUMN l CASCADE; ----- - -unimplemented -ALTER TABLE defaultdb.foo DROP COLUMN m ----- - -unimplemented -ALTER TABLE defaultdb.foo DROP COLUMN n ----- - unimplemented ALTER TABLE defaultdb.foo DROP COLUMN o, ADD COLUMN p INT ---- diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go index 9d34e0f96b16..8bf945c81425 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_column.go @@ -116,7 +116,7 @@ func init() { scpb.Status_ABSENT, scpb.Status_DELETE_ONLY, func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.Type((*scpb.ColumnNotNull)(nil)), + from.TypeFilter(rulesVersionKey, isNonIndexBackedConstraint, isSubjectTo2VersionInvariant), to.Type((*scpb.Column)(nil)), JoinOnColumnID(from, to, "table-id", "col-id"), } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index 0e0c016ea81d..3f25cefcdc44 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -1547,7 +1547,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - toAbsent($column-constraint-Target, $column-Target) @@ -1560,7 +1560,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - transient($column-constraint-Target, $column-Target) @@ -1573,7 +1573,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - $column-constraint-Target[TargetStatus] = TRANSIENT_ABSENT @@ -1587,7 +1587,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - $column-constraint-Target[TargetStatus] = ABSENT @@ -4895,7 +4895,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - toAbsent($column-constraint-Target, $column-Target) @@ -4908,7 +4908,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - transient($column-constraint-Target, $column-Target) @@ -4921,7 +4921,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - $column-constraint-Target[TargetStatus] = TRANSIENT_ABSENT @@ -4935,7 +4935,7 @@ deprules kind: SameStagePrecedence to: column-Node query: - - $column-constraint[Type] = '*scpb.ColumnNotNull' + - $column-constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint', '*scpb.ColumnNotNull'] - $column[Type] = '*scpb.Column' - joinOnColumnID($column-constraint, $column, $table-id, $col-id) - $column-constraint-Target[TargetStatus] = ABSENT