Skip to content

Commit

Permalink
sql: mark functions as Immutable/Stable/Volatile as just Pure/Impure
Browse files Browse the repository at this point in the history
This commit adds the "Volatile" argument to all function properties,
replacing the "Impure" definition. I've replaced all "Impure" checks
with "!= VolatilityImmutable` to give equivalent functionality for now -
we can enhance the folding and the like in a later commit. The planner
tweaks can come from someone more familiar with it :).

Also updated the pg_proc table, which should be the only visible user
change.

Release note (sql change): Populated the `pg_proc` table's `provolatile`
field based on the internal builtin volatility definition. This value
used to always be NULL.
  • Loading branch information
otan committed May 6, 2020
1 parent 80fdcc7 commit 198edc0
Show file tree
Hide file tree
Showing 25 changed files with 203 additions and 126 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ func parseValues(tableDesc *sqlbase.TableDescriptor, values string) ([]sqlbase.E
for colIdx, expr := range rowTuple {
col := &tableDesc.Columns[colIdx]
typedExpr, err := sqlbase.SanitizeVarFreeExpr(
expr, col.Type, "avro", semaCtx, false /* allowImpure */)
expr, col.Type, "avro", semaCtx, false /* allowNonImmutable */)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/partitionccl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func valueEncodePartitionTuple(

var semaCtx tree.SemaContext
typedExpr, err := sqlbase.SanitizeVarFreeExpr(expr, cols[i].Type, "partition",
&semaCtx, false /* allowImpure */)
&semaCtx, false /* allowNonImmutable */)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,7 +370,7 @@ func makeFunc(s *Smither, ctx Context, typ *types.T, refs colRefs) (tree.TypedEx
return nil, false
}
fn := fns[s.rnd.Intn(len(fns))]
if s.disableImpureFns && fn.def.Impure {
if s.disableNonImmutableFns && fn.def.Volatility != tree.VolatilityImmutable {
return nil, false
}
for _, ignore := range s.ignoreFNs {
Expand Down
30 changes: 15 additions & 15 deletions pkg/internal/sqlsmith/sqlsmith.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,17 +75,17 @@ type Smither struct {
scalarExprWeights, boolExprWeights []scalarExprWeight
scalarExprSampler, boolExprSampler *scalarExprSampler

disableWith bool
disableImpureFns bool
disableLimits bool
disableWindowFuncs bool
simpleDatums bool
avoidConsts bool
vectorizable bool
outputSort bool
postgres bool
ignoreFNs []*regexp.Regexp
complexity float64
disableWith bool
disableNonImmutableFns bool
disableLimits bool
disableWindowFuncs bool
simpleDatums bool
avoidConsts bool
vectorizable bool
outputSort bool
postgres bool
ignoreFNs []*regexp.Regexp
complexity float64

bulkSrv *httptest.Server
bulkFiles map[string][]byte
Expand Down Expand Up @@ -268,9 +268,9 @@ var DisableWith = simpleOption("disable WITH", func(s *Smither) {
s.disableWith = true
})

// DisableImpureFns causes the Smither to disable impure functions.
var DisableImpureFns = simpleOption("disable impure funcs", func(s *Smither) {
s.disableImpureFns = true
// DisableNonImmutableFns causes the Smither to disable non-immutable functions.
var DisableNonImmutableFns = simpleOption("disable non-immutable funcs", func(s *Smither) {
s.disableNonImmutableFns = true
})

// DisableCRDBFns causes the Smither to disable crdb_internal functions.
Expand Down Expand Up @@ -337,7 +337,7 @@ var OutputSort = simpleOption("output sort", func(s *Smither) {
var CompareMode = multiOption(
"compare mode",
DisableMutations(),
DisableImpureFns(),
DisableNonImmutableFns(),
DisableCRDBFns(),
IgnoreFNs("^version"),
DisableLimits(),
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -965,7 +965,7 @@ func applyColumnMutation(
} else {
colDatumType := col.Type
expr, err := sqlbase.SanitizeVarFreeExpr(
t.Default, colDatumType, "DEFAULT", &params.p.semaCtx, true, /* allowImpure */
t.Default, colDatumType, "DEFAULT", &params.p.semaCtx, true, /* allowNonImmutable */
)
if err != nil {
return err
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -2257,7 +2257,7 @@ func validateComputedColumn(
return err
}
if _, err := sqlbase.SanitizeVarFreeExpr(
replacedExpr, defType, "computed column", semaCtx, false, /* allowImpure */
replacedExpr, defType, "computed column", semaCtx, false, /* allowNonImmutable */
); err != nil {
return err
}
Expand Down Expand Up @@ -2327,7 +2327,7 @@ func MakeCheckConstraint(
}

if _, err := sqlbase.SanitizeVarFreeExpr(
expr, types.Bool, "CHECK", semaCtx, true, /* allowImpure */
expr, types.Bool, "CHECK", semaCtx, true, /* allowNonImmutable */
); err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execute.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ func fillInPlaceholders(
}
typedExpr, err := sqlbase.SanitizeVarFreeExpr(
e, typ, "EXECUTE parameter", /* context */
&semaCtx, true /* allowImpure */)
&semaCtx, true /* allowNonImmutable */)
if err != nil {
return nil, pgerror.WithCandidateCode(err, pgcode.WrongObjectType)
}
Expand Down
40 changes: 32 additions & 8 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -1353,14 +1353,14 @@ FROM pg_catalog.pg_proc
WHERE proname='substring'
----
proname proisstrict proretset provolatile proparallel
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false NULL NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL
substring false false i NULL

query TIIOTTTT colnames
SELECT proname, pronargs, pronargdefaults, prorettype, proargtypes, proallargtypes, proargmodes, proargdefaults
Expand Down Expand Up @@ -1408,6 +1408,30 @@ WHERE proname='json_extract_path'
proname provariadic pronargs prorettype proargtypes proargmodes
json_extract_path 25 2 3802 3802 25 {i,v}

# Check provolatile is set.
query TT colnames
SELECT proname, provolatile
FROM pg_catalog.pg_proc
WHERE proname IN ('random', 'current_timestamp', 'substring')
----
proname provolatile
current_timestamp s
current_timestamp s
current_timestamp s
current_timestamp s
current_timestamp s
current_timestamp s
random v
substring i
substring i
substring i
substring i
substring i
substring i
substring i
substring i


user testuser

# Should be globally visible
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -511,7 +511,7 @@ func (h *harness) prepareUsingAPI(tb testing.TB) {
typ,
"", /* context */
&h.semaCtx,
true, /* allowImpure */
true, /* allowNonImmutable */
)
if err != nil {
tb.Fatalf("%v", err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -623,7 +623,7 @@ func (v *fastIsConstVisitor) VisitPre(expr tree.Expr) (recurse bool, newExpr tre

switch t := expr.(type) {
case *tree.FuncExpr:
if t.IsImpure() {
if t.Volatility() != tree.VolatilityImmutable {
v.isConst = false
return false, expr
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/memo/logical_props_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1388,8 +1388,8 @@ func BuildSharedProps(e opt.Expr, shared *props.Shared) {
}

case *FunctionExpr:
if t.Properties.Impure {
// Impure functions can return different value on each call.
if t.Properties.Volatility != tree.VolatilityImmutable {
// Volatile functions can return different value on each call.
shared.CanHaveSideEffects = true
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/norm/norm_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func TestRuleFunctionAssumption(t *testing.T) {
t.Errorf("could not find properties for function %s", name)
continue
}
if props.Impure {
if props.Volatility != tree.VolatilityImmutable {
t.Errorf("%s should not be folded because it is impure", name)
}
if props.Category == categorySystemInfo || props.Category == categoryDateAndTime {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/norm/testdata/rules/comp
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ select
├── (i:2 + k:1) > 4 [outer=(1,2)]
└── (i:2 * 2) >= 3 [outer=(2)]

# Impure function should not be considered constant.
# Volatile function should not be considered constant.
norm expect-not=CommuteConstInequality
SELECT * FROM a WHERE random()::int>a.i+a.i
----
Expand Down
23 changes: 18 additions & 5 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -2179,6 +2179,19 @@ CREATE TABLE pg_catalog.pg_proc (
argmodes = tree.DNull
variadicType = oidZero
}

var volatile byte
switch props.Volatility {
case tree.VolatilityImmutable:
volatile = 'i'
case tree.VolatilityStable:
volatile = 's'
case tree.VolatilityVolatile:
volatile = 'v'
default:
return errors.Newf("unexpected volatility %d on proname %s", props.Volatility, name)
}

err := addRow(
h.BuiltinOid(name, &builtin), // oid
dName, // proname
Expand All @@ -2192,11 +2205,11 @@ CREATE TABLE pg_catalog.pg_proc (
tree.MakeDBool(tree.DBool(isAggregate)), // proisagg
tree.MakeDBool(tree.DBool(isWindow)), // proiswindow
tree.DBoolFalse, // prosecdef
tree.MakeDBool(tree.DBool(!props.Impure)), // proleakproof
tree.DBoolFalse, // proisstrict
tree.MakeDBool(tree.DBool(isRetSet)), // proretset
tree.DNull, // provolatile
tree.DNull, // proparallel
tree.MakeDBool(tree.DBool(props.Volatility != tree.VolatilityVolatile)), // proleakproof
tree.DBoolFalse, // proisstrict
tree.MakeDBool(tree.DBool(isRetSet)), // proretset
tree.NewDString(string(volatile)), // provolatile
tree.DNull, // proparallel
tree.NewDInt(tree.DInt(builtin.Types.Length())), // pronargs
tree.NewDInt(tree.DInt(0)), // pronargdefaults
retType, // prorettype
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/sem/builtins/aggregate_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ func initAggregateBuiltins() {
panic("duplicate builtin: " + k)
}

if !v.props.Impure {
if v.props.Volatility != tree.VolatilityVolatile {
panic(fmt.Sprintf("%s: aggregate functions should all be impure, found %v", k, v))
}
if v.props.Class != tree.AggregateClass {
Expand Down Expand Up @@ -68,7 +68,7 @@ func initAggregateBuiltins() {
}

func aggProps() tree.FunctionProperties {
return tree.FunctionProperties{Class: tree.AggregateClass, Impure: true}
return tree.FunctionProperties{Class: tree.AggregateClass, Volatility: tree.VolatilityVolatile}
}

func aggPropsNullableArgs() tree.FunctionProperties {
Expand Down Expand Up @@ -318,8 +318,16 @@ var aggregates = map[string]builtinDefinition{
"Aggregates values as a JSON or JSONB array."),
),

"json_object_agg": makeBuiltin(tree.FunctionProperties{UnsupportedWithIssue: 33285, Class: tree.AggregateClass, Impure: true}),
"jsonb_object_agg": makeBuiltin(tree.FunctionProperties{UnsupportedWithIssue: 33285, Class: tree.AggregateClass, Impure: true}),
"json_object_agg": makeBuiltin(tree.FunctionProperties{
UnsupportedWithIssue: 33285,
Class: tree.AggregateClass,
Volatility: tree.VolatilityVolatile,
}),
"jsonb_object_agg": makeBuiltin(tree.FunctionProperties{
UnsupportedWithIssue: 33285,
Class: tree.AggregateClass,
Volatility: tree.VolatilityVolatile,
}),

AnyNotNull: makePrivate(makeBuiltin(aggProps(),
makeAggOverloadWithReturnType(
Expand Down
Loading

0 comments on commit 198edc0

Please sign in to comment.