Skip to content

Commit

Permalink
sql/schemachanger: declarative drop function
Browse files Browse the repository at this point in the history
This commit implements DROP FUNCTION given existing
declarative schema changer infra (rules and ops).

`RemoveObjectParent` is added to handle the removal
of function signature from schema.

Release note: None
  • Loading branch information
chengxiong-ruan committed Jan 19, 2023
1 parent 9a0d8c3 commit b6d46c5
Show file tree
Hide file tree
Showing 65 changed files with 1,949 additions and 56 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -123,8 +123,7 @@ pq: cannot drop sequence sq1 because other objects depend on it
exec-sql
DROP TABLE sc1.tbl1
----
pq: cannot drop relation "tbl1" because function "f1" depends on it
HINT: you can drop f1 instead.
pq: cannot drop table tbl1 because other objects depend on it

exec-sql
ALTER TABLE sc1.tbl1 RENAME TO tbl1_new
Expand Down Expand Up @@ -274,8 +273,7 @@ pq: cannot drop sequence sq1 because other objects depend on it
exec-sql
DROP TABLE sc1.tbl1
----
pq: cannot drop relation "tbl1" because function "f1" depends on it
HINT: you can drop f1 instead.
pq: cannot drop table tbl1 because other objects depend on it

exec-sql
ALTER TABLE sc1.tbl1 RENAME TO tbl1_new
Expand Down
5 changes: 5 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

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

Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ add synthetic descriptor #108:
version: "1"
# end StatementPhase
# begin PreCommitPhase
## PreCommitPhase stage 1 of 1 with 22 MutationType ops
## PreCommitPhase stage 1 of 1 with 25 MutationType ops
delete database namespace entry {0 0 multi_region_test_db} -> 104
delete schema namespace entry {104 0 public} -> 105
delete object namespace entry {104 105 crdb_internal_region} -> 106
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ add synthetic descriptor #108:
version: "1"
# end StatementPhase
# begin PreCommitPhase
## PreCommitPhase stage 1 of 1 with 14 MutationType ops
## PreCommitPhase stage 1 of 1 with 15 MutationType ops
delete object namespace entry {104 105 table_regional_by_row} -> 108
upsert descriptor #106
type:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ add synthetic descriptor #108:
version: "1"
# end StatementPhase
# begin PreCommitPhase
## PreCommitPhase stage 1 of 1 with 9 MutationType ops
## PreCommitPhase stage 1 of 1 with 10 MutationType ops
delete object namespace entry {104 105 table_regional_by_table} -> 108
upsert descriptor #106
type:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,14 +81,17 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE;
│ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}
│ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}
│ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1}
│ └── 22 Mutation operations
│ └── 25 Mutation operations
│ ├── MarkDescriptorAsDropped {"DescriptorID":104}
│ ├── RemoveDatabaseRoleSettings {"DatabaseID":104}
│ ├── MarkDescriptorAsDropped {"DescriptorID":105}
│ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}}
│ ├── MarkDescriptorAsDropped {"DescriptorID":106}
│ ├── RemoveObjectParent {"ObjectID":106,"ParentSchemaID":105}
│ ├── MarkDescriptorAsDropped {"DescriptorID":107}
│ ├── RemoveObjectParent {"ObjectID":107,"ParentSchemaID":105}
│ ├── MarkDescriptorAsDropped {"DescriptorID":108}
│ ├── RemoveObjectParent {"ObjectID":108,"ParentSchemaID":105}
│ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108}
│ ├── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}}
│ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":105,"Name":"public"}}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,9 @@ Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹tab
│ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}
│ │ ├── PUBLIC → ABSENT IndexPartitioning:{DescID: 108, IndexID: 1}
│ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1}
│ └── 14 Mutation operations
│ └── 15 Mutation operations
│ ├── MarkDescriptorAsDropped {"DescriptorID":108}
│ ├── RemoveObjectParent {"ObjectID":108,"ParentSchemaID":105}
│ ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":108}
│ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":108}
│ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,9 @@ Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹tab
│ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}
│ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}
│ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1}
│ └── 9 Mutation operations
│ └── 10 Mutation operations
│ ├── MarkDescriptorAsDropped {"DescriptorID":108}
│ ├── RemoveObjectParent {"ObjectID":108,"ParentSchemaID":105}
│ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108}
│ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}}
│ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":108}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -564,7 +564,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ │ └── • skip PUBLIC → ABSENT operations
│ │ rule: "skip index dependents removal ops on relation drop"
│ │
│ └── • 22 Mutation operations
│ └── • 25 Mutation operations
│ │
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 104
Expand All @@ -583,12 +583,24 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE;
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 106
│ │
│ ├── • RemoveObjectParent
│ │ ObjectID: 106
│ │ ParentSchemaID: 105
│ │
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 107
│ │
│ ├── • RemoveObjectParent
│ │ ObjectID: 107
│ │ ParentSchemaID: 105
│ │
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 108
│ │
│ ├── • RemoveObjectParent
│ │ ObjectID: 108
│ │ ParentSchemaID: 105
│ │
│ ├── • RemoveBackReferenceInTypes
│ │ BackReferencedDescriptorID: 108
│ │ TypeIDs:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -361,11 +361,25 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_
│ │ └── • skip PUBLIC → ABSENT operations
│ │ rule: "skip index dependents removal ops on relation drop"
│ │
<<<<<<< HEAD
│ └── • 14 Mutation operations
=======
│ └── • 16 Mutation operations
>>>>>>> be3b07d938 (sql/schemachanger: declarative drop function)
│ │
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 108
│ │
<<<<<<< HEAD
=======
│ ├── • RemoveAllTableComments
│ │ TableID: 108
│ │
│ ├── • RemoveObjectParent
│ │ ObjectID: 108
│ │ ParentSchemaID: 105
│ │
>>>>>>> be3b07d938 (sql/schemachanger: declarative drop function)
│ ├── • RemoveColumnDefaultExpression
│ │ ColumnID: 2
│ │ TableID: 108
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,11 +261,15 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_
│ │ └── • skip PUBLIC → ABSENT operations
│ │ rule: "skip index dependents removal ops on relation drop"
│ │
│ └── • 9 Mutation operations
│ └── • 10 Mutation operations
│ │
│ ├── • MarkDescriptorAsDropped
│ │ DescriptorID: 108
│ │
│ ├── • RemoveObjectParent
│ │ ObjectID: 108
│ │ ParentSchemaID: 105
│ │
│ ├── • RemoveBackReferenceInTypes
│ │ BackReferencedDescriptorID: 108
│ │ TypeIDs:
Expand Down
35 changes: 23 additions & 12 deletions pkg/sql/drop_function_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ SELECT nextval(105:::REGCLASS);`,
err = sql.TestingDescsTxn(ctx, s, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error {
_, err := col.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Function(ctx, 109)
require.Error(t, err)
require.Regexp(t, "descriptor is being dropped", err.Error())
require.Regexp(t, "function undefined", err.Error())

// Make sure columns and indexes has correct back references.
tn := tree.MakeTableNameWithSchema("defaultdb", "public", "t")
Expand Down Expand Up @@ -197,6 +197,7 @@ CREATE TABLE t(
a INT PRIMARY KEY,
b INT,
C INT,
d INT,
INDEX t_idx_b(b),
INDEX t_idx_c(c)
);
Expand All @@ -209,6 +210,7 @@ CREATE FUNCTION test_sc.f(a notmyworkday) RETURNS INT IMMUTABLE LANGUAGE SQL AS
SELECT a FROM t;
SELECT b FROM t@t_idx_b;
SELECT c FROM t@t_idx_c;
SELECT d FROM t;
SELECT a FROM v;
SELECT nextval('sq1');
$$;
Expand All @@ -225,20 +227,23 @@ USE defaultdb;
tDB.Exec(t, "SET use_declarative_schema_changer = off;")

testCases := []struct {
stmt string
expectedErr string
stmt string
expectedErr string
dscExpectedErr string
}{
{
stmt: "DROP SEQUENCE sq1",
expectedErr: "pq: cannot drop sequence sq1 because other objects depend on it",
},
{
stmt: "DROP TABLE t",
expectedErr: `pq: cannot drop relation "t" because function "f" depends on it`,
stmt: "DROP TABLE t",
expectedErr: `pq: cannot drop relation "t" because function "f" depends on it`,
dscExpectedErr: `pq: cannot drop table t because other objects depend on it`,
},
{
stmt: "DROP VIEW v",
expectedErr: `pq: cannot drop relation "v" because function "f" depends on it`,
stmt: "DROP VIEW v",
expectedErr: `pq: cannot drop relation "v" because function "f" depends on it`,
dscExpectedErr: `pq: cannot drop view v because other objects depend on it`,
},
{
stmt: "ALTER TABLE t RENAME TO t_new",
Expand All @@ -249,8 +254,8 @@ USE defaultdb;
expectedErr: `pq: cannot set schema on relation "t" because function "f" depends on it`,
},
{
stmt: "ALTER TABLE t DROP COLUMN b",
expectedErr: `pq: cannot drop column "b" because function "f" depends on it`,
stmt: "ALTER TABLE t DROP COLUMN d",
expectedErr: `pq: cannot drop column "d" because function "f" depends on it`,
},
{
stmt: "ALTER TABLE t RENAME COLUMN b TO bb",
Expand Down Expand Up @@ -296,7 +301,11 @@ USE defaultdb;
for i, tc := range testCases {
t.Run(strconv.Itoa(i), func(t *testing.T) {
_, err := sqlDB.Exec(tc.stmt)
require.Equal(t, tc.expectedErr, err.Error())
if tc.dscExpectedErr != "" {
require.Equal(t, tc.dscExpectedErr, err.Error())
} else {
require.Equal(t, tc.expectedErr, err.Error())
}
})
}
}
Expand All @@ -312,6 +321,7 @@ CREATE TABLE t(
a INT PRIMARY KEY,
b INT,
C INT,
d INT,
INDEX t_idx_b(b),
INDEX t_idx_c(c)
);
Expand All @@ -324,6 +334,7 @@ CREATE FUNCTION test_sc.f(a notmyworkday) RETURNS INT IMMUTABLE LANGUAGE SQL AS
SELECT a FROM t;
SELECT b FROM t@t_idx_b;
SELECT c FROM t@t_idx_c;
SELECT d FROM t;
SELECT a FROM v;
SELECT nextval('sq1');
$$;
Expand All @@ -347,7 +358,7 @@ $$;
},
{
testName: "drop column",
stmt: "ALTER TABLE t DROP COLUMN b CASCADE",
stmt: "ALTER TABLE t DROP COLUMN d CASCADE",
},
{
testName: "drop index",
Expand Down Expand Up @@ -420,7 +431,7 @@ $$;
err = sql.TestingDescsTxn(ctx, s, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error {
_, err := col.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Function(ctx, 113)
require.Error(t, err)
require.Regexp(t, "descriptor is being dropped", err.Error())
require.Regexp(t, "function undefined", err.Error())
return nil
})
require.NoError(t, err)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scbuild/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/funcdesc",
"//pkg/sql/catalog/nstree",
"//pkg/sql/catalog/resolver",
"//pkg/sql/catalog/schemaexpr",
Expand Down
39 changes: 39 additions & 0 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
Expand Down Expand Up @@ -928,6 +929,44 @@ func (b *builderState) ResolveConstraint(
})
}

func (b *builderState) ResolveUDF(
fnObj *tree.FuncObj, p scbuildstmt.ResolveParams,
) scbuildstmt.ElementResultSet {
fd, err := b.cr.ResolveFunction(b.ctx, fnObj.FuncName.ToUnresolvedObjectName().ToUnresolvedName(), b.semaCtx.SearchPath)
if err != nil {
if p.IsExistenceOptional && errors.Is(err, tree.ErrFunctionUndefined) {
return nil
}
panic(err)
}

paramTypes, err := fnObj.ParamTypes(b.ctx, b.cr)
if err != nil {
return nil
}
ol, err := fd.MatchOverload(paramTypes, fnObj.FuncName.Schema(), b.semaCtx.SearchPath)
if err != nil {
if p.IsExistenceOptional && errors.Is(err, tree.ErrFunctionUndefined) {
return nil
}
panic(err)
}

if !ol.IsUDF {
panic(
errors.Errorf(
"cannot perform schema change on function %s%s because it is required by the database system",
fnObj.FuncName.Object(), ol.Signature(true),
),
)
}

fnID := funcdesc.UserDefinedFunctionOIDToID(ol.Oid)
b.mustOwn(fnID)
b.ensureDescriptor(fnID)
return b.descCache[fnID].ers
}

func (b *builderState) ensureDescriptor(id catid.DescID) {
if _, found := b.descCache[id]; found {
return
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"create_index.go",
"dependencies.go",
"drop_database.go",
"drop_function.go",
"drop_index.go",
"drop_owned_by.go",
"drop_schema.go",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,6 +260,15 @@ func dropColumn(
dropRestrictDescriptor(b, e.SequenceID)
undroppedSeqBackrefsToCheck.Add(e.SequenceID)
}
case *scpb.FunctionBody:
if behavior != tree.DropCascade {
_, _, fnName := scpb.FindFunctionName(b.QueryByID(e.FunctionID))
panic(sqlerrors.NewDependentObjectErrorf(
"cannot drop column %q because function %q depends on it",
cn.Name, fnName.Name),
)
}
dropCascadeDescriptor(b, e.FunctionID)
default:
b.Drop(e)
}
Expand Down Expand Up @@ -361,6 +370,12 @@ func walkDropColumnDependencies(b BuildCtx, col *scpb.Column, fn func(e scpb.Ele
catalog.MakeTableColSet(elt.ReferencedColumnIDs...).Contains(col.ColumnID) {
fn(e)
}
case *scpb.FunctionBody:
for _, ref := range elt.UsesTables {
if ref.TableID == col.TableID && catalog.MakeTableColSet(ref.ColumnIDs...).Contains(col.ColumnID) {
fn(e)
}
}
}
})
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -305,6 +305,9 @@ type NameResolver interface {
// ResolveIndex retrieves an index by name and returns its elements.
ResolveIndex(relationID catid.DescID, indexName tree.Name, p ResolveParams) ElementResultSet

// ResolveUDF retrieves a user defined function and returns its elements.
ResolveUDF(fnObj *tree.FuncObj, p ResolveParams) ElementResultSet

// ResolveIndexByName retrieves a table which contains the target
// index and returns its elements. Name of database, schema or table may be
// missing.
Expand Down
Loading

0 comments on commit b6d46c5

Please sign in to comment.