Skip to content

Commit

Permalink
sql: make mutations visible to volatile UDFs
Browse files Browse the repository at this point in the history
The volatility of a UDF affects the visibility of mutations made by the
statement calling the function. A volatile function will see these
mutations. Also, statements within a volatile function's body will see
changes made by previous statements the function body (note that this is
left untested in this commit because we do not currently support
mutations within UDF bodies). In contrast, a stable, immutable, or
leakproof function will see a snapshot of the data as of the start of
the statement calling the function.

Release note: None
  • Loading branch information
mgartner committed Aug 11, 2022
1 parent 7829091 commit c4bf42a
Show file tree
Hide file tree
Showing 5 changed files with 80 additions and 0 deletions.
39 changes: 39 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -946,3 +946,42 @@ query I
SELECT fetch_b(99999999)
----
NULL

subtest volatility

statement ok
CREATE TABLE kv (k INT PRIMARY KEY, v INT);
INSERT INTO kv VALUES (1, 1), (2, 2), (3, 3);
CREATE FUNCTION get_l(i INT) RETURNS INT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$
SELECT v FROM kv WHERE k = i;
$$;
CREATE FUNCTION get_i(i INT) RETURNS INT IMMUTABLE LANGUAGE SQL AS $$
SELECT v FROM kv WHERE k = i;
$$;
CREATE FUNCTION get_s(i INT) RETURNS INT STABLE LANGUAGE SQL AS $$
SELECT v FROM kv WHERE k = i;
$$;
CREATE FUNCTION get_v(i INT) RETURNS INT VOLATILE LANGUAGE SQL AS $$
SELECT v FROM kv WHERE k = i;
$$;
CREATE FUNCTION int_identity_v(i INT) RETURNS INT VOLATILE LANGUAGE SQL AS $$
SELECT i;
$$;

# Only the volatile functions should see the changes made by the UPDATE in the
# CTE.
query IIIIIIII colnames
WITH u AS (
UPDATE kv SET v = v + 10 RETURNING k
)
SELECT
get_l(k) l1, get_l(int_identity_v(k)) l2,
get_i(k) i1, get_i(int_identity_v(k)) i2,
get_s(k) s1, get_s(int_identity_v(k)) s2,
get_v(k) v1, get_v(int_identity_v(k)) v2
FROM u;
----
l1 l2 i1 i2 s1 s2 v1 v2
1 1 1 1 1 1 11 11
2 2 2 2 2 2 12 12
3 3 3 3 3 3 13 13
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/execbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -743,6 +743,7 @@ func (b *Builder) buildUDF(ctx *buildScalarCtx, scalar opt.ScalarExpr) (tree.Typ
planFn,
len(udf.Body),
udf.Typ,
udf.Volatility,
udf.CalledOnNullInput,
), nil
}
7 changes: 7 additions & 0 deletions pkg/sql/opt/ops/scalar.opt
Original file line number Diff line number Diff line change
Expand Up @@ -1244,6 +1244,13 @@ define UDFPrivate {

# Volatility is the user-provided volatility of the function given during
# CREATE FUNCTION.
#
# Volatility affects the visibility of mutations made by the statement
# calling the function. A volatile function will see these mutations. Also,
# statements within a volatile function's body will see changes made by
# previous statements in the function body. In contrast, a stable,
# immutable, or leakproof function will see a snapshot of the data as of the
# start of the statement calling the function.
Volatility Volatility

# CalledOnNullInput is true if the function should be called when any of its
Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/routine.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,9 @@ package sql
import (
"context"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -44,6 +46,18 @@ func (p *planner) EvalRoutineExpr(
defer rch.Close(ctx)
rrw := NewRowResultWriter(&rch)

// Configure stepping for volatile routines so that mutations made by the
// invoking statement are visible to the routine.
txn := p.Txn()
if expr.Volatility == volatility.Volatile {
prevSteppingMode := txn.ConfigureStepping(ctx, kv.SteppingEnabled)
prevSeqNum := txn.GetLeafTxnInputState(ctx).ReadSeqNum
defer func() {
_ = p.Txn().ConfigureStepping(ctx, prevSteppingMode)
err = txn.SetReadSeqNum(prevSeqNum)
}()
}

// Execute each statement in the routine sequentially.
ef := newExecFactory(p)
for i := 0; i < expr.NumStmts; i++ {
Expand All @@ -62,6 +76,14 @@ func (p *planner) EvalRoutineExpr(
w = &droppingResultWriter{}
}

// Place a sequence point before each statement in the routine for
// volatile functions.
if expr.Volatility == volatility.Volatile {
if err := txn.Step(ctx); err != nil {
return nil, err
}
}

// TODO(mgartner): Add a new tracing.ChildSpan to the context for better
// tracing of UDFs, like we do with apply-joins.
err = runPlanInsidePlan(ctx, p.RunParams(ctx), plan.(*planComponents), w)
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/sem/tree/routine.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package tree
import (
"context"

"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -50,6 +51,14 @@ type RoutineExpr struct {
// Typ is the type of the routine's result.
Typ *types.T

// Volatility affects the visibility of mutations made by the statement
// invoking the routine. A volatile routine will see these mutations. Also,
// statements within a volatile function's body will see changes made by
// previous statements in the routine. In contrast, a stable, immutable,
// or leakproof function will see a snapshot of the data as of the start of
// the statement calling the function.
Volatility volatility.V

// CalledOnNullInput is true if the function should be called when any of
// its inputs are NULL. If false, the function will not be evaluated in the
// presence of null inputs, and will instead evaluate directly to NULL.
Expand All @@ -65,13 +74,15 @@ func NewTypedRoutineExpr(
planFn RoutinePlanFn,
numStmts int,
typ *types.T,
v volatility.V,
calledOnNullInput bool,
) *RoutineExpr {
return &RoutineExpr{
Input: input,
PlanFn: planFn,
NumStmts: numStmts,
Typ: typ,
Volatility: v,
CalledOnNullInput: calledOnNullInput,
name: name,
}
Expand Down

0 comments on commit c4bf42a

Please sign in to comment.