Skip to content

Commit

Permalink
sql: fix conflicting udf options validation
Browse files Browse the repository at this point in the history
This commit fixes a bug where function options can be conflicting
with each other because we forgot to add option type names into
the dedupe map. A helper is added to handle the validation.

This commit also fixes another bug in ALTER FUNCTION where leakproof
can be set for a non-immutable function. A helper is also added so
the logic can be reused.

Release note: None
Release justification: Fixing a bug that function options can be
conflicting with each other. Also Fixing a bug that leakproof can
be set for non-immutable function with ALTER FUNCTION
  • Loading branch information
chengxiong-ruan committed Aug 18, 2022
1 parent 04c6a1a commit e0ce0fc
Show file tree
Hide file tree
Showing 9 changed files with 203 additions and 32 deletions.
14 changes: 7 additions & 7 deletions pkg/sql/alter_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package sql

import (
"context"
"reflect"

"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -64,19 +63,20 @@ func (n *alterFunctionOptionsNode) startExec(params runParams) error {
// referenced by other objects. This is needed when want to allow function
// references. Need to think about in what condition a function can be altered
// or not.
options := make(map[string]struct{})
if err := tree.ValidateFuncOptions(n.n.Options); err != nil {
return err
}
for _, option := range n.n.Options {
optTypeName := reflect.TypeOf(option).Name()
if _, ok := options[optTypeName]; ok {
return pgerror.New(pgcode.Syntax, "conflicting or redundant options")
}
// Note that language and function body cannot be altered, and it's blocked
// from parser level with "common_func_opt_item" syntax.
err := setFuncOption(params, fnDesc, option)
if err != nil {
return err
}
options[optTypeName] = struct{}{}
}

if err := funcdesc.CheckLeakProofVolatility(fnDesc); err != nil {
return err
}

return params.p.writeFuncSchemaChange(params.ctx, fnDesc)
Expand Down
19 changes: 16 additions & 3 deletions pkg/sql/catalog/funcdesc/func_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
Expand Down Expand Up @@ -198,9 +200,7 @@ func (desc *immutable) ValidateSelf(vea catalog.ValidationErrorAccumulator) {
}
}

if desc.LeakProof && desc.Volatility != catpb.Function_IMMUTABLE {
vea.Report(errors.AssertionFailedf("leakproof is set for non-immutable function"))
}
vea.Report(CheckLeakProofVolatility(desc))

for i, dep := range desc.DependedOnBy {
if dep.ID == descpb.InvalidID {
Expand Down Expand Up @@ -658,3 +658,16 @@ func UserDefinedFunctionOIDToID(oid oid.Oid) (descpb.ID, error) {
func IsOIDUserDefinedFunc(oid oid.Oid) bool {
return catid.IsOIDUserDefined(oid)
}

// CheckLeakProofVolatility returns an error when a function is defined as
// leakproof but not immutable. See more details in comments for volatility.V.
func CheckLeakProofVolatility(fn catalog.FunctionDescriptor) error {
if fn.GetLeakProof() && fn.GetVolatility() != catpb.Function_IMMUTABLE {
return pgerror.Newf(
pgcode.InvalidFunctionDefinition,
"cannot set leakproof on function with non-immutable volatility: %s",
fn.GetVolatility().String(),
)
}
return nil
}
2 changes: 1 addition & 1 deletion pkg/sql/catalog/funcdesc/func_desc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ func TestValidateFuncDesc(t *testing.T) {
},
},
{
"leakproof is set for non-immutable function",
"cannot set leakproof on function with non-immutable volatility: VOLATILE",
descpb.FunctionDescriptor{
Name: "f",
ID: funcDescID,
Expand Down
17 changes: 5 additions & 12 deletions pkg/sql/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,12 +91,8 @@ func (n *createFunctionNode) createNewFunction(
return err
}
}
if udfDesc.LeakProof && udfDesc.Volatility != catpb.Function_IMMUTABLE {
return pgerror.Newf(
pgcode.InvalidFunctionDefinition,
"cannot create leakproof function with non-immutable volatility: %s",
udfDesc.Volatility.String(),
)
if err := funcdesc.CheckLeakProofVolatility(udfDesc); err != nil {
return err
}

if err := n.addUDFReferences(udfDesc, params); err != nil {
Expand Down Expand Up @@ -167,12 +163,9 @@ func (n *createFunctionNode) replaceFunction(udfDesc *funcdesc.Mutable, params r
return err
}
}
if udfDesc.LeakProof && udfDesc.Volatility != catpb.Function_IMMUTABLE {
return pgerror.Newf(
pgcode.InvalidFunctionDefinition,
"cannot create leakproof function with non-immutable volatility: %s",
udfDesc.Volatility.String(),
)

if err := funcdesc.CheckLeakProofVolatility(udfDesc); err != nil {
return err
}

// Removing all existing references before adding new references.
Expand Down
33 changes: 30 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -4,12 +4,36 @@ a INT PRIMARY KEY,
b INT
)

statement error pq: cannot create leakproof function with non-immutable volatility: STABLE
statement error pq: cannot set leakproof on function with non-immutable volatility: STABLE
CREATE FUNCTION f(a int) RETURNS INT LEAKPROOF STABLE LANGUAGE SQL AS 'SELECT 1'

statement error pq: return type mismatch in function declared to return int\nDETAIL: Actual return type is string
CREATE FUNCTION f() RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT 'hello' $$

statement error pq: STABLE: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT IMMUTABLE STABLE LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: STRICT: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT CALLED ON NULL INPUT STABLE STRICT LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: RETURNS NULL ON NULL INPUT: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT CALLED ON NULL INPUT STABLE RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: NOT LEAKPROOF: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT LEAKPROOF NOT LEAKPROOF LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: AS \$\$ SELECT 2 \$\$: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$ AS $$ SELECT 2 $$;

statement error pq: LANGUAGE SQL: conflicting or redundant options
CREATE FUNCTION f() RETURNS INT IMMUTABLE LANGUAGE SQL LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: no language specified
CREATE FUNCTION f() RETURNS INT IMMUTABLE AS $$ SELECT 1 $$;

statement error pq: no function body specified
CREATE FUNCTION f() RETURNS INT IMMUTABLE LANGUAGE SQL;

statement ok
CREATE FUNCTION a(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT i'

Expand Down Expand Up @@ -1017,9 +1041,12 @@ CREATE FUNCTION public.f_test_alter_opt(IN INT8)
SELECT 1;
$$

statement error pq: conflicting or redundant options
statement error pq: IMMUTABLE: conflicting or redundant options
ALTER FUNCTION f_test_alter_opt IMMUTABLE IMMUTABLE

statement error pq: cannot set leakproof on function with non-immutable volatility: STABLE
ALTER FUNCTION f_test_alter_opt STABLE LEAKPROOF

statement ok
ALTER FUNCTION f_test_alter_opt IMMUTABLE LEAKPROOF STRICT;

Expand Down Expand Up @@ -1370,7 +1397,7 @@ CREATE OR REPLACE FUNCTION f_test_cor(a INT, b INT) RETURNS STRING IMMUTABLE LAN
statement error pq: cannot change return type of existing function
CREATE OR REPLACE FUNCTION f_test_cor(a INT, b INT) RETURNS SETOF INT IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$;

statement error pq: cannot create leakproof function with non-immutable volatility: VOLATILE
statement error pq: cannot set leakproof on function with non-immutable volatility: VOLATILE
CREATE OR REPLACE FUNCTION f_test_cor(a INT, b INT) RETURNS INT LEAKPROOF LANGUAGE SQL AS $$ SELECT 1 $$;

query T
Expand Down
10 changes: 4 additions & 6 deletions pkg/sql/opt/optbuilder/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@
package optbuilder

import (
"reflect"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
Expand Down Expand Up @@ -61,16 +59,16 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateFunction, inScope *scope) (
panic(unimplemented.New("CREATE FUNCTION sql_body", "CREATE FUNCTION...sql_body unimplemented"))
}

if err := tree.ValidateFuncOptions(cf.Options); err != nil {
panic(err)
}

// Look for function body string from function options.
// Note that function body can be an empty string.
funcBodyFound := false
languageFound := false
var funcBodyStr string
options := make(map[string]struct{})
for _, option := range cf.Options {
if _, ok := options[reflect.TypeOf(option).Name()]; ok {
panic(pgerror.New(pgcode.Syntax, "conflicting or redundant options"))
}
switch opt := option.(type) {
case tree.FunctionBodyStr:
funcBodyFound = true
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,7 @@ go_test(
"type_check_test.go",
"type_name_test.go",
"typing_test.go",
"udf_test.go",
"var_expr_test.go",
],
data = glob(["testdata/**"]) + ["//pkg/sql/parser:sql.y"],
Expand Down
47 changes: 47 additions & 0 deletions pkg/sql/sem/tree/udf.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,11 @@ import (
"github.com/cockroachdb/errors"
)

// ErrConflictingFunctionOption indicates that there are conflicting or
// redundant function options from user input to either create or alter a
// function.
var ErrConflictingFunctionOption = pgerror.New(pgcode.Syntax, "conflicting or redundant options")

// FunctionName represent a function name in a UDF relevant statement, either
// DDL or DML statement. Similar to TableName, it is constructed for incoming
// SQL queries from an UnresolvedObjectName.
Expand Down Expand Up @@ -501,3 +506,45 @@ func MaybeFailOnUDFUsage(expr TypedExpr) error {
}
return nil
}

// ValidateFuncOptions checks whether there are conflicting or redundant
// function options in the given slice.
func ValidateFuncOptions(options FunctionOptions) error {
var hasLang, hasBody, hasLeakProof, hasVolatility, hasNullInputBehavior bool
err := func(opt FunctionOption) error {
return errors.Wrapf(ErrConflictingFunctionOption, "%s", AsString(opt))
}
for _, option := range options {
switch option.(type) {
case FunctionLanguage:
if hasLang {
return err(option)
}
hasLang = true
case FunctionBodyStr:
if hasBody {
return err(option)
}
hasBody = true
case FunctionLeakproof:
if hasLeakProof {
return err(option)
}
hasLeakProof = true
case FunctionVolatility:
if hasVolatility {
return err(option)
}
hasVolatility = true
case FunctionNullInputBehavior:
if hasNullInputBehavior {
return err(option)
}
hasNullInputBehavior = true
default:
return pgerror.Newf(pgcode.InvalidParameterValue, "unknown function option: ", AsString(option))
}
}

return nil
}
92 changes: 92 additions & 0 deletions pkg/sql/sem/tree/udf_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package tree_test

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

func TestConflictingFunctionOptions(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testCases := []struct {
testName string
options tree.FunctionOptions
expectedErr string
}{
{
testName: "no conflict",
options: tree.FunctionOptions{
tree.FunctionVolatile, tree.FunctionLeakproof(true), tree.FunctionCalledOnNullInput, tree.FunctionLangSQL, tree.FunctionBodyStr("hi"),
},
expectedErr: "",
},
{
testName: "volatility conflict",
options: tree.FunctionOptions{
tree.FunctionVolatile, tree.FunctionStable,
},
expectedErr: "STABLE: conflicting or redundant options",
},
{
testName: "null input behavior conflict 1",
options: tree.FunctionOptions{
tree.FunctionCalledOnNullInput, tree.FunctionReturnsNullOnNullInput,
},
expectedErr: "RETURNS NULL ON NULL INPUT: conflicting or redundant options",
},
{
testName: "null input behavior conflict 2",
options: tree.FunctionOptions{
tree.FunctionCalledOnNullInput, tree.FunctionStrict,
},
expectedErr: "STRICT: conflicting or redundant options",
},
{
testName: "leakproof conflict",
options: tree.FunctionOptions{
tree.FunctionLeakproof(true), tree.FunctionLeakproof(false),
},
expectedErr: "NOT LEAKPROOF: conflicting or redundant options",
},
{
testName: "language conflict",
options: tree.FunctionOptions{
tree.FunctionLangSQL, tree.FunctionLangSQL,
},
expectedErr: "LANGUAGE SQL: conflicting or redundant options",
},
{
testName: "function body conflict",
options: tree.FunctionOptions{
tree.FunctionBodyStr("queries"), tree.FunctionBodyStr("others"),
},
expectedErr: "AS $$others$$: conflicting or redundant options",
},
}

for _, tc := range testCases {
t.Run(tc.testName, func(t *testing.T) {
err := tree.ValidateFuncOptions(tc.options)
if tc.expectedErr == "" {
require.NoError(t, err)
return
}
require.Equal(t, tc.expectedErr, err.Error())
})
}
}

0 comments on commit e0ce0fc

Please sign in to comment.