Skip to content

Commit

Permalink
sql/schemachanger: add support for tracking function dependencies
Browse files Browse the repository at this point in the history
Previously, we did not allow user-defined functions to invoke other
user-defined functions due to a lack of dependency tracking. This patch
adds support for dependency tracking within the declarative schema
changer and includes an end-to-end test to confirm correct updates to
the descriptors.

Fixes: #119930
Epic: CRDB-19398
Release note: none
  • Loading branch information
fqazi committed Mar 6, 2024
1 parent dec11fa commit b896271
Show file tree
Hide file tree
Showing 29 changed files with 819 additions and 26 deletions.
28 changes: 28 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.

3 changes: 0 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/drop_function
Original file line number Diff line number Diff line change
Expand Up @@ -263,18 +263,15 @@ CREATE FUNCTION f_called_by_b2() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 + f_cal
statement ok;
CREATE FUNCTION f_b() RETURNS INT LANGUAGE SQL AS $$ SELECT (f_called_by_b() + f_called_by_b2()) /f_called_by_b2() $$;

onlyif config local-legacy-schema-changer
statement error pgcode 2BP01 cannot drop function \"f_called_by_b\" because other objects \(\[test.public.f_called_by_b2, test.public.f_b\]\) still depend on it
DROP FUNCTION f_called_by_b;

onlyif config local-legacy-schema-changer
statement error pgcode 2BP01 cannot drop function \"f_called_by_b2\" because other objects \(\[test.public.f_b\]\) still depend on it
DROP FUNCTION f_called_by_b2;

statement ok
CREATE OR REPLACE FUNCTION f_b() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$;

onlyif config local-legacy-schema-changer
statement error pgcode 2BP01 cannot drop function \"f_called_by_b\" because other objects \(\[test.public.f_called_by_b2\]\) still depend on it
DROP FUNCTION f_called_by_b;

Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/reference_provider.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ type tableDescReferences []descpb.TableDescriptor_Reference
type referenceProvider struct {
tableReferences map[descpb.ID]tableDescReferences
viewReferences map[descpb.ID]tableDescReferences
referencedFunctions catalog.DescriptorIDSet
referencedSequences catalog.DescriptorIDSet
referencedTypes catalog.DescriptorIDSet
allRelationIDs catalog.DescriptorIDSet
Expand Down Expand Up @@ -64,6 +65,16 @@ func (r *referenceProvider) ForEachTableReference(
return nil
}

// ForEachFunctionReference implements scbuildstmt.ReferenceProvider.
func (r *referenceProvider) ForEachFunctionReference(f func(functionID descpb.ID) error) error {
for _, functionID := range r.referencedFunctions.Ordered() {
if err := f(functionID); err != nil {
return err
}
}
return nil
}

// ForEachViewReference implements scbuildstmt.ReferenceProvider
func (r *referenceProvider) ForEachViewReference(
f func(viewID descpb.ID, colIDs descpb.ColumnIDs) error,
Expand Down Expand Up @@ -110,7 +121,7 @@ func (f *referenceProviderFactory) NewReferenceProvider(
// For the time being this is only used for CREATE FUNCTION. We need to handle
// CREATE VIEW when it's needed.
createFnExpr := optFactory.Memo().RootExpr().(*memo.CreateFunctionExpr)
tableReferences, typeReferences, _, err := toPlanDependencies(createFnExpr.Deps, createFnExpr.TypeDeps, createFnExpr.FuncDeps)
tableReferences, typeReferences, functionReferences, err := toPlanDependencies(createFnExpr.Deps, createFnExpr.TypeDeps, createFnExpr.FuncDeps)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -141,6 +152,9 @@ func (f *referenceProviderFactory) NewReferenceProvider(
}
}

for functionID := range functionReferences {
ret.referencedFunctions.Add(functionID)
}
return ret, nil
}

Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/schemachanger/scbuild/builder_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -1578,6 +1578,16 @@ func (b *builderState) WrapFunctionBody(
panic(err)
}

if err := refProvider.ForEachFunctionReference(func(id descpb.ID) error {
fnBody.UsesFunctions = append(fnBody.UsesFunctions,
scpb.FunctionBody_FunctionReference{
FunctionID: id,
})
return nil
}); err != nil {
panic(err)
}

fnBody.UsesSequenceIDs = refProvider.ReferencedSequences().Ordered()
fnBody.UsesTypeIDs = refProvider.ReferencedTypes().Ordered()
return fnBody
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,7 @@ func CreateFunction(b BuildCtx, n *tree.CreateRoutine) {
refProvider := b.BuildReferenceProvider(n)
validateTypeReferences(b, refProvider, db.DatabaseID)
validateFunctionRelationReferences(b, refProvider, db.DatabaseID)
validateFunctionToFunctionReferences(b, refProvider, db.DatabaseID)
b.Add(b.WrapFunctionBody(fnID, fnBodyStr, lang, refProvider))
b.LogEventForExistingTarget(&fn)
}
Expand All @@ -209,6 +210,30 @@ func validateFunctionRelationReferences(
}
}

// validateFunctionToFunctionReferences validates no function references are
// cross database.
func validateFunctionToFunctionReferences(
b BuildCtx, refProvider ReferenceProvider, parentDBID descpb.ID,
) {
err := refProvider.ForEachFunctionReference(func(id descpb.ID) error {
funcElts := b.QueryByID(id)
funcName := funcElts.FilterFunctionName().MustGetOneElement()
schemaParent := funcElts.FilterSchemaChild().MustGetOneElement()
schemaNamespace := b.QueryByID(schemaParent.SchemaID).FilterNamespace().MustGetOneElement()
if schemaNamespace.DatabaseID != parentDBID {
name := tree.MakeRoutineNameFromPrefix(b.NamePrefix(schemaNamespace), tree.Name(funcName.Name))
return pgerror.Newf(
pgcode.FeatureNotSupported,
"the function cannot refer to other databases",
name.String())
}
return nil
})
if err != nil {
panic(err)
}
}

func validateFunctionLeakProof(options tree.RoutineOptions, vp funcinfo.VolatilityProperties) {
if err := vp.Apply(options); err != nil {
panic(err)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -417,6 +417,9 @@ type ReferenceProvider interface {
// ForEachViewReference iterate through all referenced views and the reference
// details with the given function.
ForEachViewReference(f func(viewID descpb.ID, colIDs descpb.ColumnIDs) error) error
// ForEachFunctionReference iterates through all referenced functions for each
// function.
ForEachFunctionReference(f func(id descpb.ID) error) error
// ReferencedSequences returns all referenced sequence IDs
ReferencedSequences() catalog.DescriptorIDSet
// ReferencedTypes returns all referenced type IDs (not including implicit
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/schemachanger/scdecomp/decomp.go
Original file line number Diff line number Diff line change
Expand Up @@ -900,6 +900,9 @@ func (w *walkCtx) walkFunction(fnDesc catalog.FunctionDescriptor) {
}
}
}
for _, funcDep := range fnDesc.GetDependsOnFunctions() {
fnBody.UsesFunctions = append(fnBody.UsesFunctions, scpb.FunctionBody_FunctionReference{FunctionID: funcDep})
}
for _, backRef := range fnDesc.GetDependedOnBy() {
w.backRefs.Add(backRef.ID)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scdecomp/testdata/function
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ ElementState:
functionId: 110
lang:
lang: SQL
usesFunctions: []
usesSequenceIds:
- 105
usesTables:
Expand Down
31 changes: 31 additions & 0 deletions pkg/sql/schemachanger/scexec/scmutationexec/references.go
Original file line number Diff line number Diff line change
Expand Up @@ -451,6 +451,24 @@ func (i *immediateVisitor) RemoveBackReferencesInRelations(
return nil
}

func (i *immediateVisitor) RemoveBackReferenceInFunctions(
ctx context.Context, op scop.RemoveBackReferenceInFunctions,
) error {
for _, f := range op.FunctionIDs {
backRefFunc, err := i.checkOutFunction(ctx, f)
if err != nil {
return err
}
for i, dep := range backRefFunc.DependedOnBy {
if dep.ID == op.BackReferencedDescriptorID {
backRefFunc.DependedOnBy = append(backRefFunc.DependedOnBy[:i], backRefFunc.DependedOnBy[i+1:]...)
break
}
}
}
return nil
}

func removeViewBackReferencesInRelation(
ctx context.Context, m *immediateVisitor, relationID, backReferencedID descpb.ID,
) error {
Expand Down Expand Up @@ -515,6 +533,7 @@ func (i *immediateVisitor) UpdateFunctionRelationReferences(
}
relIDs := catalog.DescriptorIDSet{}
relIDToReferences := make(map[descpb.ID][]descpb.TableDescriptor_Reference)
functionIDs := catalog.DescriptorIDSet{}

for _, ref := range op.TableReferences {
relIDs.Add(ref.TableID)
Expand Down Expand Up @@ -544,6 +563,18 @@ func (i *immediateVisitor) UpdateFunctionRelationReferences(
relIDToReferences[seqID] = append(relIDToReferences[seqID], dep)
}

for _, functionRef := range op.FunctionReferences {
backRefFunc, err := i.checkOutFunction(ctx, functionRef.FunctionID)
if err != nil {
return err
}
if err := backRefFunc.AddFunctionReference(op.FunctionID); err != nil {
return err
}
functionIDs.Add(functionRef.FunctionID)
}
fn.DependsOnFunctions = functionIDs.Ordered()

for relID, refs := range relIDToReferences {
if err := updateBackReferencesInRelation(ctx, i, relID, op.FunctionID, refs); err != nil {
return err
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/schemachanger/scop/immediate_mutation.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,6 +530,13 @@ type RemoveBackReferenceInTypes struct {
TypeIDs []descpb.ID
}

type RemoveBackReferenceInFunctions struct {
immediateMutationOp

BackReferencedDescriptorID descpb.ID
FunctionIDs []descpb.ID
}

// UpdateTableBackReferencesInSequences updates back references to a table expression
// (in a column or a check constraint) in the specified sequences.
type UpdateTableBackReferencesInSequences struct {
Expand Down Expand Up @@ -807,10 +814,11 @@ type UpdateFunctionTypeReferences struct {

type UpdateFunctionRelationReferences struct {
immediateMutationOp
FunctionID descpb.ID
TableReferences []scpb.FunctionBody_TableReference
ViewReferences []scpb.FunctionBody_ViewReference
SequenceIDs []descpb.ID
FunctionID descpb.ID
TableReferences []scpb.FunctionBody_TableReference
ViewReferences []scpb.FunctionBody_ViewReference
SequenceIDs []descpb.ID
FunctionReferences []scpb.FunctionBody_FunctionReference
}

type SetObjectParentID struct {
Expand Down

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

5 changes: 5 additions & 0 deletions pkg/sql/schemachanger/scpb/elements.proto
Original file line number Diff line number Diff line change
Expand Up @@ -712,6 +712,10 @@ message FunctionBody {
repeated uint32 column_ids = 2 [(gogoproto.customname) = "ColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"];
}

message FunctionReference {
uint32 function_id = 1 [(gogoproto.customname) = "FunctionID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
}

uint32 function_id = 1 [(gogoproto.customname) = "FunctionID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
string body = 2;
cockroach.sql.catalog.catpb.FunctionLanguage lang = 3 [(gogoproto.nullable) = false];
Expand All @@ -720,6 +724,7 @@ message FunctionBody {
repeated ViewReference uses_views = 5 [(gogoproto.nullable) = false];
repeated uint32 uses_sequence_ids = 6 [(gogoproto.customname) = "UsesSequenceIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
repeated uint32 uses_type_ids = 7 [(gogoproto.customname) = "UsesTypeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"];
repeated FunctionReference uses_functions = 8 [(gogoproto.nullable) = false];
}

message FunctionParamDefaultExpression {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/schemachanger/scpb/uml/table.puml
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ FunctionBody : []UsesTables
FunctionBody : []UsesViews
FunctionBody : []UsesSequenceIDs
FunctionBody : []UsesTypeIDs
FunctionBody : []UsesFunctions

object FunctionLeakProof

Expand Down
22 changes: 17 additions & 5 deletions pkg/sql/schemachanger/scplan/internal/opgen/opgen_function_body.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,11 @@ func init() {
}),
emit(func(this *scpb.FunctionBody) *scop.UpdateFunctionRelationReferences {
return &scop.UpdateFunctionRelationReferences{
FunctionID: this.FunctionID,
TableReferences: this.UsesTables,
ViewReferences: this.UsesViews,
SequenceIDs: this.UsesSequenceIDs,
FunctionID: this.FunctionID,
TableReferences: this.UsesTables,
ViewReferences: this.UsesViews,
SequenceIDs: this.UsesSequenceIDs,
FunctionReferences: this.UsesFunctions,
}
}),
),
Expand Down Expand Up @@ -72,7 +73,18 @@ func init() {
BackReferencedID: this.FunctionID,
RelationIDs: relationIDs,
}
})),
}),
emit(func(this *scpb.FunctionBody) *scop.RemoveBackReferenceInFunctions {
functionIDs := make([]descpb.ID, 0, len(this.UsesFunctions))
for _, f := range this.UsesFunctions {
functionIDs = append(functionIDs, f.FunctionID)
}
return &scop.RemoveBackReferenceInFunctions{
BackReferencedDescriptorID: this.FunctionID,
FunctionIDs: functionIDs,
}
}),
),
),
)
}

0 comments on commit b896271

Please sign in to comment.