Skip to content

Commit

Permalink
sql,schemachanger:Disallow dropping index if it's referenced in UDF/View
Browse files Browse the repository at this point in the history
This commits disallows schema changes to a table that would drop an
index that is referenced explicitly via index hinting in a UDF or view
body.

Release note: None
  • Loading branch information
Xiang-Gu committed Aug 29, 2023
1 parent 5d00346 commit dcb2acf
Show file tree
Hide file tree
Showing 6 changed files with 130 additions and 3 deletions.
4 changes: 4 additions & 0 deletions pkg/sql/add_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,10 @@ func (p *planner) addColumnImpl(
)
}

if err := p.disallowDroppingPrimaryIndexReferencedInUDFOrView(params.ctx, desc); err != nil {
return err
}

var colOwnedSeqDesc *tabledesc.Mutable
newDef, seqPrefix, seqName, seqOpts, err := params.p.processSerialLikeInColumnDef(params.ctx, d, tn)
if err != nil {
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -67,6 +68,10 @@ func (p *planner) AlterPrimaryKey(
return err
}

if err := p.disallowDroppingPrimaryIndexReferencedInUDFOrView(ctx, tableDesc); err != nil {
return err
}

if alterPrimaryKeyLocalitySwap != nil {
if err := p.checkNoRegionChangeUnderway(
ctx,
Expand Down Expand Up @@ -804,3 +809,25 @@ func setKeySuffixColumnIDsFromPrimary(
}
return nil
}

// disallowDroppingPrimaryIndexReferencedInUDFOrView returns an non-nil error
// if current primary index is referenced explicitly in a UDF or view.
// This is used for ADD COLUMN, DROP COLUMN, and ALTER PRIMARY KEY commands
// because their implementation could need to drop the old/current primary index
// and create new ones.
func (p *planner) disallowDroppingPrimaryIndexReferencedInUDFOrView(
ctx context.Context, tableDesc *tabledesc.Mutable,
) error {
currentPrimaryIndex := tableDesc.GetPrimaryIndex()
for _, tableRef := range tableDesc.DependedOnBy {
if tableRef.IndexID == currentPrimaryIndex.GetID() {
// canRemoveDependent with `DropDefault` will return the right error.
err := p.canRemoveDependent(
ctx, "index", currentPrimaryIndex.GetName(), tableDesc.ParentID, tableRef, tree.DropDefault)
if err != nil {
return errors.WithDetail(err, sqlerrors.PrimaryIndexSwapDetail)
}
}
}
return nil
}
4 changes: 4 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1623,6 +1623,10 @@ func dropColumnImpl(
)
}

if err := params.p.disallowDroppingPrimaryIndexReferencedInUDFOrView(params.ctx, tableDesc); err != nil {
return nil, err
}

// If the dropped column uses a sequence, remove references to it from that sequence.
if colToDrop.NumUsesSequences() > 0 {
if err := params.p.removeSequenceDependencies(params.ctx, tableDesc, colToDrop); err != nil {
Expand Down
47 changes: 47 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -3433,3 +3433,50 @@ set sql_safe_updates=false

statement error pgcode 42P10 column "n" is referenced by.*
alter table t_drop_cascade_with_key drop column n cascade;

subtest end

# This subtest ensures that if a schema change requires dropping an index,
# primary or secondary, and that index is referenced in a UDF body or a view
# query via index hinting, then we disallow such schema changes.
subtest 108974

statement ok
CREATE TABLE t_108974_f(i INT PRIMARY KEY, j INT NOT NULL, k INT, INDEX (i,j));
INSERT INTO t_108974_f SELECT p,p+1,p+2 FROM generate_series(1,100) AS tmp(p);
CREATE TABLE t_108974_v(i INT PRIMARY KEY, j INT NOT NULL, k INT, INDEX (i,j));
INSERT INTO t_108974_v SELECT p,p+1,p+2 FROM generate_series(1,100) AS tmp(p);
CREATE FUNCTION f_108974() RETURNS RECORD LANGUAGE SQL AS
$$
SELECT i, j FROM t_108974_f;
SELECT i, j FROM t_108974_f@t_108974_f_pkey;
SELECT i, j FROM t_108974_f@t_108974_f_i_j_idx;
SELECT i, j FROM t_108974_f@[0];
SELECT i, j FROM t_108974_f@[1];
SELECT i, j FROM t_108974_f@[2];
$$;
CREATE VIEW v_108974 AS SELECT i, j FROM t_108974_v@t_108974_v_pkey;
SET sql_safe_updates = false;

statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_i_j_idx" because function "f_108974" depends on it
DROP INDEX t_108974_f@t_108974_f_i_j_idx;

statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it
ALTER TABLE t_108974_f ALTER PRIMARY KEY USING COLUMNS (j);

statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it
ALTER TABLE t_108974_v ALTER PRIMARY KEY USING COLUMNS (j);

statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it
ALTER TABLE t_108974_f ADD COLUMN p INT DEFAULT 30;

statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it
ALTER TABLE t_108974_v ADD COLUMN p INT DEFAULT 30;

statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it
ALTER TABLE t_108974_f DROP COLUMN k;

statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it
ALTER TABLE t_108974_v DROP COLUMN k;

subtest end
40 changes: 40 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -180,6 +181,45 @@ func AlterTable(b BuildCtx, n *tree.AlterTable) {
}
maybeDropRedundantPrimaryIndexes(b, tbl.TableID)
maybeRewriteTempIDsInPrimaryIndexes(b, tbl.TableID)
disallowDroppingPrimaryIndexReferencedInUDFOrView(b, tbl.TableID, n.String())
}

// disallowDroppingPrimaryIndexReferencedInUDFOrView prevents dropping old (current)
// primary index that is referenced explicitly via index hinting in UDF or View body.
func disallowDroppingPrimaryIndexReferencedInUDFOrView(
b BuildCtx, tableID catid.DescID, stmtSQLString string,
) {
chain := getPrimaryIndexChain(b, tableID)
if !chain.isInflatedAtAll() {
// No new primary index needs to be added at all, which means old/current
// primary index does not need to be dropped.
return
}

toBeDroppedIndexID := chain.oldSpec.primary.IndexID
toBeDroppedIndexName := chain.oldSpec.name.Name
b.BackReferences(tableID).Filter(publicTargetFilter).ForEachTarget(func(target scpb.TargetStatus, e scpb.Element) {
switch el := e.(type) {
case *scpb.FunctionBody:
for _, ref := range el.UsesTables {
if ref.TableID == tableID && ref.IndexID == toBeDroppedIndexID {
fnName := b.QueryByID(el.FunctionID).FilterFunctionName().MustGetOneElement().Name
panic(errors.WithDetail(
sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "function", fnName),
sqlerrors.PrimaryIndexSwapDetail))
}
}
case *scpb.View:
for _, ref := range el.ForwardReferences {
if ref.ToID == tableID && ref.IndexID == toBeDroppedIndexID {
viewName := b.QueryByID(el.ViewID).FilterNamespace().MustGetOneElement().Name
panic(errors.WithDetail(
sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "view", viewName),
sqlerrors.PrimaryIndexSwapDetail))
}
}
}
})
}

// maybeRewriteTempIDsInPrimaryIndexes is part of the post-processing
Expand Down
11 changes: 8 additions & 3 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -1341,7 +1341,7 @@ func (pic *primaryIndexChain) inflate(b BuildCtx) {
// 7. (old != inter1 && inter1 != inter2 && inter2 == final), drop inter2
// 8. (old != inter1 && inter1 != inter2 && inter2 != final), do nothing
func (pic *primaryIndexChain) deflate(b BuildCtx) {
if !pic.isInflated() {
if !pic.isFullyInflated() {
return
}
tableID := pic.oldSpec.primary.TableID
Expand Down Expand Up @@ -1449,11 +1449,16 @@ func nonNilPrimaryIndexSpecSelector(spec *indexSpec) bool {
return spec.primary != nil
}

// isInflated return true if all new primary indexes are non-nil.
func (pic *primaryIndexChain) isInflated() bool {
// isFullyInflated return true if all new primary indexes are non-nil.
func (pic *primaryIndexChain) isFullyInflated() bool {
return pic.inter1Spec.primary != nil && pic.inter2Spec.primary != nil && pic.finalSpec.primary != nil
}

// isInflatedAtAll return true if any new primary index is non-nil.
func (pic *primaryIndexChain) isInflatedAtAll() bool {
return pic.inter1Spec.primary != nil || pic.inter2Spec.primary != nil || pic.finalSpec.primary != nil
}

// chainType returns the type of the chain.
func (pic *primaryIndexChain) chainType() (ret chainType) {
val := 0
Expand Down

0 comments on commit dcb2acf

Please sign in to comment.