Skip to content

Commit

Permalink
Merge #97579 #97708
Browse files Browse the repository at this point in the history
97579: schemachanger: Support dropping column with referenced constraints r=Xiang-Gu a=Xiang-Gu

This PR enables dropping columns with referenced constraints in the declarative schema changer.

As a prerequisite step, we also added support to dropping a UWI constraint when there is a dependent
FK constraint in both the legacy and declarative schema changer (commit 2).

Commit 2 Fixes: #96787, Fixes: #97538
Commit 3 Fixes: #96727

Epic: None

97708: backupccl: move vmodule read out of loop r=dt a=dt

Spotted in profiling by `@lidorcarmel.`

Release note: none.
Epic: none.

Co-authored-by: Xiang Gu <[email protected]>
Co-authored-by: David Taylor <[email protected]>
  • Loading branch information
3 people committed Feb 27, 2023
3 parents e0f858e + f84bd02 + e54ddc0 commit 7a3778b
Show file tree
Hide file tree
Showing 11 changed files with 297 additions and 124 deletions.
7 changes: 5 additions & 2 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
Expand Down Expand Up @@ -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
Expand All @@ -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 {
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
124 changes: 124 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -2940,3 +2940,127 @@ 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

# 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)
)
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/testdata/pgtest/notice
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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)
}
Expand Down Expand Up @@ -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 {
Expand All @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
Loading

0 comments on commit 7a3778b

Please sign in to comment.