diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 2b2a4712de34..2323a6f566d9 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -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 } diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index 61d570baf608..165f1a99bfe2 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -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 } diff --git a/pkg/geo/geo.go b/pkg/geo/geo.go index f710f2dcebe3..dc391a949c49 100644 --- a/pkg/geo/geo.go +++ b/pkg/geo/geo.go @@ -177,6 +177,15 @@ func ParseGeography(str string) (*Geography, error) { return NewGeography(spatialObject), nil } +// MustParseGeography behaves as ParseGeography, but panics if there is an error. +func MustParseGeography(str string) *Geography { + g, err := ParseGeography(str) + if err != nil { + panic(err) + } + return g +} + // ParseGeographyFromEWKT parses the EWKT into a Geography. func ParseGeographyFromEWKT( ewkt geopb.EWKT, srid geopb.SRID, defaultSRIDOverwriteSetting defaultSRIDOverwriteSetting, diff --git a/pkg/internal/sqlsmith/scalar.go b/pkg/internal/sqlsmith/scalar.go index 653acc78cf8a..668d83f6b29e 100644 --- a/pkg/internal/sqlsmith/scalar.go +++ b/pkg/internal/sqlsmith/scalar.go @@ -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 { diff --git a/pkg/internal/sqlsmith/sqlsmith.go b/pkg/internal/sqlsmith/sqlsmith.go index f81115624b41..2601ffb10d30 100644 --- a/pkg/internal/sqlsmith/sqlsmith.go +++ b/pkg/internal/sqlsmith/sqlsmith.go @@ -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 @@ -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. @@ -337,7 +337,7 @@ var OutputSort = simpleOption("output sort", func(s *Smither) { var CompareMode = multiOption( "compare mode", DisableMutations(), - DisableImpureFns(), + DisableNonImmutableFns(), DisableCRDBFns(), IgnoreFNs("^version"), DisableLimits(), diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index e565589fdffc..89cca8202425 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -970,7 +970,7 @@ func applyColumnMutation( } else { colDatumType := col.Type expr, err := sqlbase.SanitizeVarFreeExpr( - t.Default, colDatumType, "DEFAULT", ¶ms.p.semaCtx, true, /* allowImpure */ + t.Default, colDatumType, "DEFAULT", ¶ms.p.semaCtx, true, /* allowNonImmutable */ ) if err != nil { return err diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index eb8fa97f8a14..0cb7952b1a74 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -2275,7 +2275,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 } @@ -2345,7 +2345,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 } diff --git a/pkg/sql/execute.go b/pkg/sql/execute.go index 9579d5e0efa7..eb399a784c67 100644 --- a/pkg/sql/execute.go +++ b/pkg/sql/execute.go @@ -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) } diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 9c971581df3f..1e6569814e6b 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -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 @@ -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 diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index f069c9856b3d..e5826eaf103b 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -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) diff --git a/pkg/sql/opt/exec/execbuilder/scalar.go b/pkg/sql/opt/exec/execbuilder/scalar.go index 982864eb723e..2afc4efe5b1d 100644 --- a/pkg/sql/opt/exec/execbuilder/scalar.go +++ b/pkg/sql/opt/exec/execbuilder/scalar.go @@ -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 } diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 61b2dce95273..c6014850fd16 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -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 { + // Non-immutable functions can return different value on each call. shared.CanHaveSideEffects = true } diff --git a/pkg/sql/opt/norm/norm_test.go b/pkg/sql/opt/norm/norm_test.go index 3593a430ba61..8146bbfc35c0 100644 --- a/pkg/sql/opt/norm/norm_test.go +++ b/pkg/sql/opt/norm/norm_test.go @@ -86,8 +86,8 @@ func TestRuleFunctionAssumption(t *testing.T) { t.Errorf("could not find properties for function %s", name) continue } - if props.Impure { - t.Errorf("%s should not be folded because it is impure", name) + if props.Volatility != tree.VolatilityImmutable { + t.Errorf("%s should not be folded because it is not immutable", name) } if props.Category == categorySystemInfo || props.Category == categoryDateAndTime { switch name { diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index c65eeee724ee..0adf8799104a 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -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 ---- diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index c228b034452a..60b90b53148a 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -2179,6 +2179,7 @@ CREATE TABLE pg_catalog.pg_proc ( argmodes = tree.DNull variadicType = oidZero } + err := addRow( h.BuiltinOid(name, &builtin), // oid dName, // proname @@ -2192,11 +2193,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(props.Volatility)), // provolatile + tree.DNull, // proparallel tree.NewDInt(tree.DInt(builtin.Types.Length())), // pronargs tree.NewDInt(tree.DInt(0)), // pronargdefaults retType, // prorettype diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index 4de14aac587a..a8262945bd01 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -38,8 +38,8 @@ func initAggregateBuiltins() { panic("duplicate builtin: " + k) } - if !v.props.Impure { - panic(fmt.Sprintf("%s: aggregate functions should all be impure, found %v", k, v)) + if v.props.Volatility != tree.VolatilityVolatile { + panic(fmt.Sprintf("%s: aggregate functions should all be volatile, found %v", k, v)) } if v.props.Class != tree.AggregateClass { panic(fmt.Sprintf("%s: aggregate functions should be marked with the tree.AggregateClass "+ @@ -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 { @@ -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( diff --git a/pkg/sql/sem/builtins/all_builtins.go b/pkg/sql/sem/builtins/all_builtins.go index 5e67875c30ad..50fcd94a5252 100644 --- a/pkg/sql/sem/builtins/all_builtins.go +++ b/pkg/sql/sem/builtins/all_builtins.go @@ -42,6 +42,14 @@ func init() { AllAggregateBuiltinNames = make([]string, 0, len(aggregates)) tree.FunDefs = make(map[string]*tree.FunctionDefinition) for name, def := range builtins { + // TODO: instead of the default being immutable (which could be a + // problem if a new function forgets to set its volatility if not immutable), + // explicitly define volatility of all functions and add a test that asserts + // it is never the zero value. + if def.props.Volatility == 0 { + def.props.Volatility = tree.VolatilityImmutable + builtins[name] = def + } fDef := tree.NewFunctionDefinition(name, &def.props, def.overloads) tree.FunDefs[name] = fDef if fDef.Private { @@ -56,7 +64,7 @@ func init() { } } - // Generate missing categories. + // Generate missing categories and volatilities. for _, name := range AllBuiltinNames { def := builtins[name] if def.props.Category == "" { diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 6444165eec8e..c85e9459da76 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -421,8 +421,8 @@ var builtins = map[string]builtinDefinition{ "gen_random_uuid": makeBuiltin( tree.FunctionProperties{ - Category: categoryIDGeneration, - Impure: true, + Category: categoryIDGeneration, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -1510,7 +1510,7 @@ CockroachDB supports the following flags: "random": makeBuiltin( tree.FunctionProperties{ - Impure: true, + Volatility: tree.VolatilityVolatile, NeedsRepeatedEvaluation: true, }, tree.Overload{ @@ -1525,8 +1525,8 @@ CockroachDB supports the following flags: "unique_rowid": makeBuiltin( tree.FunctionProperties{ - Category: categoryIDGeneration, - Impure: true, + Category: categoryIDGeneration, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -1548,7 +1548,7 @@ CockroachDB supports the following flags: tree.FunctionProperties{ Category: categorySequences, DistsqlBlacklist: true, - Impure: true, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"sequence_name", types.String}}, @@ -1573,7 +1573,7 @@ CockroachDB supports the following flags: tree.FunctionProperties{ Category: categorySequences, DistsqlBlacklist: true, - Impure: true, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"sequence_name", types.String}}, @@ -1596,8 +1596,8 @@ CockroachDB supports the following flags: "lastval": makeBuiltin( tree.FunctionProperties{ - Category: categorySequences, - Impure: true, + Category: categorySequences, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -1619,7 +1619,7 @@ CockroachDB supports the following flags: tree.FunctionProperties{ Category: categorySequences, DistsqlBlacklist: true, - Impure: true, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"sequence_name", types.String}, {"value", types.Int}}, @@ -1779,7 +1779,7 @@ CockroachDB supports the following flags: // https://www.postgresql.org/docs/10/static/functions-datetime.html "age": makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{Volatility: tree.VolatilityStable}, tree.Overload{ Types: tree.ArgTypes{{"val", types.TimestampTZ}}, ReturnType: tree.FixedReturnType(types.Interval), @@ -1789,6 +1789,7 @@ CockroachDB supports the following flags: Info: "Calculates the interval between `val` and the current time.", }, tree.Overload{ + // NOTE(otan): This could be Immutable if Volatility was per overload. Types: tree.ArgTypes{{"end", types.TimestampTZ}, {"begin", types.TimestampTZ}}, ReturnType: tree.FixedReturnType(types.Interval), Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { @@ -1799,7 +1800,7 @@ CockroachDB supports the following flags: ), "current_date": makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{Volatility: tree.VolatilityStable}, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.Date), @@ -1817,7 +1818,7 @@ CockroachDB supports the following flags: "localtime": txnTimeWithPrecisionBuiltin(false), "statement_timestamp": makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{Volatility: tree.VolatilityStable}, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.TimestampTZ), @@ -1838,7 +1839,7 @@ CockroachDB supports the following flags: ), tree.FollowerReadTimestampFunctionName: makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{Volatility: tree.VolatilityVolatile}, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.TimestampTZ), @@ -1864,8 +1865,8 @@ return without an error.`, "cluster_logical_timestamp": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityStable, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -1884,7 +1885,9 @@ may increase either contention or retry errors, or both.`, ), "clock_timestamp": makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{ + Volatility: tree.VolatilityVolatile, + }, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.TimestampTZ), @@ -1905,7 +1908,10 @@ may increase either contention or retry errors, or both.`, ), "timeofday": makeBuiltin( - tree.FunctionProperties{Category: categoryDateAndTime, Impure: true}, + tree.FunctionProperties{ + Category: categoryDateAndTime, + Volatility: tree.VolatilityVolatile, + }, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.String), @@ -2890,8 +2896,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.force_error": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"errorCode", types.String}, {"msg", types.String}}, @@ -2912,8 +2918,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.notice": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"msg", types.String}}, @@ -2941,8 +2947,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.force_assertion_error": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"msg", types.String}}, @@ -2957,8 +2963,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.force_panic": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"msg", types.String}}, @@ -2976,8 +2982,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.force_log_fatal": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"msg", types.String}}, @@ -3001,8 +3007,8 @@ may increase either contention or retry errors, or both.`, // different than the current statement's transaction. "crdb_internal.force_retry": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"val", types.Interval}}, @@ -3050,8 +3056,8 @@ may increase either contention or retry errors, or both.`, // Identity function which is marked as impure to avoid constant folding. "crdb_internal.no_constant_folding": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"input", types.Any}}, @@ -3177,8 +3183,8 @@ may increase either contention or retry errors, or both.`, "crdb_internal.set_vmodule": makeBuiltin( tree.FunctionProperties{ - Category: categorySystemInfo, - Impure: true, + Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"vmodule_string", types.String}}, @@ -3603,8 +3609,8 @@ func getSubstringFromIndexOfLength(str, errMsg string, start, length int) (strin var uuidV4Impl = makeBuiltin( tree.FunctionProperties{ - Category: categoryIDGeneration, - Impure: true, + Category: categoryIDGeneration, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{}, @@ -3716,8 +3722,8 @@ func txnTSWithPrecisionOverloads(preferTZOverload bool) []tree.Overload { func txnTSImplBuiltin(preferTZOverload bool) builtinDefinition { return makeBuiltin( tree.FunctionProperties{ - Category: categoryDateAndTime, - Impure: true, + Category: categoryDateAndTime, + Volatility: tree.VolatilityVolatile, }, txnTSOverloads(preferTZOverload)..., ) @@ -3726,8 +3732,8 @@ func txnTSImplBuiltin(preferTZOverload bool) builtinDefinition { func txnTSWithPrecisionImplBuiltin(preferTZOverload bool) builtinDefinition { return makeBuiltin( tree.FunctionProperties{ - Category: categoryDateAndTime, - Impure: true, + Category: categoryDateAndTime, + Volatility: tree.VolatilityStable, }, txnTSWithPrecisionOverloads(preferTZOverload)..., ) @@ -3736,7 +3742,7 @@ func txnTSWithPrecisionImplBuiltin(preferTZOverload bool) builtinDefinition { func txnTimeWithPrecisionBuiltin(preferTZOverload bool) builtinDefinition { tzAdditionalDesc, noTZAdditionalDesc := getTimeAdditionalDesc(preferTZOverload) return makeBuiltin( - tree.FunctionProperties{Impure: true}, + tree.FunctionProperties{Volatility: tree.VolatilityStable}, tree.Overload{ Types: tree.ArgTypes{}, ReturnType: tree.FixedReturnType(types.TimeTZ), diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index 38c227ffdd14..99b964cf1869 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -43,8 +43,8 @@ func initGeneratorBuiltins() { panic("duplicate builtin: " + k) } - if !v.props.Impure { - panic(fmt.Sprintf("generator functions should all be impure, found %v", v)) + if v.props.Volatility != tree.VolatilityVolatile { + panic(fmt.Sprintf("generator functions should all be volatile, found %v", v)) } if v.props.Class != tree.GeneratorClass { panic(fmt.Sprintf("generator functions should be marked with the tree.GeneratorClass "+ @@ -57,15 +57,15 @@ func initGeneratorBuiltins() { func genProps() tree.FunctionProperties { return tree.FunctionProperties{ - Impure: true, - Class: tree.GeneratorClass, - Category: categoryGenerator, + Volatility: tree.VolatilityVolatile, + Class: tree.GeneratorClass, + Category: categoryGenerator, } } func genPropsWithLabels(returnLabels []string) tree.FunctionProperties { return tree.FunctionProperties{ - Impure: true, + Volatility: tree.VolatilityVolatile, Class: tree.GeneratorClass, Category: categoryGenerator, ReturnLabels: returnLabels, @@ -252,9 +252,9 @@ var generators = map[string]builtinDefinition{ "crdb_internal.check_consistency": makeBuiltin( tree.FunctionProperties{ - Impure: true, - Class: tree.GeneratorClass, - Category: categorySystemInfo, + Volatility: tree.VolatilityVolatile, + Class: tree.GeneratorClass, + Category: categorySystemInfo, }, makeGeneratorOverload( tree.ArgTypes{ diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 0a75c3d00a9d..51fef55ca0fc 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -1049,7 +1049,7 @@ SELECT description tree.FunctionProperties{ // pg_sleep is marked as impure so it doesn't get executed during // normalization. - Impure: true, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"seconds", types.Float}}, @@ -1704,7 +1704,7 @@ SELECT description tree.FunctionProperties{ Category: categorySystemInfo, DistsqlBlacklist: true, - Impure: true, + Volatility: tree.VolatilityVolatile, }, tree.Overload{ Types: tree.ArgTypes{{"setting_name", types.String}, {"new_value", types.String}, {"is_local", types.Bool}}, diff --git a/pkg/sql/sem/builtins/window_builtins.go b/pkg/sql/sem/builtins/window_builtins.go index 2a1a340ffbd1..306a03c3153a 100644 --- a/pkg/sql/sem/builtins/window_builtins.go +++ b/pkg/sql/sem/builtins/window_builtins.go @@ -27,8 +27,8 @@ func initWindowBuiltins() { panic("duplicate builtin: " + k) } - if !v.props.Impure { - panic(fmt.Sprintf("%s: window functions should all be impure, found %v", k, v)) + if v.props.Volatility != tree.VolatilityVolatile { + panic(fmt.Sprintf("%s: window functions should all be volatile, found %v", k, v)) } if v.props.Class != tree.WindowClass { panic(fmt.Sprintf("%s: window functions should be marked with the tree.WindowClass "+ @@ -46,8 +46,8 @@ func initWindowBuiltins() { func winProps() tree.FunctionProperties { return tree.FunctionProperties{ - Impure: true, - Class: tree.WindowClass, + Volatility: tree.VolatilityVolatile, + Class: tree.WindowClass, } } diff --git a/pkg/sql/sem/tree/expr.go b/pkg/sql/sem/tree/expr.go index e85541a52fd6..3329d77727ef 100644 --- a/pkg/sql/sem/tree/expr.go +++ b/pkg/sql/sem/tree/expr.go @@ -1292,11 +1292,13 @@ func (node *FuncExpr) IsWindowFunctionApplication() bool { return node.WindowDef != nil } -// IsImpure returns whether the function application is impure, meaning that it -// potentially returns a different value when called in the same statement with -// the same parameters. -func (node *FuncExpr) IsImpure() bool { - return node.fnProps != nil && node.fnProps.Impure +// Volatility returns the volatility of the given function. +// See Volatility definition. +func (node *FuncExpr) Volatility() Volatility { + if node.fnProps == nil { + return VolatilityImmutable + } + return node.fnProps.Volatility } // IsDistSQLBlacklist returns whether the function is not supported by DistSQL. diff --git a/pkg/sql/sem/tree/function_definition.go b/pkg/sql/sem/tree/function_definition.go index 193262932e01..3c413bb9534a 100644 --- a/pkg/sql/sem/tree/function_definition.go +++ b/pkg/sql/sem/tree/function_definition.go @@ -28,6 +28,29 @@ type FunctionDefinition struct { FunctionProperties } +// Volatility indicates whether the result of a function is dependent *only* +// on the values of its explicit arguments, or can change due to outside factors +// (such as parameter variables or table contents). +// +// This matches the postgres definition of volatility. +// +// NOTE: functions having side-effects, such as setval(), +// must be labeled volatile to ensure they will not get optimized away, +// even if the actual return value is not changeable. +type Volatility byte + +const ( + // VolatilityImmutable indicates the builtin result never changes for a + // given input. + VolatilityImmutable Volatility = 'i' + // VolatilityStable indicates the builtin result never changes during + // a given scan. + VolatilityStable Volatility = 's' + // VolatilityVolatile indicates the builtin result can change even + // within a scan. These functions are also "impure". + VolatilityVolatile Volatility = 'v' +) + // FunctionProperties defines the properties of the built-in // functions that are common across all overloads. type FunctionProperties struct { @@ -57,13 +80,10 @@ type FunctionProperties struct { // aggregate functions. NeedsRepeatedEvaluation bool - // Impure is set to true when a function potentially returns a - // different value when called in the same statement with the same - // parameters. e.g.: random(), clock_timestamp(). Some functions - // like now() return the same value in the same statement, but - // different values in separate statements, and should not be marked - // as impure. - Impure bool + // Volatility signifies whether the given function is volatile. + // NOTE(otan): This should technically be per overload, but will + // get the job done for the majority of the cases we care about. + Volatility Volatility // DistsqlBlacklist is set to true when a function depends on // members of the EvalContext that are not marshaled by DistSQL diff --git a/pkg/sql/sem/tree/normalize.go b/pkg/sql/sem/tree/normalize.go index 3fa54cd15768..f71af729736c 100644 --- a/pkg/sql/sem/tree/normalize.go +++ b/pkg/sql/sem/tree/normalize.go @@ -823,7 +823,7 @@ func (v *isConstVisitor) VisitPre(expr Expr) (recurse bool, newExpr Expr) { switch t := expr.(type) { case *FuncExpr: - if t.IsImpure() { + if t.Volatility() != VolatilityImmutable { v.isConst = false return false, expr } @@ -895,7 +895,7 @@ func (v *fastIsConstVisitor) VisitPre(expr Expr) (recurse bool, newExpr Expr) { switch t := expr.(type) { case *FuncExpr: - if t.IsImpure() { + if t.Volatility() != VolatilityImmutable { v.isConst = false return false, expr } diff --git a/pkg/sql/sem/tree/overload.go b/pkg/sql/sem/tree/overload.go index 0828f04f9368..28d5f006a1ee 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -45,6 +45,9 @@ type Overload struct { Types TypeList ReturnType ReturnTyper + // Volatility signifies whether the given function is volatile. + Volatility Volatility + // PreferredOverload determines overload resolution as follows. // When multiple overloads are eligible based on types even after all of of // the heuristics to pick one have been used, if one of the overloads is a diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index be34c47cfd65..4942ea4fe0cb 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -137,8 +137,8 @@ const ( // This is used e.g. when processing the calls inside ROWS FROM. RejectNestedGenerators - // RejectImpureFunctions rejects any non-const functions like now(). - RejectImpureFunctions + // RejectNonImmutableFunctions rejects any non-const functions like now(). + RejectNonImmutableFunctions // RejectSubqueries rejects subqueries in scalar contexts. RejectSubqueries @@ -164,9 +164,9 @@ type ScalarProperties struct { // contained a SRF. SeenGenerator bool - // SeenImpureFunctions is set to true if the expression originally - // contained an impure function. - SeenImpure bool + // SeenNonImmutableFunctions is set to true if the expression originally + // contained a function which was not immutable. + SeenNonImmutable bool // inFuncExpr is temporarily set to true while type checking the // parameters of a function. Used to process RejectNestedGenerators @@ -805,15 +805,15 @@ func (sc *SemaContext) checkFunctionUsage(expr *FuncExpr, def *FunctionDefinitio } sc.Properties.Derived.SeenGenerator = true } - if def.Impure { - if sc.Properties.required.rejectFlags&RejectImpureFunctions != 0 { + if def.Volatility != VolatilityImmutable { + if sc.Properties.required.rejectFlags&RejectNonImmutableFunctions != 0 { // The code FeatureNotSupported is a bit misleading here, // because we probably can't support the feature at all. However // this error code matches PostgreSQL's in the same conditions. return pgerror.Newf(pgcode.FeatureNotSupported, "impure functions are not allowed in %s", sc.Properties.required.context) } - sc.Properties.Derived.SeenImpure = true + sc.Properties.Derived.SeenNonImmutable = true } return nil } diff --git a/pkg/sql/sqlbase/table.go b/pkg/sql/sqlbase/table.go index 0b35d1e43c8e..7414c55d9fd3 100644 --- a/pkg/sql/sqlbase/table.go +++ b/pkg/sql/sqlbase/table.go @@ -36,7 +36,7 @@ func SanitizeVarFreeExpr( expectedType *types.T, context string, semaCtx *tree.SemaContext, - allowImpure bool, + allowNonImmutable bool, ) (tree.TypedExpr, error) { if tree.ContainsVars(expr) { return nil, pgerror.Newf(pgcode.Syntax, @@ -50,8 +50,8 @@ func SanitizeVarFreeExpr( // Ensure that the expression doesn't contain special functions. flags := tree.RejectSpecial - if !allowImpure { - flags |= tree.RejectImpureFunctions + if !allowNonImmutable { + flags |= tree.RejectNonImmutableFunctions } semaCtx.Properties.Require(context, flags) @@ -172,7 +172,7 @@ func MakeColumnDefDescs( // and does not contain invalid functions. var err error if typedExpr, err = SanitizeVarFreeExpr( - d.DefaultExpr.Expr, resType, "DEFAULT", semaCtx, true, /* allowImpure */ + d.DefaultExpr.Expr, resType, "DEFAULT", semaCtx, true, /* allowNonImmutable */ ); err != nil { return nil, nil, nil, err } @@ -233,7 +233,7 @@ func EvalShardBucketCount( ) (int32, error) { const invalidBucketCountMsg = `BUCKET_COUNT must be an integer greater than 1` typedExpr, err := SanitizeVarFreeExpr( - shardBuckets, types.Int, "BUCKET_COUNT", semaCtx, true, /* allowImpure */ + shardBuckets, types.Int, "BUCKET_COUNT", semaCtx, true, /* allowNonImmutable */ ) if err != nil { return 0, err diff --git a/pkg/sql/sqlbase/testutils.go b/pkg/sql/sqlbase/testutils.go index f9da92241eed..c250a62e0883 100644 --- a/pkg/sql/sqlbase/testutils.go +++ b/pkg/sql/sqlbase/testutils.go @@ -533,12 +533,50 @@ var ( &tree.DInterval{Duration: duration.MakeDuration(0, 0, 290*12)}, }, types.GeographyFamily: { - // TODO(otan): more interesting datums - &tree.DGeography{Geography: geo.MustParseGeographyFromEWKBRaw([]byte("\x01\x01\x00\x00\x20\xe6\x10\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"))}, + // NOTE(otan): we cannot use WKT here because roachtests do not have geos uploaded. + // If we parse WKT ourselves or upload GEOS on every roachtest, we may be able to avoid this. + // POINT(1.0 1.0) + &tree.DGeography{Geography: geo.MustParseGeography("0101000000000000000000F03F000000000000F03F")}, + // LINESTRING(1.0 1.0, 2.0 2.0) + &tree.DGeography{Geography: geo.MustParseGeography("010200000002000000000000000000F03F000000000000F03F00000000000000400000000000000040")}, + // POLYGON((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0)) + &tree.DGeography{Geography: geo.MustParseGeography("0103000000010000000500000000000000000000000000000000000000000000000000F03F0000000000000000000000000000F03F000000000000F03F0000000000000000000000000000F03F00000000000000000000000000000000")}, + // POLYGON((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0), (0.2 0.2, 0.2 0.4, 0.4 0.4, 0.4 0.2, 0.2 0.2)) + &tree.DGeography{Geography: geo.MustParseGeography("0103000000020000000500000000000000000000000000000000000000000000000000F03F0000000000000000000000000000F03F000000000000F03F0000000000000000000000000000F03F00000000000000000000000000000000050000009A9999999999C93F9A9999999999C93F9A9999999999C93F9A9999999999D93F9A9999999999D93F9A9999999999D93F9A9999999999D93F9A9999999999C93F9A9999999999C93F9A9999999999C93F")}, + // MULTIPOINT ((10 40), (40 30), (20 20), (30 10)) + &tree.DGeography{Geography: geo.MustParseGeography("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440")}, + // MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10)) + &tree.DGeography{Geography: geo.MustParseGeography("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440")}, + // MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)),((20 35, 10 30, 10 10, 30 5, 45 20, 20 35),(30 20, 20 15, 20 25, 30 20))) + &tree.DGeography{Geography: geo.MustParseGeography("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440")}, + // GEOMETRYCOLLECTION (POINT (40 10),LINESTRING (10 10, 20 20, 10 40),POLYGON ((40 40, 20 45, 45 30, 40 40))) + &tree.DGeography{Geography: geo.MustParseGeography("01070000000300000001010000000000000000004440000000000000244001020000000300000000000000000024400000000000002440000000000000344000000000000034400000000000002440000000000000444001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440")}, + // GEOMETRYCOLLECTION EMPTY + // TODO(#48533): this is broken for empty GEOMETRYCOLLECTIONs in the ewkb.Write function. + // &tree.DGeography{Geography: geo.MustParseGeography("010700000000000000")}, }, types.GeometryFamily: { - // TODO(otan): more interesting datums - &tree.DGeometry{Geometry: geo.MustParseGeometryFromEWKBRaw([]byte("\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0\x3f\x00\x00\x00\x00\x00\x00\xf0\x3f"))}, + // NOTE(otan): we cannot use WKT here because roachtests do not have geos uploaded. + // If we parse WKT ourselves or upload GEOS on every roachtest, we may be able to avoid this. + // POINT(1.0 1.0) + &tree.DGeometry{Geometry: geo.MustParseGeometry("0101000000000000000000F03F000000000000F03F")}, + // LINESTRING(1.0 1.0, 2.0 2.0) + &tree.DGeometry{Geometry: geo.MustParseGeometry("010200000002000000000000000000F03F000000000000F03F00000000000000400000000000000040")}, + // POLYGON((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0)) + &tree.DGeometry{Geometry: geo.MustParseGeometry("0103000000010000000500000000000000000000000000000000000000000000000000F03F0000000000000000000000000000F03F000000000000F03F0000000000000000000000000000F03F00000000000000000000000000000000")}, + // POLYGON((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0), (0.2 0.2, 0.2 0.4, 0.4 0.4, 0.4 0.2, 0.2 0.2)) + &tree.DGeometry{Geometry: geo.MustParseGeometry("0103000000020000000500000000000000000000000000000000000000000000000000F03F0000000000000000000000000000F03F000000000000F03F0000000000000000000000000000F03F00000000000000000000000000000000050000009A9999999999C93F9A9999999999C93F9A9999999999C93F9A9999999999D93F9A9999999999D93F9A9999999999D93F9A9999999999D93F9A9999999999C93F9A9999999999C93F9A9999999999C93F")}, + // MULTIPOINT ((10 40), (40 30), (20 20), (30 10)) + &tree.DGeometry{Geometry: geo.MustParseGeometry("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440")}, + // MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10)) + &tree.DGeometry{Geometry: geo.MustParseGeometry("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440")}, + // MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)),((20 35, 10 30, 10 10, 30 5, 45 20, 20 35),(30 20, 20 15, 20 25, 30 20))) + &tree.DGeometry{Geometry: geo.MustParseGeometry("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440")}, + // GEOMETRYCOLLECTION (POINT (40 10),LINESTRING (10 10, 20 20, 10 40),POLYGON ((40 40, 20 45, 45 30, 40 40))) + &tree.DGeometry{Geometry: geo.MustParseGeometry("01070000000300000001010000000000000000004440000000000000244001020000000300000000000000000024400000000000002440000000000000344000000000000034400000000000002440000000000000444001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440")}, + // GEOMETRYCOLLECTION EMPTY + // TODO(#48533): this is broken for empty GEOMETRYCOLLECTIONs in the ewkb.Write function. + // &tree.DGeometry{Geometry: geo.MustParseGeometry("010700000000000000")}, }, types.StringFamily: { tree.NewDString(""),