diff --git a/githooks/commit-msg b/githooks/commit-msg index dfb1b02da454..146aed08dfb9 100755 --- a/githooks/commit-msg +++ b/githooks/commit-msg @@ -52,7 +52,7 @@ IFS=' notes=($($grep -iE '^release note' "$1")) # Set this to 1 to require a release justification note. -require_justification=1 +require_justification=0 justification=($($grep -iE '^release justification: \S+' "$1")) IFS=$saveIFS diff --git a/githooks/prepare-commit-msg b/githooks/prepare-commit-msg index 63961047c5e2..b84a3bc0bc55 100755 --- a/githooks/prepare-commit-msg +++ b/githooks/prepare-commit-msg @@ -2,7 +2,7 @@ # # Prepare the commit message by adding a release note. -require_justification=1 +require_justification=0 set -euo pipefail if [[ "${2-}" = "message" ]]; then diff --git a/pkg/ccl/backupccl/alter_backup_schedule.go b/pkg/ccl/backupccl/alter_backup_schedule.go index 6dc583cd7629..7c34537e8c56 100644 --- a/pkg/ccl/backupccl/alter_backup_schedule.go +++ b/pkg/ccl/backupccl/alter_backup_schedule.go @@ -53,7 +53,6 @@ func loadSchedules( if err != nil { return s, err } - args := &backuppb.ScheduledBackupExecutionArgs{} if err := pbtypes.UnmarshalAny(schedule.ExecutionArgs().Args, args); err != nil { return s, errors.Wrap(err, "un-marshaling args") @@ -109,8 +108,7 @@ func doAlterBackupSchedules( eval *alterBackupScheduleEval, resultsCh chan<- tree.Datums, ) error { - s, err := loadSchedules( - ctx, p, eval) + s, err := loadSchedules(ctx, p, eval) if err != nil { return err } @@ -126,14 +124,16 @@ func doAlterBackupSchedules( s.incJob.ScheduleID()) } - // Note that even ADMIN is subject to these restrictions. We expect to - // add a finer-grained permissions model soon. - if s.fullJob.Owner() != p.User() { - return pgerror.Newf(pgcode.InsufficientPrivilege, "only the OWNER of a schedule may alter it") + // Check that the user is admin or the owner of the schedules being altered. + isAdmin, err := p.UserHasAdminRole(ctx, p.User()) + if err != nil { + return err } - - if s.incJob != nil && s.incJob.Owner() != p.User() { - return pgerror.Newf(pgcode.InsufficientPrivilege, "only the OWNER of a schedule may alter it") + isOwnerOfFullJob := s.fullJob == nil || s.fullJob.Owner() == p.User() + isOwnerOfIncJob := s.incJob == nil || s.incJob.Owner() == p.User() + if !isAdmin && !(isOwnerOfFullJob && isOwnerOfIncJob) { + return pgerror.New(pgcode.InsufficientPrivilege, "must be admin or owner of the "+ + "schedules being altered.") } if s, err = processFullBackupRecurrence( diff --git a/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges b/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges index 9e399340a8c4..469f6b11dd03 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges +++ b/pkg/ccl/backupccl/testdata/backup-restore/schedule-privileges @@ -1,6 +1,8 @@ new-server name=s1 ---- +subtest create-scheduled-privileges + exec-sql CREATE DATABASE foo; ---- @@ -15,15 +17,15 @@ CREATE EXTERNAL CONNECTION foo AS 'userfile:///foo'; # Admin can create all schedules. exec-sql -CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; +CREATE SCHEDULE foocluster_admin FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; ---- exec-sql -CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; +CREATE SCHEDULE foodb_admin FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; ---- exec-sql -CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; +CREATE SCHEDULE footable_admin FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; ---- # Non-root admin can create all schedules. @@ -33,15 +35,15 @@ GRANT ADMIN TO testuser; ---- exec-sql user=testuser -CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; +CREATE SCHEDULE foocluster_admintestuser FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; ---- exec-sql user=testuser -CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; +CREATE SCHEDULE foodb_admintestuser FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; ---- exec-sql user=testuser -CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; +CREATE SCHEDULE footable_admintestuser FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; ---- # Non-root non-admin cannot create any schedules. @@ -78,13 +80,115 @@ GRANT USAGE ON EXTERNAL CONNECTION foo TO testuser; ---- exec-sql user=testuser -CREATE SCHEDULE foocluster FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; +CREATE SCHEDULE foocluster_testuser FOR BACKUP INTO 'external://foo/cluster' RECURRING '@hourly'; ---- exec-sql user=testuser -CREATE SCHEDULE foodb FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; +CREATE SCHEDULE foodb_testuser FOR BACKUP DATABASE foo INTO 'external://foo/database' RECURRING '@hourly'; ---- exec-sql user=testuser -CREATE SCHEDULE footable FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; +CREATE SCHEDULE footable_testuser FOR BACKUP TABLE foo.foo INTO 'external://foo/table' RECURRING '@hourly'; +---- + +subtest end + +subtest control-schedules-privileges + +# Save the schedule IDs for the schedules created by admin root. +let $fullID $incID +with schedules as (show schedules) select id from schedules where label='foocluster_admin' order by command->>'backup_type' asc; +---- + +query-sql +with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label, command FROM schedules WHERE id IN +($fullID, $incID) ORDER BY next_run; +---- +foocluster_admin BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached +foocluster_admin BACKUP INTO 'external://foo/cluster' WITH detached + +# nonadmin testuser is not allowed to drop a schedule they do not own. +exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser +DROP SCHEDULE $fullID +---- +regex matches error + +exec-sql +GRANT ADMIN TO testuser +---- + +# testuser has been granted admin so the drop should succeed. +exec-sql user=testuser +DROP SCHEDULE $fullID +---- + +query-sql +with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label,command,owner FROM schedules WHERE id IN +($fullID, $incID) ORDER BY next_run; +---- +foocluster_admin BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached root + +exec-sql +REVOKE ADMIN FROM testuser +---- + +exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser +DROP SCHEDULE $incID +---- +regex matches error + +# Save the schedule IDs for the schedules that were created by non-admin testuser. +let $testuserFullID $testuserIncID +with schedules as (show schedules) select id from schedules where label='foocluster_testuser' order by command->>'backup_type' asc; +---- + +query-sql +with schedules as (SHOW SCHEDULES FOR BACKUP) SELECT label, command, owner FROM schedules WHERE id +IN ($testuserFullID, $testuserIncID) ORDER BY next_run; +---- +foocluster_testuser BACKUP INTO LATEST IN 'external://foo/cluster' WITH detached testuser +foocluster_testuser BACKUP INTO 'external://foo/cluster' WITH detached testuser + +# testuser owns these schedules so should be able to pause, resume, drop, alter +# them without admin. +exec-sql user=testuser +PAUSE SCHEDULE $testuserFullID; +PAUSE SCHEDULE $testuserIncID; +---- + +exec-sql user=testuser +RESUME SCHEDULE $testuserFullID; +RESUME SCHEDULE $testuserIncID; +---- + +exec-sql user=testuser +ALTER BACKUP SCHEDULE $testuserFullID SET WITH revision_history = false; +---- + +exec-sql user=testuser +DROP SCHEDULE $testuserFullID; +DROP SCHEDULE $testuserIncID; +---- + +# But testuser can't drop, alter, resume or pause the root owned schedules. +exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to PAUSE it) user=testuser +PAUSE SCHEDULE $incID ---- +regex matches error + +exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to RESUME it) user=testuser +RESUME SCHEDULE $incID +---- +regex matches error + +exec-sql user=testuser expect-error-regex=(incremental backup schedule [0-9]+ has no corresponding full backup schedule) +ALTER BACKUP SCHEDULE $incID SET WITH revision_history = false; +---- +regex matches error + +exec-sql expect-error-regex=(must be admin or owner of the schedule [0-9]+ to DROP it) user=testuser +DROP SCHEDULE $incID; +---- +regex matches error + +subtest end diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index ff2224dd9fff..d5d160a205c2 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -92,6 +92,7 @@ go_library( "//pkg/sql/sem/catid", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", + "//pkg/sql/sem/volatility", "//pkg/sql/sessiondatapb", "//pkg/sql/sqlutil", "//pkg/sql/types", diff --git a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel index c851d9b62c88..ffca7d1840f5 100644 --- a/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdcevent/BUILD.bazel @@ -12,6 +12,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent", visibility = ["//visibility:public"], deps = [ + "//pkg/build", "//pkg/ccl/changefeedccl/changefeedbase", "//pkg/keys", "//pkg/kv", @@ -34,6 +35,7 @@ go_library( "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/iterutil", + "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], @@ -57,6 +59,7 @@ go_test( "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", + "//pkg/settings/cluster", "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/ccl/changefeedccl/cdcevent/event.go b/pkg/ccl/changefeedccl/cdcevent/event.go index dcb4ba7c87f4..af82a284287c 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event.go +++ b/pkg/ccl/changefeedccl/cdcevent/event.go @@ -12,6 +12,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -21,10 +22,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/cache" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -530,3 +533,29 @@ func TestingGetFamilyIDFromKey( _, familyID, err := decoder.(*eventDecoder).rfCache.tableDescForKey(context.Background(), key, ts) return familyID, err } + +// MakeRowFromTuple converts a SQL datum produced by, for example, SELECT ROW(foo.*), +// into the same kind of cdcevent.Row you'd get as a result of an insert, but without +// the primary key. +func MakeRowFromTuple(evalCtx *eval.Context, t *tree.DTuple) Row { + r := Projection{EventDescriptor: &EventDescriptor{}} + names := t.ResolvedType().TupleLabels() + for i, d := range t.D { + var name string + if names == nil { + name = fmt.Sprintf("col%d", i+1) + } else { + name = names[i] + } + r.AddValueColumn(name, d.ResolvedType()) + if err := r.SetValueDatumAt(evalCtx, i, d); err != nil { + if build.IsRelease() { + log.Warningf(context.Background(), "failed to set row value from tuple due to error %v", err) + _ = r.SetValueDatumAt(evalCtx, i, tree.DNull) + } else { + panic(err) + } + } + } + return Row(r) +} diff --git a/pkg/ccl/changefeedccl/cdcevent/event_test.go b/pkg/ccl/changefeedccl/cdcevent/event_test.go index 401a7dd63ead..d077580ae310 100644 --- a/pkg/ccl/changefeedccl/cdcevent/event_test.go +++ b/pkg/ccl/changefeedccl/cdcevent/event_test.go @@ -18,11 +18,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/distsql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -397,3 +400,61 @@ func slurpDatums(t *testing.T, it Iterator) (res []string) { })) return res } + +func TestMakeRowFromTuple(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + i := tree.NewDInt(1234) + f := tree.NewDFloat(12.34) + s := tree.NewDString("testing") + typ := types.MakeTuple([]*types.T{types.Int, types.Float, types.String}) + unlabeledTuple := tree.NewDTuple(typ, i, f, s) + st := cluster.MakeTestingClusterSettings() + evalCtx := eval.MakeTestingEvalContext(st) + + rowFromUnlabeledTuple := MakeRowFromTuple(&evalCtx, unlabeledTuple) + expectedCols := []struct { + name string + typ *types.T + valAsString string + }{ + {name: "col1", typ: types.Int, valAsString: "1234"}, + {name: "col2", typ: types.Float, valAsString: "12.34"}, + {name: "col3", typ: types.String, valAsString: "testing"}, + } + + remainingCols := expectedCols + + require.NoError(t, rowFromUnlabeledTuple.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error { + current := remainingCols[0] + remainingCols = remainingCols[1:] + require.Equal(t, current.name, col.Name) + require.Equal(t, current.typ, col.Typ) + require.Equal(t, current.valAsString, tree.AsStringWithFlags(d, tree.FmtExport)) + return nil + })) + + require.Empty(t, remainingCols) + + typ.InternalType.TupleLabels = []string{"a", "b", "c"} + labeledTuple := tree.NewDTuple(typ, i, f, s) + + expectedCols[0].name = "a" + expectedCols[1].name = "b" + expectedCols[2].name = "c" + + remainingCols = expectedCols + + rowFromLabeledTuple := MakeRowFromTuple(&evalCtx, labeledTuple) + + require.NoError(t, rowFromLabeledTuple.ForEachColumn().Datum(func(d tree.Datum, col ResultColumn) error { + current := remainingCols[0] + remainingCols = remainingCols[1:] + require.Equal(t, current.name, col.Name) + require.Equal(t, current.typ, col.Typ) + require.Equal(t, current.valAsString, tree.AsStringWithFlags(d, tree.FmtExport)) + return nil + })) + +} diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 3ee88fb62ad9..6f0f1c3ab261 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -252,6 +252,35 @@ func TestChangefeedBasics(t *testing.T) { // cloudStorageTest is a regression test for #36994. } +func TestToJSONAsChangefeed(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testFn := func(t *testing.T, s TestServer, f cdctest.TestFeedFactory) { + sqlDB := sqlutils.MakeSQLRunner(s.DB) + sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING)`) + sqlDB.Exec(t, `INSERT INTO foo values (1, 'hello')`) + sqlDB.CheckQueryResults(t, + `SELECT crdb_internal.to_json_as_changefeed_with_flags(foo.*) from foo`, + [][]string{{`{"after": {"a": 1, "b": "hello"}}`}}, + ) + sqlDB.CheckQueryResults(t, + `SELECT crdb_internal.to_json_as_changefeed_with_flags(foo.*, 'updated', 'diff') from foo`, + [][]string{{`{"after": {"a": 1, "b": "hello"}, "before": null, "updated": "0.0000000000"}`}}, + ) + + sqlDB.CheckQueryResults(t, + `SELECT crdb_internal.to_json_as_changefeed_with_flags(foo.*, 'updated', 'envelope=row') from foo`, + [][]string{{`{"__crdb__": {"updated": "0.0000000000"}, "a": 1, "b": "hello"}`}}, + ) + + sqlDB.ExpectErr(t, `unknown envelope: lobster`, + `SELECT crdb_internal.to_json_as_changefeed_with_flags(foo.*, 'updated', 'envelope=lobster') from foo`) + } + + cdcTest(t, testFn) +} + func TestChangefeedIdleness(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/changefeedccl/encoder_json.go b/pkg/ccl/changefeedccl/encoder_json.go index 5bda71975651..39e16954255c 100644 --- a/pkg/ccl/changefeedccl/encoder_json.go +++ b/pkg/ccl/changefeedccl/encoder_json.go @@ -12,13 +12,19 @@ import ( "bytes" "context" gojson "encoding/json" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" @@ -195,3 +201,62 @@ func (e *jsonEncoder) EncodeResolvedTimestamp( } return gojson.Marshal(jsonEntries) } + +var placeholderCtx = eventContext{topic: "topic"} + +// EncodeAsJSONChangefeedWithFlags implements the crdb_internal.to_json_as_changefeed_with_flags +// builtin. +func EncodeAsJSONChangefeedWithFlags(r cdcevent.Row, flags ...string) ([]byte, error) { + optsMap := make(map[string]string, len(flags)) + for _, f := range flags { + split := strings.SplitN(f, "=", 2) + k := split[0] + var v string + if len(split) == 2 { + v = split[1] + } + optsMap[k] = v + } + opts, err := changefeedbase.MakeStatementOptions(optsMap).GetEncodingOptions() + if err != nil { + return nil, err + } + // If this function ends up needing to be optimized, cache or pool these. + // Nontrivial to do as an encoder generally isn't safe to call on different + // rows in parallel. + e, err := makeJSONEncoder(opts, changefeedbase.Targets{}) + if err != nil { + return nil, err + } + return e.EncodeValue(context.TODO(), placeholderCtx, r, cdcevent.Row{}) + +} + +func init() { + + overload := tree.Overload{ + Types: tree.VariadicType{FixedTypes: []*types.T{types.AnyTuple}, VarType: types.String}, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + row := cdcevent.MakeRowFromTuple(evalCtx, tree.MustBeDTuple(args[0])) + flags := make([]string, len(args)-1) + for i, d := range args[1:] { + flags[i] = string(tree.MustBeDString(d)) + } + o, err := EncodeAsJSONChangefeedWithFlags(row, flags...) + if err != nil { + return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, ``) + } + return tree.NewDBytes(tree.DBytes(o)), nil + }, + Info: "Strings can be of the form 'resolved' or 'resolved=1s'.", + // Probably actually stable, but since this is tightly coupled to changefeed logic by design, + // best to be defensive. + Volatility: volatility.Volatile, + } + + utilccl.RegisterCCLBuiltin("crdb_internal.to_json_as_changefeed_with_flags", + `Encodes a tuple the way a changefeed would output it if it were inserted as a row or emitted by a changefeed expression, and returns the raw bytes. + Flags such as 'diff' modify the encoding as though specified in the WITH portion of a changefeed.`, + overload) +} diff --git a/pkg/ccl/utilccl/BUILD.bazel b/pkg/ccl/utilccl/BUILD.bazel index 3cadd3e0cc7b..526bd045f141 100644 --- a/pkg/ccl/utilccl/BUILD.bazel +++ b/pkg/ccl/utilccl/BUILD.bazel @@ -3,7 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "utilccl", - srcs = ["license_check.go"], + srcs = [ + "builtins.go", + "license_check.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/utilccl", visibility = ["//visibility:public"], deps = [ @@ -14,6 +17,8 @@ go_library( "//pkg/settings/cluster", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/sem/builtins/builtinsregistry", + "//pkg/sql/sem/tree", "//pkg/util/envutil", "//pkg/util/log", "//pkg/util/metric", diff --git a/pkg/ccl/utilccl/builtins.go b/pkg/ccl/utilccl/builtins.go new file mode 100644 index 000000000000..649cce631a8b --- /dev/null +++ b/pkg/ccl/utilccl/builtins.go @@ -0,0 +1,24 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package utilccl + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +// RegisterCCLBuiltin adds a builtin defined in CCL code to the global builtins registry. +func RegisterCCLBuiltin(name string, description string, overload tree.Overload) { + props := tree.FunctionProperties{ + Class: tree.NormalClass, + Category: `CCL-only internal function`, + } + + builtinsregistry.Register(name, &props, []tree.Overload{overload}) +} diff --git a/pkg/cli/clisqlshell/sql.go b/pkg/cli/clisqlshell/sql.go index afff797a760f..b828304ec344 100644 --- a/pkg/cli/clisqlshell/sql.go +++ b/pkg/cli/clisqlshell/sql.go @@ -1552,7 +1552,7 @@ func (c *cliState) handleConnectInternal(cmd []string) error { if dbName == "" { dbName = currURL.GetDatabase() } - fmt.Fprintf(c.iCtx.stdout, "Connection string: %s\n", currURL.ToPQ()) + fmt.Fprintf(c.iCtx.stdout, "Connection string: %s\n", currURL.ToPQRedacted()) fmt.Fprintf(c.iCtx.stdout, "You are connected to database %q as user %q.\n", dbName, currURL.GetUsername()) return nil diff --git a/pkg/cli/doctor.go b/pkg/cli/doctor.go index f396021a6a60..443e3391b0ac 100644 --- a/pkg/cli/doctor.go +++ b/pkg/cli/doctor.go @@ -349,7 +349,7 @@ func fromZipDir( retErr error, ) { // To make parsing user functions code happy. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() descTable = make(doctor.DescriptorTable, 0) if err := slurp(zipDirPath, "system.descriptor.txt", func(row string) error { diff --git a/pkg/cli/interactive_tests/test_connect_cmd.tcl b/pkg/cli/interactive_tests/test_connect_cmd.tcl index e882c0a4bfa5..0beeaae97749 100644 --- a/pkg/cli/interactive_tests/test_connect_cmd.tcl +++ b/pkg/cli/interactive_tests/test_connect_cmd.tcl @@ -26,6 +26,7 @@ send "create user foo with password 'abc';\r" eexpect "CREATE ROLE" eexpect root@ eexpect "/t>" +end_test start_test "Check that the client-side connect cmd prints the current conn details" send "\\c\r" @@ -33,6 +34,7 @@ eexpect "Connection string:" eexpect "You are connected to database \"t\" as user \"root\"" eexpect root@ eexpect "/t>" +end_test start_test "Check that the client-side connect cmd can change databases" send "\\c postgres\r" @@ -184,6 +186,20 @@ end_test send "\\q\r" eexpect eof +start_test "Check that the client-side connect cmd prints the current conn details with password redacted" + +spawn $argv sql --certs-dir=$certs_dir --url=postgres://foo:abc@localhost:26257/defaultdb +eexpect foo@ +send "\\c\r" +eexpect "Connection string: postgresql://foo:~~~~~~@" +eexpect "You are connected to database \"defaultdb\" as user \"foo\"" +eexpect foo@ +eexpect "/defaultdb>" +end_test + +send "\\q\r" +eexpect eof + stop_server $argv # Some more tests with the insecure mode. diff --git a/pkg/cmd/docgen/funcs.go b/pkg/cmd/docgen/funcs.go index d0ee941f220e..f568d7a2f862 100644 --- a/pkg/cmd/docgen/funcs.go +++ b/pkg/cmd/docgen/funcs.go @@ -44,17 +44,17 @@ func init() { } if err := os.WriteFile( - filepath.Join(outDir, "functions.md"), generateFunctions(builtins.AllBuiltinNames, true), 0644, + filepath.Join(outDir, "functions.md"), generateFunctions(builtins.AllBuiltinNames(), true), 0644, ); err != nil { return err } if err := os.WriteFile( - filepath.Join(outDir, "aggregates.md"), generateFunctions(builtins.AllAggregateBuiltinNames, false), 0644, + filepath.Join(outDir, "aggregates.md"), generateFunctions(builtins.AllAggregateBuiltinNames(), false), 0644, ); err != nil { return err } if err := os.WriteFile( - filepath.Join(outDir, "window_functions.md"), generateFunctions(builtins.AllWindowBuiltinNames, false), 0644, + filepath.Join(outDir, "window_functions.md"), generateFunctions(builtins.AllWindowBuiltinNames(), false), 0644, ); err != nil { return err } diff --git a/pkg/jobs/delegate_control_test.go b/pkg/jobs/delegate_control_test.go index 5586a08f5489..0272f38acec2 100644 --- a/pkg/jobs/delegate_control_test.go +++ b/pkg/jobs/delegate_control_test.go @@ -12,9 +12,7 @@ package jobs import ( "context" - gosql "database/sql" "fmt" - "net/url" "strings" "testing" "time" @@ -25,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/stretchr/testify/require" @@ -138,23 +135,6 @@ func TestScheduleControl(t *testing.T) { th.sqlDB.Exec(t, "DROP SCHEDULES "+querySchedules) require.Equal(t, 0, len(th.sqlDB.QueryStr(t, querySchedules))) }) - - t.Run("pause-non-privileged-user", func(t *testing.T) { - scheduleID := makeSchedule("one-schedule", "@daily") - - th.sqlDB.Exec(t, `CREATE USER testuser`) - pgURL, cleanupFunc := sqlutils.PGUrl( - t, th.server.ServingSQLAddr(), "NonPrivileged-testuser", - url.User("testuser"), - ) - defer cleanupFunc() - testuser, err := gosql.Open("postgres", pgURL.String()) - require.NoError(t, err) - defer testuser.Close() - - _, err = testuser.Exec("PAUSE SCHEDULE $1", scheduleID) - require.EqualError(t, err, "pq: only users with the admin role are allowed to PAUSE SCHEDULES") - }) } func TestJobsControlForSchedules(t *testing.T) { diff --git a/pkg/server/pgurl/generate.go b/pkg/server/pgurl/generate.go index 052efd548a96..75560bbdbf29 100644 --- a/pkg/server/pgurl/generate.go +++ b/pkg/server/pgurl/generate.go @@ -103,6 +103,26 @@ func (u *URL) ToPQ() *url.URL { return nu } +// ToPQRedacted converts the URL to a connection string supported +// by drivers using libpq or compatible, with the password redacted. +func (u *URL) ToPQRedacted() *url.URL { + nu, opts := u.baseURL() + + if u.username != "" { + nu.User = url.User(u.username) + } + switch u.authn { + case authnPassword, authnPasswordWithClientCert: + if u.hasPassword { + // Use '~' since it does not need to be escaped. + nu.User = url.UserPassword(u.username, "~~~~~~") + } + } + + nu.RawQuery = opts.Encode() + return nu +} + // String makes URL printable. func (u *URL) String() string { return u.ToPQ().String() } diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index e0f516a94c44..75e9d5734ba2 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -105,6 +105,27 @@ func (n *alterSchemaNode) startExec(params runParams) error { return err } + // Ensure that the new name is a valid schema name. + if err := schemadesc.IsSchemaNameValid(newName); err != nil { + return err + } + + // Check that there isn't a name collision with the new name. + found, _, err := schemaExists(params.ctx, params.p.txn, params.p.Descriptors(), n.db.ID, newName) + if err != nil { + return err + } + if found { + return sqlerrors.NewSchemaAlreadyExistsError(newName) + } + + lookupFlags := tree.CommonLookupFlags{Required: true, AvoidLeased: true} + if err := maybeFailOnDependentDescInRename( + params.ctx, params.p, n.db, n.desc.GetName(), lookupFlags, catalog.Schema, + ); err != nil { + return err + } + if err := params.p.renameSchema( params.ctx, n.db, n.desc, newName, tree.AsStringWithFQNames(n.n, params.Ann()), ); err != nil { @@ -204,20 +225,6 @@ func (p *planner) renameSchema( Name: desc.GetName(), } - // Check that there isn't a name collision with the new name. - found, _, err := schemaExists(ctx, p.txn, p.Descriptors(), db.ID, newName) - if err != nil { - return err - } - if found { - return sqlerrors.NewSchemaAlreadyExistsError(newName) - } - - // Ensure that the new name is a valid schema name. - if err := schemadesc.IsSchemaNameValid(newName); err != nil { - return err - } - // Set the new name for the descriptor. oldName := oldNameKey.GetName() desc.SetName(newName) diff --git a/pkg/sql/builtin_test.go b/pkg/sql/builtin_test.go index fd1524c9dde5..1b3323c87a1d 100644 --- a/pkg/sql/builtin_test.go +++ b/pkg/sql/builtin_test.go @@ -42,7 +42,7 @@ func TestFuncNull(t *testing.T) { } } - for _, name := range builtins.AllBuiltinNames { + for _, name := range builtins.AllBuiltinNames() { switch strings.ToLower(name) { case "crdb_internal.force_panic", "crdb_internal.force_log_fatal", "pg_sleep": continue diff --git a/pkg/sql/catalog/schemaexpr/check_constraint_test.go b/pkg/sql/catalog/schemaexpr/check_constraint_test.go index 82f155aa20d3..ffd06d33e84a 100644 --- a/pkg/sql/catalog/schemaexpr/check_constraint_test.go +++ b/pkg/sql/catalog/schemaexpr/check_constraint_test.go @@ -26,7 +26,7 @@ func TestCheckConstraintBuilder_Build(t *testing.T) { semaCtx := tree.MakeSemaContext() // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() database := tree.Name("foo") table := tree.Name("bar") diff --git a/pkg/sql/catalog/schemaexpr/expr_test.go b/pkg/sql/catalog/schemaexpr/expr_test.go index 591e9699eb0a..395558fef91a 100644 --- a/pkg/sql/catalog/schemaexpr/expr_test.go +++ b/pkg/sql/catalog/schemaexpr/expr_test.go @@ -27,7 +27,7 @@ func TestValidateExpr(t *testing.T) { semaCtx := tree.MakeSemaContext() // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() database := tree.Name("foo") table := tree.Name("bar") @@ -111,7 +111,7 @@ func TestValidateExpr(t *testing.T) { func TestExtractColumnIDs(t *testing.T) { // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() table := tree.Name("foo") desc := testTableDesc( @@ -154,7 +154,7 @@ func TestExtractColumnIDs(t *testing.T) { func TestValidColumnReferences(t *testing.T) { // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() table := tree.Name("foo") desc := testTableDesc( diff --git a/pkg/sql/catalog/schemaexpr/partial_index_test.go b/pkg/sql/catalog/schemaexpr/partial_index_test.go index 97c882ba7ee7..e4a150faafa6 100644 --- a/pkg/sql/catalog/schemaexpr/partial_index_test.go +++ b/pkg/sql/catalog/schemaexpr/partial_index_test.go @@ -26,7 +26,7 @@ func TestIndexPredicateValidator_Validate(t *testing.T) { semaCtx := tree.MakeSemaContext() // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() database := tree.Name("foo") table := tree.Name("bar") diff --git a/pkg/sql/colexec/builtin_funcs_test.go b/pkg/sql/colexec/builtin_funcs_test.go index 91e941229161..15b1db776935 100644 --- a/pkg/sql/colexec/builtin_funcs_test.go +++ b/pkg/sql/colexec/builtin_funcs_test.go @@ -37,7 +37,7 @@ func TestBasicBuiltinFunctions(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) // Trick to get the init() for the builtins package to run. - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() ctx := context.Background() st := cluster.MakeTestingClusterSettings() evalCtx := eval.MakeTestingEvalContext(st) @@ -179,7 +179,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b } func BenchmarkBuiltinFunctions(b *testing.B) { - _ = builtins.AllBuiltinNames + _ = builtins.AllBuiltinNames() for _, useSel := range []bool{true, false} { for _, hasNulls := range []bool{true, false} { b.Run(fmt.Sprintf("useSel=%t,hasNulls=%t", useSel, hasNulls), func(b *testing.B) { diff --git a/pkg/sql/control_schedules.go b/pkg/sql/control_schedules.go index 5250ca16cac6..a246558e643a 100644 --- a/pkg/sql/control_schedules.go +++ b/pkg/sql/control_schedules.go @@ -19,6 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -57,19 +59,22 @@ func JobSchedulerEnv(execCfg *ExecutorConfig) scheduledjobs.JobSchedulerEnv { return scheduledjobs.ProdJobSchedulerEnv } -// loadSchedule loads schedule information. +// loadSchedule loads schedule information as the node user. func loadSchedule(params runParams, scheduleID tree.Datum) (*jobs.ScheduledJob, error) { env := JobSchedulerEnv(params.ExecCfg()) schedule := jobs.NewScheduledJob(env) // Load schedule expression. This is needed for resume command, but we // also use this query to check for the schedule existence. + // + // Run the query as the node user since we perform our own privilege checks + // before using the returned schedule. datums, cols, err := params.ExecCfg().InternalExecutor.QueryRowExWithCols( params.ctx, "load-schedule", - params.p.Txn(), sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + params.p.Txn(), sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, fmt.Sprintf( - "SELECT schedule_id, next_run, schedule_expr, executor_type, execution_args FROM %s WHERE schedule_id = $1", + "SELECT schedule_id, next_run, schedule_expr, executor_type, execution_args, owner FROM %s WHERE schedule_id = $1", env.ScheduledJobsTableName(), ), scheduleID) @@ -136,6 +141,16 @@ func (n *controlSchedulesNode) startExec(params runParams) error { continue // not an error if schedule does not exist } + isAdmin, err := params.p.UserHasAdminRole(params.ctx, params.p.User()) + if err != nil { + return err + } + isOwner := schedule.Owner() == params.p.User() + if !isAdmin && !isOwner { + return pgerror.Newf(pgcode.InsufficientPrivilege, "must be admin or owner of the "+ + "schedule %d to %s it", schedule.ScheduleID(), n.command.String()) + } + switch n.command { case tree.PauseSchedule: schedule.Pause() diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 56d512c8a260..da97d27fb104 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2428,7 +2428,7 @@ CREATE TABLE crdb_internal.builtin_functions ( details STRING NOT NULL )`, populate: func(ctx context.Context, _ *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { - for _, name := range builtins.AllBuiltinNames { + for _, name := range builtins.AllBuiltinNames() { props, overloads := builtinsregistry.GetBuiltinProperties(name) for _, f := range overloads { if err := addRow( @@ -2649,7 +2649,15 @@ CREATE TABLE crdb_internal.create_function_statements ( } for i := range treeNode.Options { if body, ok := treeNode.Options[i].(tree.FunctionBodyStr); ok { - stmtStrs := strings.Split(string(body), "\n") + typeReplacedBody, err := formatFunctionQueryTypesForDisplay(ctx, &p.semaCtx, p.SessionData(), string(body)) + if err != nil { + return err + } + seqReplacedBody, err := formatQuerySequencesForDisplay(ctx, &p.semaCtx, typeReplacedBody, true /* multiStmt */) + if err != nil { + return err + } + stmtStrs := strings.Split(seqReplacedBody, "\n") for i := range stmtStrs { stmtStrs[i] = "\t" + stmtStrs[i] } diff --git a/pkg/sql/create_function.go b/pkg/sql/create_function.go index 4c137a147b4f..03ddc336bc2a 100644 --- a/pkg/sql/create_function.go +++ b/pkg/sql/create_function.go @@ -15,6 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -58,6 +59,12 @@ func (n *createFunctionNode) startExec(params runParams) error { return unimplemented.NewWithIssue(85144, "CREATE FUNCTION...sql_body unimplemented") } + for _, dep := range n.planDeps { + if dbID := dep.desc.GetParentID(); dbID != n.dbDesc.GetID() && dbID != keys.SystemDatabaseID { + return pgerror.Newf(pgcode.FeatureNotSupported, "the function cannot refer to other databases") + } + } + scDesc, err := params.p.descCollection.GetMutableSchemaByName( params.ctx, params.p.Txn(), n.dbDesc, n.scDesc.GetName(), tree.SchemaLookupFlags{Required: true, RequireMutable: true}, @@ -67,15 +74,22 @@ func (n *createFunctionNode) startExec(params runParams) error { } mutScDesc := scDesc.(*schemadesc.Mutable) - udfMutableDesc, isNew, err := n.getMutableFuncDesc(mutScDesc, params) - if err != nil { - return err - } + var retErr error + params.p.runWithOptions(resolveFlags{contextDatabaseID: n.dbDesc.GetID()}, func() { + retErr = func() error { + udfMutableDesc, isNew, err := n.getMutableFuncDesc(mutScDesc, params) + if err != nil { + return err + } - if isNew { - return n.createNewFunction(udfMutableDesc, mutScDesc, params) - } - return n.replaceFunction(udfMutableDesc, params) + if isNew { + return n.createNewFunction(udfMutableDesc, mutScDesc, params) + } + return n.replaceFunction(udfMutableDesc, params) + }() + }) + + return retErr } func (*createFunctionNode) Next(params runParams) (bool, error) { return false, nil } @@ -404,7 +418,11 @@ func setFuncOption(params runParams, udfDesc *funcdesc.Mutable, option tree.Func if err != nil { return err } - udfDesc.SetFuncBody(seqReplacedFuncBody) + typeReplacedFuncBody, err := serializeUserDefinedTypes(params.ctx, params.p.SemaCtx(), seqReplacedFuncBody, true /* multiStmt */) + if err != nil { + return err + } + udfDesc.SetFuncBody(typeReplacedFuncBody) default: return pgerror.Newf(pgcode.InvalidParameterValue, "Unknown function option %q", t) } diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 0d1abf794304..77845b370d82 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -414,7 +414,7 @@ func makeViewTableDesc( desc.ViewQuery = sequenceReplacedQuery } - typeReplacedQuery, err := serializeUserDefinedTypes(ctx, semaCtx, desc.ViewQuery) + typeReplacedQuery, err := serializeUserDefinedTypes(ctx, semaCtx, desc.ViewQuery, false /* multiStmt */) if err != nil { return tabledesc.Mutable{}, err } @@ -493,7 +493,7 @@ func replaceSeqNamesWithIDs( // and serialize any user defined types, so that renaming the type // does not corrupt the view. func serializeUserDefinedTypes( - ctx context.Context, semaCtx *tree.SemaContext, viewQuery string, + ctx context.Context, semaCtx *tree.SemaContext, queries string, multiStmt bool, ) (string, error) { replaceFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { var innerExpr tree.Expr @@ -534,16 +534,39 @@ func serializeUserDefinedTypes( return false, parsedExpr, nil } - stmt, err := parser.ParseOne(viewQuery) - if err != nil { - return "", errors.Wrap(err, "failed to parse view query") + var stmts tree.Statements + if multiStmt { + parsedStmts, err := parser.Parse(queries) + if err != nil { + return "", errors.Wrap(err, "failed to parse query") + } + stmts = make(tree.Statements, len(parsedStmts)) + for i, stmt := range parsedStmts { + stmts[i] = stmt.AST + } + } else { + stmt, err := parser.ParseOne(queries) + if err != nil { + return "", errors.Wrap(err, "failed to parse query") + } + stmts = tree.Statements{stmt.AST} } - newStmt, err := tree.SimpleStmtVisit(stmt.AST, replaceFunc) - if err != nil { - return "", err + fmtCtx := tree.NewFmtCtx(tree.FmtSimple) + for i, stmt := range stmts { + newStmt, err := tree.SimpleStmtVisit(stmt, replaceFunc) + if err != nil { + return "", err + } + if i > 0 { + fmtCtx.WriteString("\n") + } + fmtCtx.FormatNode(newStmt) + if multiStmt { + fmtCtx.WriteString(";") + } } - return newStmt.String(), nil + return fmtCtx.CloseAndGetString(), nil } // replaceViewDesc modifies and returns the input view descriptor changed diff --git a/pkg/sql/drop_function_test.go b/pkg/sql/drop_function_test.go index 3636d445208f..a0ed70cc7353 100644 --- a/pkg/sql/drop_function_test.go +++ b/pkg/sql/drop_function_test.go @@ -353,6 +353,10 @@ $$; testName: "drop database", stmt: "DROP DATABASE test_db CASCADE", }, + { + testName: "drop schema", + stmt: "DROP SCHEMA test_sc CASCADE", + }, } for _, tc := range testCases { diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 702316530153..18f786594687 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -4672,7 +4672,7 @@ FROM pg_proc p JOIN pg_type t ON t.typinput = p.oid WHERE t.typname = '_int4' ---- -2018 array_in array_in +2013 array_in array_in ## #16285 ## int2vectors should be 0-indexed @@ -4710,7 +4710,7 @@ SELECT cur_max_builtin_oid FROM [SELECT max(oid) as cur_max_builtin_oid FROM pg_ query TT SELECT proname, oid FROM pg_catalog.pg_proc WHERE oid = $cur_max_builtin_oid ---- -to_regtype 2038 +pg_get_functiondef 2035 ## Ensure that unnest works with oid wrapper arrays diff --git a/pkg/sql/logictest/testdata/logic_test/pgoidtype b/pkg/sql/logictest/testdata/logic_test/pgoidtype index aacccd73027f..5f363e125bbc 100644 --- a/pkg/sql/logictest/testdata/logic_test/pgoidtype +++ b/pkg/sql/logictest/testdata/logic_test/pgoidtype @@ -83,7 +83,7 @@ WHERE relname = 'pg_constraint' query OOOO SELECT 'upper'::REGPROC, 'upper'::REGPROCEDURE, 'pg_catalog.upper'::REGPROCEDURE, 'upper'::REGPROC::OID ---- -upper upper upper 833 +upper upper upper 829 query error invalid function name SELECT 'invalid.more.pg_catalog.upper'::REGPROCEDURE @@ -91,7 +91,7 @@ SELECT 'invalid.more.pg_catalog.upper'::REGPROCEDURE query OOO SELECT 'upper(int)'::REGPROC, 'upper(int)'::REGPROCEDURE, 'upper(int)'::REGPROC::OID ---- -upper upper 833 +upper upper 829 query error unknown function: blah\(\) SELECT 'blah(ignored, ignored)'::REGPROC, 'blah(ignored, ignored)'::REGPROCEDURE diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index ff40b4b2e492..fc7841c4c882 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -132,24 +132,31 @@ CREATE SCHEMA yourschema statement error pq: schema "yourschema" already exists ALTER SCHEMA myschema RENAME TO yourschema -statement ok +statement error cannot rename schema because relation "test.myschema.v" depends on relation "test.myschema.tb" ALTER SCHEMA myschema RENAME TO myschema2 +statement ok +CREATE SCHEMA myschema2; +CREATE TABLE myschema2.tb2 (a INT PRIMARY KEY); + +statement ok +ALTER SCHEMA myschema2 RENAME TO myschema3 + # We should be able to resolve objects under the new schema name. query T -SELECT * FROM myschema2.tb2 +SELECT * FROM myschema3.tb2 ---- # The names should be drained after executing, so we should be able # to make another schema with the old name. statement ok -CREATE SCHEMA myschema +CREATE SCHEMA myschema2 statement ok BEGIN statement ok -ALTER SCHEMA myschema RENAME TO another_schema +ALTER SCHEMA myschema2 RENAME TO another_schema statement ok ALTER SCHEMA another_schema RENAME TO another_one @@ -163,39 +170,41 @@ CREATE SCHEMA empty; DROP SCHEMA empty let $schema_id -SELECT id FROM system.namespace WHERE name = 'myschema' +SELECT id FROM system.namespace WHERE name = 'myschema2' -# Create some objects under myschema, and have them reference some objects +# Create some objects under myschema2, and have them reference some objects # in other schemas. statement ok -CREATE TABLE myschema.myschema_t1 (x INT); -CREATE TABLE myschema.myschema_t2 (x INT); -CREATE SEQUENCE myschema.myschema_seq1; -CREATE TABLE myschema.myschema_t3 (x INT DEFAULT nextval('myschema.myschema_seq1')); -CREATE TYPE myschema.myschema_ty1 AS ENUM ('schema'); +CREATE TABLE myschema2.myschema_t1 (x INT); +CREATE TABLE myschema2.myschema_t2 (x INT); +CREATE SEQUENCE myschema2.myschema_seq1; +CREATE TABLE myschema2.myschema_t3 (x INT DEFAULT nextval('myschema2.myschema_seq1')); +CREATE TYPE myschema2.myschema_ty1 AS ENUM ('schema'); CREATE SCHEMA otherschema; -CREATE VIEW otherschema.otherschema_v1 AS SELECT x FROM myschema.myschema_t1; +CREATE VIEW otherschema.otherschema_v1 AS SELECT x FROM myschema2.myschema_t1; CREATE TABLE otherschema.otherschema_t1 (x INT); -CREATE SEQUENCE otherschema.otherschema_seq1 OWNED BY myschema.myschema_t1.x; +CREATE SEQUENCE otherschema.otherschema_seq1 OWNED BY myschema2.myschema_t1.x; -statement error pq: schema "myschema" is not empty and CASCADE was not specified -DROP SCHEMA myschema +statement error pq: schema "myschema2" is not empty and CASCADE was not specified +DROP SCHEMA myschema2 # Now drop with cascade. statement ok -DROP SCHEMA myschema CASCADE +DROP SCHEMA myschema2 CASCADE query T -SELECT table_name FROM [SHOW TABLES] WHERE table_name LIKE 'myschema%' OR table_name LIKE 'otherschema%' +SELECT table_name FROM [SHOW TABLES] WHERE table_name LIKE 'myschema2%' OR table_name LIKE 'otherschema%' ---- otherschema_t1 query T -SELECT name FROM [SHOW ENUMS] WHERE name LIKE 'myschema%' +SELECT name FROM [SHOW ENUMS] WHERE name LIKE 'myschema2%' +---- # The schema should be gone. query I -SELECT id FROM system.namespace WHERE name = 'myschema' +SELECT id FROM system.namespace WHERE name = 'myschema2' +---- query IT SELECT * FROM system.descriptor WHERE id = $schema_id diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 83729583222d..ba3c4dba142a 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -143,7 +143,7 @@ CREATE FUNCTION public.f(IN a test.public.notmyworkday) SELECT a FROM test.public.t; SELECT b FROM test.public.t@t_idx_b; SELECT c FROM test.public.t@t_idx_c; - SELECT nextval(114:::REGCLASS); + SELECT nextval('public.sq1'::REGCLASS); $$ statement error pq: unimplemented: alter function depends on extension not supported.* @@ -1236,6 +1236,275 @@ SELECT f_seq_qualified_name_quoted() ---- 2 +subtest udt_rewrite + +statement ok +CREATE FUNCTION f_udt_rewrite() RETURNS notmyworkday LANGUAGE SQL AS $$ SELECT 'Monday':: notmyworkday $$; + +query T +SELECT @2 FROM [SHOW CREATE FUNCTION f_udt_rewrite]; +---- +CREATE FUNCTION public.f_udt_rewrite() + RETURNS test.public.notmyworkday + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'Monday':::test.public.notmyworkday; +$$ + +query T +SELECT f_udt_rewrite() +---- +Monday + +statement ok +ALTER TYPE notmyworkday RENAME TO notmyworkday_new; + +query T +SELECT f_udt_rewrite() +---- +Monday + +statement ok +ALTER TYPE notmyworkday_new RENAME TO notmyworkday; + +query T +SELECT f_udt_rewrite() +---- +Monday + +subtest cross_db + +statement ok +CREATE DATABASE cross_db1; +CREATE SCHEMA cross_db1.sc; +CREATE TYPE cross_db1.sc.workday AS ENUM ('MON'); +CREATE TABLE cross_db1.sc.tbl(a INT PRIMARY KEY, b cross_db1.sc.workday); +CREATE VIEW cross_db1.sc.v AS SELECT a FROM cross_db1.sc.tbl; + +statement error pq: cross database type references are not supported: cross_db1.sc.workday +CREATE FUNCTION f_cross_db(cross_db1.sc.workday) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +statement error pq: cross database type references are not supported: cross_db1.sc.workday +CREATE FUNCTION f_cross_db() RETURNS cross_db1.sc.workday LANGUAGE SQL AS $$ SELECT 'MON'::cross_db1.sc.workday $$; + +statement error pq: the function cannot refer to other databases +CREATE FUNCTION f_cross_db() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM cross_db1.sc.tbl $$; + +statement error pq: the function cannot refer to other databases +CREATE FUNCTION f_cross_db() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM cross_db1.sc.v $$; + +subtest db_rename + +statement ok +CREATE DATABASE rename_db1; +SET DATABASE = rename_db1; + +statement ok +CREATE SCHEMA sc1; +CREATE SCHEMA sc2; +CREATE TYPE sc1.workday AS ENUM ('Mon'); +CREATE TABLE sc1.tbl(a INT PRIMARY KEY); +CREATE SEQUENCE sc1.sq; + +statement ok +CREATE FUNCTION sc1.f_tbl() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM sc1.tbl $$; +CREATE FUNCTION sc1.f_type() RETURNS sc1.workday LANGUAGE SQL AS $$ SELECT 'Mon'::sc1.workday $$; +CREATE FUNCTION sc1.f_seq() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('sc1.sq') $$; +CREATE FUNCTION sc2.f_tbl() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM sc1.tbl $$; +CREATE FUNCTION sc2.f_type() RETURNS sc1.workday LANGUAGE SQL AS $$ SELECT 'Mon'::sc1.workday $$; +CREATE FUNCTION sc2.f_seq() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('sc1.sq') $$; + +query T +SELECT sc1.f_type() +---- +Mon + +query I +SELECT sc1.f_seq() +---- +1 + +query T +SELECT sc2.f_type() +---- +Mon + +query I +SELECT sc2.f_seq() +---- +2 + +statement error pq: cannot rename database because relation "rename_db1.sc1.f_tbl" depends on relation "rename_db1.sc1.tbl" +ALTER DATABASE rename_db1 RENAME TO rename_db2; + +statement ok +DROP FUNCTION sc1.f_tbl() + +statement error pq: cannot rename database because relation "rename_db1.sc2.f_tbl" depends on relation "rename_db1.sc1.tbl" +ALTER DATABASE rename_db1 RENAME TO rename_db2; + +statement ok +DROP FUNCTION sc2.f_tbl() + +statement ok +ALTER DATABASE rename_db1 RENAME TO rename_db2; + +# Make sure that db renaming does not affect types and sequences in UDF. +query T +SELECT sc1.f_type() +---- +Mon + +query I +SELECT sc1.f_seq() +---- +3 + +query T +SELECT sc2.f_type() +---- +Mon + +query I +SELECT sc2.f_seq() +---- +4 + +statement ok +SET DATABASE = test + +statement ok +CREATE DATABASE rename_sc1; +SET DATABASE = rename_sc1; + +statement ok +CREATE SCHEMA sc1; +CREATE SCHEMA sc2; +CREATE TYPE sc1.workday AS ENUM ('Mon'); +CREATE TABLE sc1.tbl(a INT PRIMARY KEY); +CREATE SEQUENCE sc1.sq; + +statement ok +CREATE FUNCTION sc1.f_tbl() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM sc1.tbl $$; +CREATE FUNCTION sc1.f_type() RETURNS sc1.workday LANGUAGE SQL AS $$ SELECT 'Mon'::sc1.workday $$; +CREATE FUNCTION sc1.f_seq() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('sc1.sq') $$; +CREATE FUNCTION sc2.f_tbl() RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM sc1.tbl $$; +CREATE FUNCTION sc2.f_type() RETURNS sc1.workday LANGUAGE SQL AS $$ SELECT 'Mon'::sc1.workday $$; +CREATE FUNCTION sc2.f_seq() RETURNS INT LANGUAGE SQL AS $$ SELECT nextval('sc1.sq') $$; + +query T +SELECT sc1.f_type() +---- +Mon + +query I +SELECT sc1.f_seq() +---- +5 + +query T +SELECT sc2.f_type() +---- +Mon + +query I +SELECT sc2.f_seq() +---- +6 + +statement error pq: cannot rename schema because relation "rename_sc1.sc1.f_tbl" depends on relation "rename_sc1.sc1.tbl" +ALTER SCHEMA sc1 RENAME TO sc1_new + +statement ok +DROP FUNCTION sc1.f_tbl() + +statement error pq: cannot rename schema because relation "rename_sc1.sc2.f_tbl" depends on relation "rename_sc1.sc1.tbl" +ALTER SCHEMA sc1 RENAME TO sc1_new + +statement ok +DROP FUNCTION sc2.f_tbl() + +statement ok +ALTER SCHEMA sc1 RENAME TO sc1_new + +# Make sure that db renaming does not affect types and sequences in UDF. +query T +SELECT sc1.f_type() +---- +Mon + +query I +SELECT sc1.f_seq() +---- +7 + +query T +SELECT sc2.f_type() +---- +Mon + +query I +SELECT sc2.f_seq() +---- +8 + +statement ok +SET DATABASE = test + +subtest select_from_seq_rename + +statement ok +CREATE DATABASE tdb_seq_select; +SET DATABASE = tdb_seq_select; + +statement ok +CREATE SCHEMA sc; +CREATE SEQUENCE sc.sq; +CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT last_value FROM sc.sq $$; + +query I +SELECT f() +---- +0 + +statement ok +ALTER SEQUENCE sc.sq RENAME TO sq_new; + +statement error pq: relation "tdb_seq_select.sc.sq" does not exist +SELECT f() + +statement ok +ALTER SEQUENCE sc.sq_new RENAME TO sq; +SELECT f(); + +statement ok +ALTER SCHEMA sc RENAME TO sc_new; + +statement error pq: unknown schema "sc" +SELECT f() + +statement ok +ALTER SCHEMA sc_new RENAME TO sc; +SELECT f() + +statement ok +ALTER DATABASE tdb_seq_select RENAME TO tdb_seq_select_new; +SET DATABASE = tdb_seq_select_new; + +statement error pq: database "tdb_seq_select" does not exist +SELECT f() + +statement ok +ALTER DATABASE tdb_seq_select_new RENAME TO tdb_seq_select; +SET DATABASE = tdb_seq_select; +SELECT f() + +statement ok +SET DATABASE = test; + subtest execution statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 2d53a9b9ed2e..61796b891858 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -1361,3 +1361,135 @@ query I SELECT * FROM v_seq_rewrite_quoted ---- 2 + +subtest rename_schema + +statement ok +CREATE DATABASE rename_sc1; +SET DATABASE = rename_sc1; + +statement ok +CREATE SCHEMA sc1; +CREATE SCHEMA sc2; +CREATE TYPE sc1.workday AS ENUM ('Mon'); +CREATE TABLE sc1.tbl(a INT PRIMARY KEY); +CREATE SEQUENCE sc1.sq; + +statement ok +CREATE VIEW sc1.v_tbl AS SELECT a FROM sc1.tbl; +CREATE VIEW sc1.v_type AS SELECT 'Mon'::sc1.workday; +CREATE VIEW sc1.v_seq AS SELECT nextval('sc1.sq'); +CREATE VIEW sc2.v_tbl AS SELECT a FROM sc1.tbl; +CREATE VIEW sc2.v_type AS SELECT 'Mon'::sc1.workday; +CREATE VIEW sc2.v_seq AS SELECT nextval('sc1.sq'); + +query T +SELECT * FROM sc1.v_type +---- +Mon + +query I +SELECT * FROM sc1.v_seq +---- +1 + +query T +SELECT * FROM sc2.v_type +---- +Mon + +query I +SELECT * FROM sc2.v_seq +---- +2 + +statement error pq: cannot rename schema because relation "rename_sc1.sc1.v_tbl" depends on relation "rename_sc1.sc1.tbl" +ALTER SCHEMA sc1 RENAME TO sc1_new + +statement ok +DROP VIEW sc1.v_tbl + +statement error pq: cannot rename schema because relation "rename_sc1.sc2.v_tbl" depends on relation "rename_sc1.sc1.tbl" +ALTER SCHEMA sc1 RENAME TO sc1_new + +statement ok +DROP VIEW sc2.v_tbl + +statement ok +ALTER SCHEMA sc1 RENAME TO sc1_new + +# Make sure that db renaming does not affect types and sequences in UDF. +query T +SELECT * FROM sc1_new.v_type +---- +Mon + +query I +SELECT * FROM sc1_new.v_seq +---- +3 + +query T +SELECT * FROM sc2.v_type +---- +Mon + +query I +SELECT * FROM sc2.v_seq +---- +4 + +statement ok +SET DATABASE = test + +subtest select_from_seq_rename + +statement ok +CREATE DATABASE tdb_seq_select; +SET DATABASE = tdb_seq_select; + +statement ok +CREATE SCHEMA sc; +CREATE SEQUENCE sc.sq; +CREATE VIEW v AS SELECT last_value FROM sc.sq; + +query I +SELECT * FROM v; +---- +0 + +statement ok +ALTER SEQUENCE sc.sq RENAME TO sq_new; + +statement error pq: relation "tdb_seq_select.sc.sq" does not exist +SELECT * FROM v; + +statement ok +ALTER SEQUENCE sc.sq_new RENAME TO sq; +SELECT * FROM v; + +statement ok +ALTER SCHEMA sc RENAME TO sc_new; + +statement error pq: unknown schema "sc" +SELECT * FROM v; + +statement ok +ALTER SCHEMA sc_new RENAME TO sc; +SELECT * FROM v; + +statement ok +ALTER DATABASE tdb_seq_select RENAME TO tdb_seq_select_new; +SET DATABASE = tdb_seq_select_new; + +statement error pq: database "tdb_seq_select" does not exist +SELECT * FROM v; + +statement ok +ALTER DATABASE tdb_seq_select_new RENAME TO tdb_seq_select; +SET DATABASE = tdb_seq_select; +SELECT * FROM v; + +statement ok +SET DATABASE = test; + diff --git a/pkg/sql/opt/memo/typing_test.go b/pkg/sql/opt/memo/typing_test.go index 6e26bbc52014..5c1a00703f7d 100644 --- a/pkg/sql/opt/memo/typing_test.go +++ b/pkg/sql/opt/memo/typing_test.go @@ -132,7 +132,7 @@ func TestTypingComparisonAssumptions(t *testing.T) { // 2. The return type of overloads is fixed. // 3. The return type for min/max aggregates is same as type of argument. func TestTypingAggregateAssumptions(t *testing.T) { - for _, name := range builtins.AllAggregateBuiltinNames { + for _, name := range builtins.AllAggregateBuiltinNames() { if name == builtins.AnyNotNull || name == "percentile_disc" || name == "percentile_cont" { diff --git a/pkg/sql/opt/optbuilder/misc_statements.go b/pkg/sql/opt/optbuilder/misc_statements.go index fc6079222cff..f7de775f48bc 100644 --- a/pkg/sql/opt/optbuilder/misc_statements.go +++ b/pkg/sql/opt/optbuilder/misc_statements.go @@ -108,10 +108,6 @@ func (b *Builder) buildCancelSessions(n *tree.CancelSessions, inScope *scope) (o func (b *Builder) buildControlSchedules( n *tree.ControlSchedules, inScope *scope, ) (outScope *scope) { - if err := b.catalog.RequireAdminRole(b.ctx, n.StatementTag()); err != nil { - panic(err) - } - // We don't allow the input statement to reference outer columns, so we // pass a "blank" scope rather than inScope. emptyScope := b.allocScope() diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 6c8681c9ae66..83bd1b7abcef 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -2298,7 +2298,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, err := forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { nspOid := h.NamespaceOid(db.GetID(), pgCatalogName) - for _, name := range builtins.AllBuiltinNames { + for _, name := range builtins.AllBuiltinNames() { // parser.Builtins contains duplicate uppercase and lowercase keys. // Only return the lowercase ones for compatibility with postgres. var first rune @@ -4295,7 +4295,7 @@ https://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html`, h := makeOidHasher() return forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { - for _, name := range builtins.AllAggregateBuiltinNames { + for _, name := range builtins.AllAggregateBuiltinNames() { if name == builtins.AnyNotNull { // any_not_null is treated as a special case. continue diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index e972cbc9cf95..bc6cb53fbb34 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -12,20 +12,16 @@ package sql import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr" - "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/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" @@ -131,112 +127,9 @@ func (n *renameDatabaseNode) startExec(params runParams) error { return err } for _, schema := range schemas { - tbNames, _, err := p.Descriptors().GetObjectNamesAndIDs( - ctx, - p.txn, - dbDesc, - schema, - tree.DatabaseListFlags{ - CommonLookupFlags: lookupFlags, - ExplicitPrefix: true, - }, - ) - if err != nil { + if err := maybeFailOnDependentDescInRename(ctx, p, dbDesc, schema, lookupFlags, catalog.Database); err != nil { return err } - lookupFlags.Required = false - // TODO(ajwerner): Make this do something better than one-at-a-time lookups - // followed by catalogkv reads on each dependency. - for i := range tbNames { - found, tbDesc, err := p.Descriptors().GetImmutableTableByName( - ctx, p.txn, &tbNames[i], tree.ObjectLookupFlags{CommonLookupFlags: lookupFlags}, - ) - if err != nil { - return err - } - if !found { - continue - } - - if err := tbDesc.ForeachDependedOnBy(func(dependedOn *descpb.TableDescriptor_Reference) error { - dependentDesc, err := p.Descriptors().Direct().MustGetTableDescByID(ctx, p.txn, dependedOn.ID) - if err != nil { - return err - } - - isAllowed, referencedCol, err := isAllowedDependentDescInRenameDatabase( - ctx, - dependedOn, - tbDesc, - dependentDesc, - dbDesc.GetName(), - ) - if err != nil { - return err - } - if isAllowed { - return nil - } - - tbTableName := tree.MakeTableNameWithSchema( - tree.Name(dbDesc.GetName()), - tree.Name(schema), - tree.Name(tbDesc.GetName()), - ) - var dependentDescQualifiedString string - if dbDesc.GetID() != dependentDesc.GetParentID() || tbDesc.GetParentSchemaID() != dependentDesc.GetParentSchemaID() { - descFQName, err := p.getQualifiedTableName(ctx, dependentDesc) - if err != nil { - log.Warningf( - ctx, - "unable to retrieve fully-qualified name of %s (id: %d): %v", - tbTableName.String(), - dependentDesc.GetID(), - err, - ) - return sqlerrors.NewDependentObjectErrorf( - "cannot rename database because a relation depends on relation %q", - tbTableName.String()) - } - dependentDescQualifiedString = descFQName.FQString() - } else { - dependentDescTableName := tree.MakeTableNameWithSchema( - tree.Name(dbDesc.GetName()), - tree.Name(schema), - tree.Name(dependentDesc.GetName()), - ) - dependentDescQualifiedString = dependentDescTableName.String() - } - depErr := sqlerrors.NewDependentObjectErrorf( - "cannot rename database because relation %q depends on relation %q", - dependentDescQualifiedString, - tbTableName.String(), - ) - - // We can have a more specific error message for sequences. - if tbDesc.IsSequence() { - hint := fmt.Sprintf( - "you can drop the column default %q of %q referencing %q", - referencedCol, - tbTableName.String(), - dependentDescQualifiedString, - ) - if dependentDesc.GetParentID() == dbDesc.GetID() { - hint += fmt.Sprintf( - " or modify the default to not reference the database name %q", - dbDesc.GetName(), - ) - } - return errors.WithHint(depErr, hint) - } - - // Otherwise, we default to the view error message. - return errors.WithHintf(depErr, - "you can drop %q instead", dependentDescQualifiedString) - }); err != nil { - return err - } - } } if err := p.renameDatabase(ctx, dbDesc, n.newName, tree.AsStringWithFQNames(n.n, params.Ann())); err != nil { @@ -253,81 +146,129 @@ func (n *renameDatabaseNode) startExec(params runParams) error { }) } -// isAllowedDependentDescInRename determines when rename database is allowed with -// a given {tbDesc, dependentDesc} with the relationship dependedOn on a db named dbName. -// Returns a bool representing whether it's allowed, a string indicating the column name -// found to contain the database (if it exists), and an error if any. -// This is a workaround for #45411 until #34416 is resolved. -func isAllowedDependentDescInRenameDatabase( +func getQualifiedDependentObjectName( ctx context.Context, - dependedOn *descpb.TableDescriptor_Reference, - tbDesc catalog.TableDescriptor, - dependentDesc catalog.TableDescriptor, + p *planner, dbName string, -) (bool, string, error) { - // If it is a sequence, and it does not contain the database name, then we have - // no reason to block it's deletion. - if !tbDesc.IsSequence() { - return false, "", nil - } - - colIDs := util.MakeFastIntSet() - for _, colID := range dependedOn.ColumnIDs { - colIDs.Add(int(colID)) - } + scName string, + desc catalog.TableDescriptor, + depDesc catalog.Descriptor, +) (string, error) { + tbTableName := tree.MakeTableNameWithSchema( + tree.Name(dbName), + tree.Name(scName), + tree.Name(desc.GetName()), + ) - for _, column := range dependentDesc.PublicColumns() { - if !colIDs.Contains(int(column.GetID())) { - continue + if desc.GetParentID() != depDesc.GetParentID() || desc.GetParentSchemaID() != depDesc.GetParentSchemaID() { + var descFQName tree.ObjectName + var err error + switch t := depDesc.(type) { + case catalog.TableDescriptor: + descFQName, err = p.getQualifiedTableName(ctx, t) + case catalog.FunctionDescriptor: + descFQName, err = p.getQualifiedFunctionName(ctx, t) + default: + return "", errors.AssertionFailedf("expected only function or table descriptor, but got %s", t.DescriptorType()) } - colIDs.Remove(int(column.GetID())) - - if !column.HasDefault() { - return false, "", errors.AssertionFailedf( - "rename_database: expected column id %d in table id %d to have a default expr", - dependedOn.ID, - dependentDesc.GetID(), + if err != nil { + log.Warningf(ctx, "unable to retrieve fully-qualified name of %s (id: %d): %v", tbTableName.String(), depDesc.GetID(), err) + return "", sqlerrors.NewDependentObjectErrorf( + "cannot rename database because a relation depends on relation %q", + tbTableName.String(), ) } - // Try parse the default expression and find the table name direct reference. - parsedExpr, err := parser.ParseExpr(column.GetDefaultExpr()) + + return descFQName.FQString(), nil + } + + switch t := depDesc.(type) { + case catalog.TableDescriptor: + depTblName := tree.MakeTableNameWithSchema(tree.Name(dbName), tree.Name(scName), tree.Name(t.GetName())) + return depTblName.String(), nil + case catalog.FunctionDescriptor: + depFnName := tree.MakeQualifiedFunctionName(dbName, scName, t.GetName()) + return depFnName.String(), nil + default: + return "", errors.AssertionFailedf("expected only function or table descriptor, but got %s", t.DescriptorType()) + } +} + +func maybeFailOnDependentDescInRename( + ctx context.Context, + p *planner, + dbDesc catalog.DatabaseDescriptor, + schema string, + lookupFlags tree.CommonLookupFlags, + renameDescType catalog.DescriptorType, +) error { + tbNames, _, err := p.Descriptors().GetObjectNamesAndIDs( + ctx, + p.txn, + dbDesc, + schema, + tree.DatabaseListFlags{ + CommonLookupFlags: lookupFlags, + ExplicitPrefix: true, + }, + ) + if err != nil { + return err + } + lookupFlags.Required = false + // TODO(ajwerner): Make this do something better than one-at-a-time lookups + // followed by catalogkv reads on each dependency. + for i := range tbNames { + found, tbDesc, err := p.Descriptors().GetImmutableTableByName( + ctx, p.txn, &tbNames[i], tree.ObjectLookupFlags{CommonLookupFlags: lookupFlags}, + ) if err != nil { - return false, "", err + return err } - typedExpr, err := tree.TypeCheck(ctx, parsedExpr, nil, column.GetType()) - if err != nil { - return false, "", err + if !found { + continue } - seqIdentifiers, err := seqexpr.GetUsedSequences(typedExpr) - if err != nil { - return false, "", err + + // Since we now only reference sequences with IDs, it's always safe to + // rename a db or schema containing the sequence. There is one exception + // being tracked with issue #87509. + if tbDesc.IsSequence() { + continue } - for _, seqIdentifier := range seqIdentifiers { - if seqIdentifier.IsByID() { - continue - } - parsedSeqName, err := parser.ParseTableName(seqIdentifier.SeqName) + + if err := tbDesc.ForeachDependedOnBy(func(dependedOn *descpb.TableDescriptor_Reference) error { + dependentDesc, err := p.Descriptors().GetMutableDescriptorByID(ctx, p.txn, dependedOn.ID) if err != nil { - return false, "", err + return err } - // There must be at least two parts for this to work. - if parsedSeqName.NumParts >= 2 { - // We only don't allow this if the database name is in there. - // This is always the last argument. - if tree.Name(parsedSeqName.Parts[parsedSeqName.NumParts-1]).Normalize() == tree.Name(dbName).Normalize() { - return false, column.GetName(), nil - } + + tbTableName := tree.MakeTableNameWithSchema( + tree.Name(dbDesc.GetName()), + tree.Name(schema), + tree.Name(tbDesc.GetName()), + ) + dependentDescQualifiedString, err := getQualifiedDependentObjectName( + ctx, p, dbDesc.GetName(), schema, tbDesc, dependentDesc, + ) + if err != nil { + return err } + depErr := sqlerrors.NewDependentObjectErrorf( + "cannot rename %s because relation %q depends on relation %q", + renameDescType, + dependentDescQualifiedString, + tbTableName.String(), + ) + + // Otherwise, we default to the view error message. + return errors.WithHintf(depErr, + "you can drop %q instead", dependentDescQualifiedString) + }); err != nil { + return err } } - if colIDs.Len() > 0 { - return false, "", errors.AssertionFailedf( - "expected to find column ids %s in table id %d", - colIDs.String(), - dependentDesc.GetID(), - ) - } - return true, "", nil + + return nil } func (n *renameDatabaseNode) Next(runParams) (bool, error) { return false, nil } diff --git a/pkg/sql/schema_resolver.go b/pkg/sql/schema_resolver.go index e70d5c326b9d..7964ecb1f2e2 100644 --- a/pkg/sql/schema_resolver.go +++ b/pkg/sql/schema_resolver.go @@ -224,6 +224,28 @@ func (sr *schemaResolver) getQualifiedTableName( return &tbName, nil } +func (sr *schemaResolver) getQualifiedFunctionName( + ctx context.Context, fnDesc catalog.FunctionDescriptor, +) (*tree.FunctionName, error) { + lookupFlags := tree.CommonLookupFlags{ + Required: true, + IncludeOffline: true, + IncludeDropped: true, + AvoidLeased: true, + } + _, dbDesc, err := sr.descCollection.GetImmutableDatabaseByID(ctx, sr.txn, fnDesc.GetParentID(), lookupFlags) + if err != nil { + return nil, err + } + scDesc, err := sr.descCollection.GetImmutableSchemaByID(ctx, sr.txn, fnDesc.GetParentSchemaID(), lookupFlags) + if err != nil { + return nil, err + } + + fnName := tree.MakeQualifiedFunctionName(dbDesc.GetName(), scDesc.GetName(), fnDesc.GetName()) + return &fnName, nil +} + // ResolveType implements the tree.TypeReferenceResolver interface. func (sr *schemaResolver) ResolveType( ctx context.Context, name *tree.UnresolvedObjectName, diff --git a/pkg/sql/schemachanger/rel/schema_rules.go b/pkg/sql/schemachanger/rel/schema_rules.go index 0fb70a0764e8..cd2edb1b96de 100644 --- a/pkg/sql/schemachanger/rel/schema_rules.go +++ b/pkg/sql/schemachanger/rel/schema_rules.go @@ -80,21 +80,26 @@ func (sc *Schema) DefNotJoin1(name string, a Var, def func(a Var) Clauses) Rule1 return sc.rule(name, notJoin, def, a).(Rule1) } -// DefNotJoin2 defines a not-join rule with two bound variable arguments. -func (sc *Schema) DefNotJoin2(name string, a, b Var, def func(a, b Var) Clauses) Rule2 { - return sc.rule(name, notJoin, def, a, b).(Rule2) -} - // Def2 defines a Rule2. func (sc *Schema) Def2(name string, a, b Var, def func(a, b Var) Clauses) Rule2 { return sc.rule(name, regular, def, a, b).(Rule2) } +// DefNotJoin2 defines a not-join rule with two bound variable arguments. +func (sc *Schema) DefNotJoin2(name string, a, b Var, def func(a, b Var) Clauses) Rule2 { + return sc.rule(name, notJoin, def, a, b).(Rule2) +} + // Def3 defines a Rule3. func (sc *Schema) Def3(name string, a, b, c Var, def func(a, b, c Var) Clauses) Rule3 { return sc.rule(name, regular, def, a, b, c).(Rule3) } +// DefNotJoin3 defines a not-join rule with three bound variable arguments. +func (sc *Schema) DefNotJoin3(name string, a, b, c Var, def func(a, b, c Var) Clauses) Rule3 { + return sc.rule(name, notJoin, def, a, b, c).(Rule3) +} + // Def4 defines a Rule4. func (sc *Schema) Def4(name string, a, b, c, d Var, def func(a, b, c, d Var) Clauses) Rule4 { return sc.rule(name, regular, def, a, b, c, d).(Rule4) diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/register.go b/pkg/sql/schemachanger/scplan/internal/opgen/register.go index ea7390354d74..0bd2e35a2bab 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/register.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/register.go @@ -230,10 +230,10 @@ func validateTargets(targets []target) error { } for s := range allStatuses { - if !absentStatuses[s] { + if nonAbsentStatuses[s] && !absentStatuses[s] { return errors.Errorf("status %s is featured in non-ABSENT targets but not in the ABSENT target", s) } - if !nonAbsentStatuses[s] { + if absentStatuses[s] && !nonAbsentStatuses[s] { return errors.Errorf("status %s is featured in ABSENT target but not in any non-ABSENT targets", s) } } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go index c82b13b76f5a..5f3ad2ba886f 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go @@ -67,6 +67,7 @@ func init() { to.typeFilter(isSimpleDependent), joinOnDescID(from, to, "desc-id"), statusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL(from.target, from.el, to.el), } }) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 7bb690c66237..84c51b1bd7e9 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -486,3 +486,30 @@ var descriptorIsNotBeingDropped = screl.Schema.DefNotJoin1( } }, ) + +// fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL creates +// a clause which leads to the outer clause failing to unify +// if the passed element `from` is a Table, `to` is a RowLevelTTl, +// and there does not exist a node with the same target as +// `fromTarget` in PUBLIC status. +// It is used to suppress rule "descriptor drop right before dependent element removal" +// for the special case where we drop a rowLevelTTL table in mixed +// version state for forward compatibility (issue #86672). +var fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL = screl.Schema.DefNotJoin3( + "fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL", + "fromTarget", "fromEl", "toEl", func(fromTarget, fromEl, toEl rel.Var) rel.Clauses { + n := rel.Var("n") + return rel.Clauses{ + fromEl.Type((*scpb.Table)(nil)), + toEl.Type((*scpb.RowLevelTTL)(nil)), + n.Type((*screl.Node)(nil)), + n.AttrEqVar(screl.Target, fromTarget), + screl.Schema.DefNotJoin1("nodeHasNoPublicStatus", "n", func(n rel.Var) rel.Clauses { + public := rel.Var("public") + return rel.Clauses{ + public.Eq(scpb.Status_PUBLIC), + n.AttrEqVar(screl.CurrentStatus, public), + } + })(n), + } + }) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index fe2e83a2e4bd..7658f3904e8f 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -15,6 +15,13 @@ descriptorIsNotBeingDropped($element): - joinTarget($descriptor, $descriptor-target) - joinOnDescID($descriptor, $element, $id) - $descriptor-target[TargetStatus] = ABSENT +fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): + not-join: + - $fromEl[Type] = '*scpb.Table' + - $toEl[Type] = '*scpb.RowLevelTTL' + - $n[Type] = '*screl.Node' + - $n[Target] = $fromTarget + - nodeHasNoPublicStatus($n) joinOnColumnID($a, $b, $desc-id, $col-id): - joinOnDescID($a, $b, $desc-id) - $a[ColumnID] = $col-id @@ -41,6 +48,10 @@ joinTargetNode($element, $target, $node): - joinTarget($element, $target) - $node[Type] = '*screl.Node' - $node[Target] = $target +nodeHasNoPublicStatus($n): + not-join: + - $public = PUBLIC + - $n[CurrentStatus] = $public nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($sharedTarget): not-join: - $n[Type] = '*screl.Node' @@ -1664,6 +1675,7 @@ deprules - toAbsent($descriptor-target, $dependent-target) - $descriptor-node[CurrentStatus] = DROPPED - $dependent-node[CurrentStatus] = ABSENT + - fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($descriptor-target, $descriptor, $dependent) - joinTargetNode($descriptor, $descriptor-target, $descriptor-node) - joinTargetNode($dependent, $dependent-target, $dependent-node) - name: descriptor drop right before removing dependent with attr ref diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules index 9131c112a2e2..340e78d09504 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/oprules @@ -15,6 +15,13 @@ descriptorIsNotBeingDropped($element): - joinTarget($descriptor, $descriptor-target) - joinOnDescID($descriptor, $element, $id) - $descriptor-target[TargetStatus] = ABSENT +fromHasPublicStatusIfFromIsTableAndToIsRowLevelTTL($fromTarget, $fromEl, $toEl): + not-join: + - $fromEl[Type] = '*scpb.Table' + - $toEl[Type] = '*scpb.RowLevelTTL' + - $n[Type] = '*screl.Node' + - $n[Target] = $fromTarget + - nodeHasNoPublicStatus($n) joinOnColumnID($a, $b, $desc-id, $col-id): - joinOnDescID($a, $b, $desc-id) - $a[ColumnID] = $col-id @@ -41,6 +48,10 @@ joinTargetNode($element, $target, $node): - joinTarget($element, $target) - $node[Type] = '*screl.Node' - $node[Target] = $target +nodeHasNoPublicStatus($n): + not-join: + - $public = PUBLIC + - $n[CurrentStatus] = $public nodeNotExistsWithStatusIn_BACKFILLED_BACKFILL_ONLY($sharedTarget): not-join: - $n[Type] = '*screl.Node' diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go index 1bc4c7b976e4..e53ea8c7e521 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/redact" ) -// Graph is a graph whose nodes are *scpb.Nodes. Graphs are constructed during +// Graph is a graph whose nodes are *screl.Nodes. Graphs are constructed during // schema change planning. Edges in the graph represent dependencies between // nodes, either due to the sequencing of statuses for a single target or due to // inter-target dependencies between statuses. diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 813a7a48dadc..2c19204b3464 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "aggregate_builtins.go", "all_builtins.go", "builtins.go", + "fixed_oids.go", "generator_builtins.go", "generator_probe_ranges.go", "geo_builtins.go", @@ -58,7 +59,6 @@ go_library( "//pkg/sql/lex", "//pkg/sql/lexbase", "//pkg/sql/memsize", - "//pkg/sql/oidext", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index c45218bd6a93..f80d91a7b303 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -37,7 +37,7 @@ import ( "github.com/twpayne/go-geom" ) -func initAggregateBuiltins() { +func init() { // Add all aggregates to the builtins map after a few sanity checks. for k, v := range aggregates { diff --git a/pkg/sql/sem/builtins/all_builtins.go b/pkg/sql/sem/builtins/all_builtins.go index d44256d23f58..8bff6e4d2bed 100644 --- a/pkg/sql/sem/builtins/all_builtins.go +++ b/pkg/sql/sem/builtins/all_builtins.go @@ -15,48 +15,50 @@ import ( "sort" "strings" - "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) -// AllBuiltinNames is an array containing all the built-in function +var allBuiltinNames orderedStrings + +// AllBuiltinNames returns a slice containing all the built-in function // names, sorted in alphabetical order. This can be used for a // deterministic walk through the Builtins map. -var AllBuiltinNames []string +func AllBuiltinNames() []string { + allBuiltinNames.sort() + return allBuiltinNames.strings +} + +var allAggregateBuiltinNames orderedStrings -// AllAggregateBuiltinNames is an array containing the subset of +// AllAggregateBuiltinNames returns a slice containing the subset of // AllBuiltinNames that corresponds to aggregate functions. -var AllAggregateBuiltinNames []string +func AllAggregateBuiltinNames() []string { + allAggregateBuiltinNames.sort() + return allAggregateBuiltinNames.strings +} + +var allWindowBuiltinNames orderedStrings -// AllWindowBuiltinNames is an array containing the subset of +// AllWindowBuiltinNames returns a slice containing the subset of // AllBuiltinNames that corresponds to window functions. -var AllWindowBuiltinNames []string +func AllWindowBuiltinNames() []string { + allWindowBuiltinNames.sort() + return allWindowBuiltinNames.strings +} func init() { - // Note: changing the order of these init functions will cause changes to OIDs - // of builtin functions. In general, it won't cause internal problems other - // than causing failures in tests which make assumption of OIDs. - initRegularBuiltins() - initAggregateBuiltins() - initWindowBuiltins() - initGeneratorBuiltins() - initGeoBuiltins() - initTrigramBuiltins() - initPGBuiltins() - initMathBuiltins() - initOverlapsBuiltins() - initReplicationBuiltins() - initPgcryptoBuiltins() - initProbeRangesBuiltins() - tree.FunDefs = make(map[string]*tree.FunctionDefinition) tree.ResolvedBuiltinFuncDefs = make(map[string]*tree.ResolvedFunctionDefinition) - builtinsregistry.Iterate(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { + + builtinsregistry.AddSubscription(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { + for i, fn := range overloads { + signature := name + fn.Signature(true) + overloads[i].Oid = signatureMustHaveHardcodedOID(signature) + } fDef := tree.NewFunctionDefinition(name, props, overloads) addResolvedFuncDef(tree.ResolvedBuiltinFuncDefs, fDef) tree.FunDefs[name] = fDef @@ -64,17 +66,13 @@ func init() { // Avoid listing help for undocumented functions. return } - AllBuiltinNames = append(AllBuiltinNames, name) + allBuiltinNames.add(name) if props.Class == tree.AggregateClass { - AllAggregateBuiltinNames = append(AllAggregateBuiltinNames, name) + allAggregateBuiltinNames.add(name) } else if props.Class == tree.WindowClass { - AllWindowBuiltinNames = append(AllWindowBuiltinNames, name) + allWindowBuiltinNames.add(name) } }) - - sort.Strings(AllBuiltinNames) - sort.Strings(AllAggregateBuiltinNames) - sort.Strings(AllWindowBuiltinNames) } func addResolvedFuncDef( @@ -100,7 +98,7 @@ func addResolvedFuncDef( } func registerBuiltin(name string, def builtinDefinition) { - for i, overload := range def.overloads { + for _, overload := range def.overloads { fnCount := 0 if overload.Fn != nil { fnCount++ @@ -125,10 +123,6 @@ func registerBuiltin(name string, def builtinDefinition) { name, fnCount, )) } - c := sqltelemetry.BuiltinCounter(name, overload.Signature(false)) - def.overloads[i].OnTypeCheck = func() { - telemetry.Inc(c) - } } if def.props.ShouldDocument() && def.props.Category == "" { def.props.Category = getCategory(def.overloads) @@ -164,3 +158,36 @@ func collectOverloads( } return makeBuiltin(props, r...) } + +// orderedStrings sorts a slice of strings lazily +// for better performance. +type orderedStrings struct { + strings []string + sorted bool +} + +// add a string without changing whether or not +// the strings are sorted yet. +func (o *orderedStrings) add(s string) { + if o.sorted { + o.insert(s) + } else { + o.strings = append(o.strings, s) + } +} + +func (o *orderedStrings) sort() { + if !o.sorted { + sort.Strings(o.strings) + } + o.sorted = true +} + +// insert assumes the strings are already sorted +// and inserts s in the right place. +func (o *orderedStrings) insert(s string) { + i := sort.SearchStrings(o.strings, s) + o.strings = append(o.strings, "") + copy(o.strings[i+1:], o.strings[i:]) + o.strings[i] = s +} diff --git a/pkg/sql/sem/builtins/all_builtins_test.go b/pkg/sql/sem/builtins/all_builtins_test.go index ffad05e13c09..c40e12481cbd 100644 --- a/pkg/sql/sem/builtins/all_builtins_test.go +++ b/pkg/sql/sem/builtins/all_builtins_test.go @@ -30,7 +30,7 @@ import ( func TestOverloadsHaveVolatility(t *testing.T) { defer leaktest.AfterTest(t)() - builtinsregistry.Iterate(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { + builtinsregistry.AddSubscription(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { for idx, overload := range overloads { assert.NotEqual( t, @@ -144,7 +144,7 @@ func TestOverloadsVolatilityMatchesPostgres(t *testing.T) { } // Check each builtin against Postgres. - builtinsregistry.Iterate(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { + builtinsregistry.AddSubscription(func(name string, props *tree.FunctionProperties, overloads []tree.Overload) { for idx, overload := range overloads { if overload.IgnoreVolatilityCheck { continue diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 49740927f591..97103b02ef85 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -53,7 +53,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" - "github.com/cockroachdb/cockroach/pkg/sql/oidext" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -96,7 +95,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/knz/strtime" - "github.com/lib/pq/oid" "golang.org/x/text/cases" "golang.org/x/text/language" ) @@ -161,28 +159,7 @@ func arrayProps() tree.FunctionProperties { return tree.FunctionProperties{Category: builtinconstants.CategoryArray} } -// builtinFuncOIDCnt is used as a counter to assign OIDs to builtin function -// overloads. OIDs are assigned in the order of overloads being initialized by -// go. That is, OIDs are given by alphabetical order of the file name a function -// is declared and the order of tree.Overload is declared in that file. So -// changing file names or function declaration position will change OIDs. -var builtinFuncOIDCnt = 0 - -// Please always use this function to creat builtinDefinition instead of -// construct it directly. Otherwise, the new builtin function will not have an -// OID. func makeBuiltin(props tree.FunctionProperties, overloads ...tree.Overload) builtinDefinition { - for i := range overloads { - builtinFuncOIDCnt++ - if builtinFuncOIDCnt > oidext.CockroachPredefinedOIDMax { - panic( - errors.AssertionFailedf( - "builtin function oid exceeds maximum predefined oid %d", oidext.CockroachPredefinedOIDMax, - ), - ) - } - overloads[i].Oid = oid.Oid(builtinFuncOIDCnt) - } return builtinDefinition{ props: props, overloads: overloads, @@ -207,7 +184,7 @@ func mustBeDIntInTenantRange(e tree.Expr) (tree.DInt, error) { return tenID, nil } -func initRegularBuiltins() { +func init() { for k, v := range regularBuiltins { registerBuiltin(k, v) } diff --git a/pkg/sql/sem/builtins/builtinsregistry/builtins_registry.go b/pkg/sql/sem/builtins/builtinsregistry/builtins_registry.go index 5eef0bd1b6e6..9fe0dbfe16e6 100644 --- a/pkg/sql/sem/builtins/builtinsregistry/builtins_registry.go +++ b/pkg/sql/sem/builtins/builtinsregistry/builtins_registry.go @@ -20,16 +20,40 @@ import "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" var registry = map[string]definition{} +// Subscription is a hook to be called once on all registered builtins. +type Subscription func(name string, props *tree.FunctionProperties, overloads []tree.Overload) + +var subscriptions []Subscription + // Register registers a builtin. Intending to be called at init time, it panics // if a function of the same name has already been registered. func Register(name string, props *tree.FunctionProperties, overloads []tree.Overload) { if _, exists := registry[name]; exists { panic("duplicate builtin: " + name) } + for i := range overloads { + var hook func() + overloads[i].OnTypeCheck = &hook + } registry[name] = definition{ props: props, overloads: overloads, } + for _, s := range subscriptions { + s(name, props, overloads) + } +} + +// AddSubscription defines a hook to be called once on all registered builtins. +// Subscriptions should be added at init() time, but are load-order independent: +// if you add a subscription after a function is registered, it will immediately +// be called on that function, while functions that are registered afterwards will +// also trigger the hook. +func AddSubscription(s Subscription) { + for name, def := range registry { + s(name, def.props, def.overloads) + } + subscriptions = append(subscriptions, s) } // GetBuiltinProperties provides low-level access to a built-in function's properties. @@ -43,13 +67,6 @@ func GetBuiltinProperties(name string) (*tree.FunctionProperties, []tree.Overloa return def.props, def.overloads } -// Iterate iterates the previously registered functions. -func Iterate(f func(name string, props *tree.FunctionProperties, overloads []tree.Overload)) { - for k, v := range registry { - f(k, v.props, v.overloads) - } -} - type definition struct { props *tree.FunctionProperties overloads []tree.Overload diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go new file mode 100644 index 000000000000..84c7a2bac85d --- /dev/null +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -0,0 +1,2051 @@ +// 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 builtins + +import ( + "fmt" + + "github.com/cockroachdb/errors" + "github.com/lib/pq/oid" +) + +// builtinOidsBySignature ensures a stable mapping of function overloads to oids. +// The oid for a given signature must not change. The signature for a given oid +// should not change except cosmetically (add a new overload instead). +// Keys in this map are function name concatenated to overload.Signature(true). +var builtinOidsBySignature = map[string]oid.Oid{ + `_st_contains(geometry_a: geometry, geometry_b: geometry) -> bool`: 1765, + `_st_containsproperly(geometry_a: geometry, geometry_b: geometry) -> bool`: 1753, + `_st_coveredby(geometry_a: geometry, geometry_b: geometry) -> bool`: 1763, + `_st_coveredby(geography_a: geography, geography_b: geography) -> bool`: 1764, + `_st_covers(geometry_a: geometry, geometry_b: geometry) -> bool`: 1771, + `_st_covers(geography_a: geography, geography_b: geography) -> bool`: 1772, + `_st_crosses(geometry_a: geometry, geometry_b: geometry) -> bool`: 1766, + `_st_dfullywithin(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1762, + `_st_dfullywithinexclusive(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1761, + `_st_dwithin(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1756, + `_st_dwithin(geography_a: geography, geography_b: geography, distance: float) -> bool`: 1757, + `_st_dwithin(geography_a: geography, geography_b: geography, distance: float, use_spheroid: bool) -> bool`: 1758, + `_st_dwithinexclusive(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1768, + `_st_dwithinexclusive(geography_a: geography, geography_b: geography, distance: float) -> bool`: 1769, + `_st_dwithinexclusive(geography_a: geography, geography_b: geography, distance: float, use_spheroid: bool) -> bool`: 1770, + `_st_equals(geometry_a: geometry, geometry_b: geometry) -> bool`: 1754, + `_st_intersects(geometry_a: geometry, geometry_b: geometry) -> bool`: 1759, + `_st_intersects(geography_a: geography, geography_b: geography) -> bool`: 1760, + `_st_overlaps(geometry_a: geometry, geometry_b: geometry) -> bool`: 1767, + `_st_touches(geometry_a: geometry, geometry_b: geometry) -> bool`: 1752, + `_st_within(geometry_a: geometry, geometry_b: geometry) -> bool`: 1755, + `abbrev(val: inet) -> string`: 873, + `abs(val: float) -> float`: 738, + `abs(val: decimal) -> decimal`: 739, + `abs(val: int) -> int`: 740, + `aclexplode(aclitems: string[]) -> tuple{oid AS grantor, oid AS grantee, string AS privilege_type, bool AS is_grantable}`: 314, + `acos(val: float) -> float`: 741, + `acosd(val: float) -> float`: 742, + `acosh(val: float) -> float`: 743, + `addgeometrycolumn(table_name: string, column_name: string, srid: int, type: string, dimension: int, use_typmod: bool) -> string`: 711, + `addgeometrycolumn(schema_name: string, table_name: string, column_name: string, srid: int, type: string, dimension: int, use_typmod: bool) -> string`: 712, + `addgeometrycolumn(catalog_name: string, schema_name: string, table_name: string, column_name: string, srid: int, type: string, dimension: int, use_typmod: bool) -> string`: 713, + `addgeometrycolumn(table_name: string, column_name: string, srid: int, type: string, dimension: int) -> string`: 714, + `addgeometrycolumn(schema_name: string, table_name: string, column_name: string, srid: int, type: string, dimension: int) -> string`: 715, + `addgeometrycolumn(catalog_name: string, schema_name: string, table_name: string, column_name: string, srid: int, type: string, dimension: int) -> string`: 716, + `age(val: timestamptz) -> interval`: 998, + `age(end: timestamptz, begin: timestamptz) -> interval`: 999, + `any_not_null(arg1: anyelement) -> anyelement`: 260, + `anyarray_in(input: anyelement) -> anyelement[]`: 2017, + `anyarray_out(anyelement[]: anyelement[]) -> bytes`: 2016, + `anyarray_recv(input: anyelement) -> anyelement[]`: 2015, + `anyarray_send(anyelement[]: anyelement[]) -> bytes`: 2014, + `anyelement_in(input: anyelement) -> anyelement`: 1857, + `anyelement_out(anyelement: anyelement) -> bytes`: 1856, + `anyelement_recv(input: anyelement) -> anyelement`: 1855, + `anyelement_send(anyelement: anyelement) -> bytes`: 1854, + `array_agg(arg1: bool) -> bool[]`: 1, + `array_agg(arg1: box2d) -> box2d[]`: 2, + `array_agg(arg1: int) -> int[]`: 3, + `array_agg(arg1: float) -> float[]`: 4, + `array_agg(arg1: decimal) -> decimal[]`: 5, + `array_agg(arg1: date) -> date[]`: 6, + `array_agg(arg1: timestamp) -> timestamp[]`: 7, + `array_agg(arg1: interval) -> interval[]`: 8, + `array_agg(arg1: geography) -> geography[]`: 9, + `array_agg(arg1: geometry) -> geometry[]`: 10, + `array_agg(arg1: string) -> string[]`: 11, + `array_agg(arg1: bytes) -> bytes[]`: 12, + `array_agg(arg1: timestamptz) -> timestamptz[]`: 13, + `array_agg(arg1: oid) -> oid[]`: 14, + `array_agg(arg1: uuid) -> uuid[]`: 15, + `array_agg(arg1: inet) -> inet[]`: 16, + `array_agg(arg1: time) -> time[]`: 17, + `array_agg(arg1: timetz) -> timetz[]`: 18, + `array_agg(arg1: jsonb) -> jsonb[]`: 19, + `array_agg(arg1: varbit) -> varbit[]`: 20, + `array_agg(arg1: anyenum) -> anyenum[]`: 21, + `array_agg(arg1: tuple) -> tuple[]`: 22, + `array_append(array: bool[], elem: bool) -> bool[]`: 1082, + `array_append(array: box2d[], elem: box2d) -> box2d[]`: 1083, + `array_append(array: int[], elem: int) -> int[]`: 1084, + `array_append(array: float[], elem: float) -> float[]`: 1085, + `array_append(array: decimal[], elem: decimal) -> decimal[]`: 1086, + `array_append(array: date[], elem: date) -> date[]`: 1087, + `array_append(array: timestamp[], elem: timestamp) -> timestamp[]`: 1088, + `array_append(array: interval[], elem: interval) -> interval[]`: 1089, + `array_append(array: geography[], elem: geography) -> geography[]`: 1090, + `array_append(array: geometry[], elem: geometry) -> geometry[]`: 1091, + `array_append(array: string[], elem: string) -> string[]`: 1092, + `array_append(array: bytes[], elem: bytes) -> bytes[]`: 1093, + `array_append(array: timestamptz[], elem: timestamptz) -> timestamptz[]`: 1094, + `array_append(array: oid[], elem: oid) -> oid[]`: 1095, + `array_append(array: uuid[], elem: uuid) -> uuid[]`: 1096, + `array_append(array: inet[], elem: inet) -> inet[]`: 1097, + `array_append(array: time[], elem: time) -> time[]`: 1098, + `array_append(array: timetz[], elem: timetz) -> timetz[]`: 1099, + `array_append(array: jsonb[], elem: jsonb) -> jsonb[]`: 1100, + `array_append(array: varbit[], elem: varbit) -> varbit[]`: 1101, + `array_append(array: anyenum[], elem: anyenum) -> anyenum[]`: 1102, + `array_append(array: tuple[], elem: tuple) -> tuple[]`: 1103, + `array_cat(left: bool[], right: bool[]) -> bool[]`: 1126, + `array_cat(left: box2d[], right: box2d[]) -> box2d[]`: 1127, + `array_cat(left: int[], right: int[]) -> int[]`: 1128, + `array_cat(left: float[], right: float[]) -> float[]`: 1129, + `array_cat(left: decimal[], right: decimal[]) -> decimal[]`: 1130, + `array_cat(left: date[], right: date[]) -> date[]`: 1131, + `array_cat(left: timestamp[], right: timestamp[]) -> timestamp[]`: 1132, + `array_cat(left: interval[], right: interval[]) -> interval[]`: 1133, + `array_cat(left: geography[], right: geography[]) -> geography[]`: 1134, + `array_cat(left: geometry[], right: geometry[]) -> geometry[]`: 1135, + `array_cat(left: string[], right: string[]) -> string[]`: 1136, + `array_cat(left: bytes[], right: bytes[]) -> bytes[]`: 1137, + `array_cat(left: timestamptz[], right: timestamptz[]) -> timestamptz[]`: 1138, + `array_cat(left: oid[], right: oid[]) -> oid[]`: 1139, + `array_cat(left: uuid[], right: uuid[]) -> uuid[]`: 1140, + `array_cat(left: inet[], right: inet[]) -> inet[]`: 1141, + `array_cat(left: time[], right: time[]) -> time[]`: 1142, + `array_cat(left: timetz[], right: timetz[]) -> timetz[]`: 1143, + `array_cat(left: jsonb[], right: jsonb[]) -> jsonb[]`: 1144, + `array_cat(left: varbit[], right: varbit[]) -> varbit[]`: 1145, + `array_cat(left: anyenum[], right: anyenum[]) -> anyenum[]`: 1146, + `array_cat(left: tuple[], right: tuple[]) -> tuple[]`: 1147, + `array_in(input: anyelement) -> anyelement[]`: 2013, + `array_length(input: anyelement[], array_dimension: int) -> int`: 1078, + `array_lower(input: anyelement[], array_dimension: int) -> int`: 1080, + `array_out(anyelement[]: anyelement[]) -> bytes`: 2012, + `array_position(array: bool[], elem: bool) -> int`: 1192, + `array_position(array: box2d[], elem: box2d) -> int`: 1193, + `array_position(array: int[], elem: int) -> int`: 1194, + `array_position(array: float[], elem: float) -> int`: 1195, + `array_position(array: decimal[], elem: decimal) -> int`: 1196, + `array_position(array: date[], elem: date) -> int`: 1197, + `array_position(array: timestamp[], elem: timestamp) -> int`: 1198, + `array_position(array: interval[], elem: interval) -> int`: 1199, + `array_position(array: geography[], elem: geography) -> int`: 1200, + `array_position(array: geometry[], elem: geometry) -> int`: 1201, + `array_position(array: string[], elem: string) -> int`: 1202, + `array_position(array: bytes[], elem: bytes) -> int`: 1203, + `array_position(array: timestamptz[], elem: timestamptz) -> int`: 1204, + `array_position(array: oid[], elem: oid) -> int`: 1205, + `array_position(array: uuid[], elem: uuid) -> int`: 1206, + `array_position(array: inet[], elem: inet) -> int`: 1207, + `array_position(array: time[], elem: time) -> int`: 1208, + `array_position(array: timetz[], elem: timetz) -> int`: 1209, + `array_position(array: jsonb[], elem: jsonb) -> int`: 1210, + `array_position(array: varbit[], elem: varbit) -> int`: 1211, + `array_position(array: anyenum[], elem: anyenum) -> int`: 1212, + `array_position(array: tuple[], elem: tuple) -> int`: 1213, + `array_positions(array: bool[], elem: bool) -> int[]`: 1214, + `array_positions(array: box2d[], elem: box2d) -> int[]`: 1215, + `array_positions(array: int[], elem: int) -> int[]`: 1216, + `array_positions(array: float[], elem: float) -> int[]`: 1217, + `array_positions(array: decimal[], elem: decimal) -> int[]`: 1218, + `array_positions(array: date[], elem: date) -> int[]`: 1219, + `array_positions(array: timestamp[], elem: timestamp) -> int[]`: 1220, + `array_positions(array: interval[], elem: interval) -> int[]`: 1221, + `array_positions(array: geography[], elem: geography) -> int[]`: 1222, + `array_positions(array: geometry[], elem: geometry) -> int[]`: 1223, + `array_positions(array: string[], elem: string) -> int[]`: 1224, + `array_positions(array: bytes[], elem: bytes) -> int[]`: 1225, + `array_positions(array: timestamptz[], elem: timestamptz) -> int[]`: 1226, + `array_positions(array: oid[], elem: oid) -> int[]`: 1227, + `array_positions(array: uuid[], elem: uuid) -> int[]`: 1228, + `array_positions(array: inet[], elem: inet) -> int[]`: 1229, + `array_positions(array: time[], elem: time) -> int[]`: 1230, + `array_positions(array: timetz[], elem: timetz) -> int[]`: 1231, + `array_positions(array: jsonb[], elem: jsonb) -> int[]`: 1232, + `array_positions(array: varbit[], elem: varbit) -> int[]`: 1233, + `array_positions(array: anyenum[], elem: anyenum) -> int[]`: 1234, + `array_positions(array: tuple[], elem: tuple) -> int[]`: 1235, + `array_prepend(elem: bool, array: bool[]) -> bool[]`: 1104, + `array_prepend(elem: box2d, array: box2d[]) -> box2d[]`: 1105, + `array_prepend(elem: int, array: int[]) -> int[]`: 1106, + `array_prepend(elem: float, array: float[]) -> float[]`: 1107, + `array_prepend(elem: decimal, array: decimal[]) -> decimal[]`: 1108, + `array_prepend(elem: date, array: date[]) -> date[]`: 1109, + `array_prepend(elem: timestamp, array: timestamp[]) -> timestamp[]`: 1110, + `array_prepend(elem: interval, array: interval[]) -> interval[]`: 1111, + `array_prepend(elem: geography, array: geography[]) -> geography[]`: 1112, + `array_prepend(elem: geometry, array: geometry[]) -> geometry[]`: 1113, + `array_prepend(elem: string, array: string[]) -> string[]`: 1114, + `array_prepend(elem: bytes, array: bytes[]) -> bytes[]`: 1115, + `array_prepend(elem: timestamptz, array: timestamptz[]) -> timestamptz[]`: 1116, + `array_prepend(elem: oid, array: oid[]) -> oid[]`: 1117, + `array_prepend(elem: uuid, array: uuid[]) -> uuid[]`: 1118, + `array_prepend(elem: inet, array: inet[]) -> inet[]`: 1119, + `array_prepend(elem: time, array: time[]) -> time[]`: 1120, + `array_prepend(elem: timetz, array: timetz[]) -> timetz[]`: 1121, + `array_prepend(elem: jsonb, array: jsonb[]) -> jsonb[]`: 1122, + `array_prepend(elem: varbit, array: varbit[]) -> varbit[]`: 1123, + `array_prepend(elem: anyenum, array: anyenum[]) -> anyenum[]`: 1124, + `array_prepend(elem: tuple, array: tuple[]) -> tuple[]`: 1125, + `array_recv(input: anyelement) -> anyelement[]`: 2011, + `array_remove(array: bool[], elem: bool) -> anyelement`: 1148, + `array_remove(array: box2d[], elem: box2d) -> anyelement`: 1149, + `array_remove(array: int[], elem: int) -> anyelement`: 1150, + `array_remove(array: float[], elem: float) -> anyelement`: 1151, + `array_remove(array: decimal[], elem: decimal) -> anyelement`: 1152, + `array_remove(array: date[], elem: date) -> anyelement`: 1153, + `array_remove(array: timestamp[], elem: timestamp) -> anyelement`: 1154, + `array_remove(array: interval[], elem: interval) -> anyelement`: 1155, + `array_remove(array: geography[], elem: geography) -> anyelement`: 1156, + `array_remove(array: geometry[], elem: geometry) -> anyelement`: 1157, + `array_remove(array: string[], elem: string) -> anyelement`: 1158, + `array_remove(array: bytes[], elem: bytes) -> anyelement`: 1159, + `array_remove(array: timestamptz[], elem: timestamptz) -> anyelement`: 1160, + `array_remove(array: oid[], elem: oid) -> anyelement`: 1161, + `array_remove(array: uuid[], elem: uuid) -> anyelement`: 1162, + `array_remove(array: inet[], elem: inet) -> anyelement`: 1163, + `array_remove(array: time[], elem: time) -> anyelement`: 1164, + `array_remove(array: timetz[], elem: timetz) -> anyelement`: 1165, + `array_remove(array: jsonb[], elem: jsonb) -> anyelement`: 1166, + `array_remove(array: varbit[], elem: varbit) -> anyelement`: 1167, + `array_remove(array: anyenum[], elem: anyenum) -> anyelement`: 1168, + `array_remove(array: tuple[], elem: tuple) -> anyelement`: 1169, + `array_replace(array: bool[], toreplace: bool, replacewith: bool) -> anyelement`: 1170, + `array_replace(array: box2d[], toreplace: box2d, replacewith: box2d) -> anyelement`: 1171, + `array_replace(array: int[], toreplace: int, replacewith: int) -> anyelement`: 1172, + `array_replace(array: float[], toreplace: float, replacewith: float) -> anyelement`: 1173, + `array_replace(array: decimal[], toreplace: decimal, replacewith: decimal) -> anyelement`: 1174, + `array_replace(array: date[], toreplace: date, replacewith: date) -> anyelement`: 1175, + `array_replace(array: timestamp[], toreplace: timestamp, replacewith: timestamp) -> anyelement`: 1176, + `array_replace(array: interval[], toreplace: interval, replacewith: interval) -> anyelement`: 1177, + `array_replace(array: geography[], toreplace: geography, replacewith: geography) -> anyelement`: 1178, + `array_replace(array: geometry[], toreplace: geometry, replacewith: geometry) -> anyelement`: 1179, + `array_replace(array: string[], toreplace: string, replacewith: string) -> anyelement`: 1180, + `array_replace(array: bytes[], toreplace: bytes, replacewith: bytes) -> anyelement`: 1181, + `array_replace(array: timestamptz[], toreplace: timestamptz, replacewith: timestamptz) -> anyelement`: 1182, + `array_replace(array: oid[], toreplace: oid, replacewith: oid) -> anyelement`: 1183, + `array_replace(array: uuid[], toreplace: uuid, replacewith: uuid) -> anyelement`: 1184, + `array_replace(array: inet[], toreplace: inet, replacewith: inet) -> anyelement`: 1185, + `array_replace(array: time[], toreplace: time, replacewith: time) -> anyelement`: 1186, + `array_replace(array: timetz[], toreplace: timetz, replacewith: timetz) -> anyelement`: 1187, + `array_replace(array: jsonb[], toreplace: jsonb, replacewith: jsonb) -> anyelement`: 1188, + `array_replace(array: varbit[], toreplace: varbit, replacewith: varbit) -> anyelement`: 1189, + `array_replace(array: anyenum[], toreplace: anyenum, replacewith: anyenum) -> anyelement`: 1190, + `array_replace(array: tuple[], toreplace: tuple, replacewith: tuple) -> anyelement`: 1191, + `array_send(anyelement[]: anyelement[]) -> bytes`: 2010, + `array_to_json(array: anyelement[]) -> jsonb`: 312, + `array_to_json(array: anyelement[], pretty_bool: bool) -> jsonb`: 313, + `array_to_string(input: anyelement[], delim: string) -> string`: 1076, + `array_to_string(input: anyelement[], delimiter: string, null: string) -> string`: 1077, + `array_upper(input: anyelement[], array_dimension: int) -> int`: 1081, + `ascii(val: string) -> int`: 894, + `asin(val: float) -> float`: 744, + `asind(val: float) -> float`: 745, + `asinh(val: float) -> float`: 746, + `atan(val: float) -> float`: 747, + `atan2(x: float, y: float) -> float`: 750, + `atan2d(x: float, y: float) -> float`: 751, + `atand(val: float) -> float`: 748, + `atanh(val: float) -> float`: 749, + `avg(arg1: int) -> decimal`: 23, + `avg(arg1: float) -> float`: 24, + `avg(arg1: decimal) -> decimal`: 25, + `avg(arg1: interval) -> interval`: 26, + `bit_and(arg1: int) -> int`: 27, + `bit_and(arg1: varbit) -> varbit`: 28, + `bit_in(input: anyelement) -> bit`: 1945, + `bit_length(val: string) -> int`: 821, + `bit_length(val: bytes) -> int`: 822, + `bit_length(val: varbit) -> int`: 823, + `bit_or(arg1: int) -> int`: 29, + `bit_or(arg1: varbit) -> varbit`: 30, + `bit_out(bit: bit) -> bytes`: 1944, + `bit_recv(input: anyelement) -> bit`: 1943, + `bit_send(bit: bit) -> bytes`: 1942, + `bool_and(arg1: bool) -> bool`: 31, + `bool_or(arg1: bool) -> bool`: 32, + `boolin(input: anyelement) -> bool`: 1949, + `boolout(bool: bool) -> bytes`: 1948, + `boolrecv(input: anyelement) -> bool`: 1947, + `boolsend(bool: bool) -> bytes`: 1946, + `box2d_in(input: anyelement) -> box2d`: 1973, + `box2d_out(box2d: box2d) -> bytes`: 1972, + `box2d_recv(input: anyelement) -> box2d`: 1971, + `box2d_send(box2d: box2d) -> bytes`: 1970, + `bpcharin(input: anyelement) -> char`: 1873, + `bpcharout(char: char) -> bytes`: 1872, + `bpcharrecv(input: anyelement) -> char`: 1871, + `bpcharsend(char: char) -> bytes`: 1870, + `broadcast(val: inet) -> inet`: 874, + `btrim(input: string, trim_chars: string) -> string`: 933, + `btrim(val: string) -> string`: 934, + `byteain(input: anyelement) -> bytes`: 1921, + `byteaout(bytes: bytes) -> bytes`: 1920, + `bytearecv(input: anyelement) -> bytes`: 1919, + `byteasend(bytes: bytes) -> bytes`: 1918, + `cardinality(input: anyelement[]) -> int`: 1079, + `cbrt(val: float) -> float`: 752, + `cbrt(val: decimal) -> decimal`: 753, + `ceil(val: float) -> float`: 754, + `ceil(val: decimal) -> decimal`: 755, + `ceil(val: int) -> float`: 756, + `ceiling(val: float) -> float`: 757, + `ceiling(val: decimal) -> decimal`: 758, + `ceiling(val: int) -> float`: 759, + `char_length(val: string) -> int`: 817, + `char_length(val: bytes) -> int`: 818, + `character_length(val: string) -> int`: 819, + `character_length(val: bytes) -> int`: 820, + `charin(input: anyelement) -> "char"`: 1929, + `charout("char": "char") -> bytes`: 1928, + `charrecv(input: anyelement) -> "char"`: 1927, + `charsend("char": "char") -> bytes`: 1926, + `chr(val: int) -> string`: 895, + `clock_timestamp() -> timestamptz`: 1037, + `clock_timestamp() -> timestamp`: 1038, + `cluster_logical_timestamp() -> decimal`: 1035, + `col_description(table_oid: oid, column_number: int) -> string`: 1422, + `compress(data: bytes, codec: string) -> bytes`: 892, + `concat(string...) -> string`: 848, + `concat_agg(arg1: string) -> string`: 33, + `concat_agg(arg1: bytes) -> bytes`: 34, + `concat_ws(string...) -> string`: 849, + `convert_from(str: bytes, enc: string) -> string`: 850, + `convert_to(str: string, enc: string) -> bytes`: 851, + `corr(arg1: float, arg2: float) -> float`: 35, + `corr(arg1: int, arg2: int) -> float`: 36, + `corr(arg1: decimal, arg2: decimal) -> float`: 37, + `corr(arg1: float, arg2: int) -> float`: 38, + `corr(arg1: float, arg2: decimal) -> float`: 39, + `corr(arg1: int, arg2: float) -> float`: 40, + `corr(arg1: int, arg2: decimal) -> float`: 41, + `corr(arg1: decimal, arg2: float) -> float`: 42, + `corr(arg1: decimal, arg2: int) -> float`: 43, + `cos(val: float) -> float`: 760, + `cosd(val: float) -> float`: 761, + `cosh(val: float) -> float`: 762, + `cot(val: float) -> float`: 763, + `cotd(val: float) -> float`: 764, + `count(arg1: anyelement) -> int`: 165, + `count_rows() -> int`: 166, + `covar_pop(arg1: float, arg2: float) -> float`: 44, + `covar_pop(arg1: int, arg2: int) -> float`: 45, + `covar_pop(arg1: decimal, arg2: decimal) -> float`: 46, + `covar_pop(arg1: float, arg2: int) -> float`: 47, + `covar_pop(arg1: float, arg2: decimal) -> float`: 48, + `covar_pop(arg1: int, arg2: float) -> float`: 49, + `covar_pop(arg1: int, arg2: decimal) -> float`: 50, + `covar_pop(arg1: decimal, arg2: float) -> float`: 51, + `covar_pop(arg1: decimal, arg2: int) -> float`: 52, + `covar_samp(arg1: float, arg2: float) -> float`: 75, + `covar_samp(arg1: int, arg2: int) -> float`: 76, + `covar_samp(arg1: decimal, arg2: decimal) -> float`: 77, + `covar_samp(arg1: float, arg2: int) -> float`: 78, + `covar_samp(arg1: float, arg2: decimal) -> float`: 79, + `covar_samp(arg1: int, arg2: float) -> float`: 80, + `covar_samp(arg1: int, arg2: decimal) -> float`: 81, + `covar_samp(arg1: decimal, arg2: float) -> float`: 82, + `covar_samp(arg1: decimal, arg2: int) -> float`: 83, + `crc32c(string...) -> int`: 918, + `crc32c(bytes...) -> int`: 919, + `crc32ieee(string...) -> int`: 916, + `crc32ieee(bytes...) -> int`: 917, + `crdb_internal.active_version() -> jsonb`: 1296, + `crdb_internal.approximate_timestamp(timestamp: decimal) -> timestamp`: 1298, + `crdb_internal.assignment_cast(val: anyelement, type: anyelement) -> anyelement`: 1341, + `crdb_internal.check_consistency(stats_only: bool, start_key: bytes, end_key: bytes) -> tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail}`: 347, + `crdb_internal.check_password_hash_format(password: bytes) -> string`: 1376, + `crdb_internal.cluster_id() -> uuid`: 1299, + `crdb_internal.cluster_name() -> string`: 1301, + `crdb_internal.cluster_setting_encoded_default(setting: string) -> string`: 1293, + `crdb_internal.compact_engine_span(node_id: int, store_id: int, start_key: bytes, end_key: bytes) -> bool`: 1356, + `crdb_internal.complete_replication_stream(stream_id: int, successful_ingestion: bool) -> int`: 1552, + `crdb_internal.complete_stream_ingestion_job(job_id: int, cutover_ts: timestamptz) -> int`: 1545, + `crdb_internal.completed_migrations() -> string[]`: 1344, + `crdb_internal.create_join_token() -> string`: 1303, + `crdb_internal.create_regclass(oid: oid, name: string) -> regclass`: 2022, + `crdb_internal.create_regnamespace(oid: oid, name: string) -> regnamespace`: 2024, + `crdb_internal.create_regproc(oid: oid, name: string) -> regproc`: 2026, + `crdb_internal.create_regprocedure(oid: oid, name: string) -> regprocedure`: 2028, + `crdb_internal.create_regrole(oid: oid, name: string) -> regrole`: 2030, + `crdb_internal.create_regtype(oid: oid, name: string) -> regtype`: 2032, + `crdb_internal.create_session_revival_token() -> bytes`: 1372, + `crdb_internal.create_sql_schema_telemetry_job() -> int`: 1378, + `crdb_internal.create_tenant(id: int) -> int`: 1302, + `crdb_internal.datums_to_bytes(anyelement...) -> bytes`: 1276, + `crdb_internal.decode_cluster_setting(setting: string, value: string) -> string`: 1294, + `crdb_internal.decode_external_plan_gist(gist: string) -> string`: 355, + `crdb_internal.decode_plan_gist(gist: string) -> string`: 354, + `crdb_internal.deserialize_session(session: bytes) -> bool`: 1371, + `crdb_internal.destroy_tenant(id: int) -> int`: 1304, + `crdb_internal.destroy_tenant(id: int, synchronous: bool) -> int`: 1305, + `crdb_internal.encode_key(table_id: int, index_id: int, row_tuple: anyelement) -> bytes`: 1307, + `crdb_internal.filter_multiregion_fields_from_zone_config_sql(val: string) -> string`: 1366, + `crdb_internal.force_assertion_error(msg: string) -> int`: 1311, + `crdb_internal.force_delete_table_data(id: int) -> bool`: 1369, + `crdb_internal.force_error(errorCode: string, msg: string) -> int`: 1308, + `crdb_internal.force_log_fatal(msg: string) -> int`: 1314, + `crdb_internal.force_panic(msg: string) -> int`: 1313, + `crdb_internal.force_retry(val: interval) -> int`: 1315, + `crdb_internal.gc_tenant(id: int) -> int`: 1354, + `crdb_internal.get_database_id(name: string) -> int`: 1328, + `crdb_internal.get_namespace_id(parent_id: int, name: string) -> int`: 1326, + `crdb_internal.get_namespace_id(parent_id: int, parent_schema_id: int, name: string) -> int`: 1327, + `crdb_internal.get_vmodule() -> string`: 1331, + `crdb_internal.get_zone_config(namespace_id: int) -> bytes`: 1329, + `crdb_internal.has_role_option(option: string) -> bool`: 1340, + `crdb_internal.increment_feature_counter(feature: string) -> bool`: 1357, + `crdb_internal.index_span(table_id: int, index_id: int) -> bytes[]`: 1322, + `crdb_internal.is_admin() -> bool`: 1339, + `crdb_internal.is_at_least_version(version: string) -> bool`: 1297, + `crdb_internal.is_constraint_active(table_name: string, constraint_name: string) -> bool`: 1382, + `crdb_internal.json_to_pb(pbname: string, json: jsonb) -> bytes`: 1273, + `crdb_internal.kv_enqueue_replica(range_id: int, queue_name: string, skip_should_queue: bool) -> bool`: 1385, + `crdb_internal.kv_enqueue_replica(range_id: int, queue_name: string, skip_should_queue: bool, should_return_trace: bool) -> string`: 1386, + `crdb_internal.kv_enqueue_replica(range_id: int, queue_name: string, skip_should_queue: bool, store_id: int) -> bool`: 1387, + `crdb_internal.kv_set_queue_active(queue_name: string, active: bool) -> bool`: 1383, + `crdb_internal.kv_set_queue_active(queue_name: string, active: bool, store_id: int) -> bool`: 1384, + `crdb_internal.lease_holder(key: bytes) -> int`: 1316, + `crdb_internal.list_sql_keys_in_range(range_id: int) -> tuple{string AS key, string AS value}`: 348, + `crdb_internal.locality_value(key: string) -> string`: 1292, + `crdb_internal.merge_statement_stats(input: jsonb[]) -> jsonb`: 1277, + `crdb_internal.merge_stats_metadata(input: jsonb[]) -> jsonb`: 1279, + `crdb_internal.merge_transaction_stats(input: jsonb[]) -> jsonb`: 1278, + `crdb_internal.no_constant_folding(input: anyelement) -> anyelement`: 1317, + `crdb_internal.node_executable_version() -> string`: 1295, + `crdb_internal.node_id() -> int`: 1300, + `crdb_internal.notice(msg: string) -> int`: 1309, + `crdb_internal.notice(severity: string, msg: string) -> int`: 1310, + `crdb_internal.num_geo_inverted_index_entries(table_id: int, index_id: int, val: geography) -> int`: 1332, + `crdb_internal.num_geo_inverted_index_entries(table_id: int, index_id: int, val: geometry) -> int`: 1333, + `crdb_internal.num_inverted_index_entries(val: jsonb) -> int`: 1334, + `crdb_internal.num_inverted_index_entries(val: anyelement[]) -> int`: 1335, + `crdb_internal.num_inverted_index_entries(val: jsonb, version: int) -> int`: 1336, + `crdb_internal.num_inverted_index_entries(val: string, version: int) -> int`: 1337, + `crdb_internal.num_inverted_index_entries(val: anyelement[], version: int) -> int`: 1338, + `crdb_internal.payloads_for_span(span_id: int) -> tuple{string AS payload_type, jsonb AS payload_jsonb}`: 349, + `crdb_internal.payloads_for_trace(trace_id: int) -> tuple{int AS span_id, string AS payload_type, jsonb AS payload_jsonb}`: 350, + `crdb_internal.pb_to_json(pbname: string, data: bytes) -> jsonb`: 1270, + `crdb_internal.pb_to_json(pbname: string, data: bytes, emit_defaults: bool) -> jsonb`: 1271, + `crdb_internal.pb_to_json(pbname: string, data: bytes, emit_defaults: bool, emit_redacted: bool) -> jsonb`: 1272, + `crdb_internal.pretty_key(raw_key: bytes, skip_fields: int) -> string`: 1323, + `crdb_internal.pretty_span(raw_key_start: bytes, raw_key_end: bytes, skip_fields: int) -> string`: 1324, + `crdb_internal.probe_ranges(timeout: interval, probe_type: unknown_enum) -> tuple{int AS range_id, string AS error, int AS end_to_end_latency_ms, string AS verbose_trace}`: 356, + `crdb_internal.range_stats(key: bytes) -> jsonb`: 1325, + `crdb_internal.read_file(uri: string) -> bytes`: 1274, + `crdb_internal.repair_ttl_table_scheduled_job(oid: oid) -> void`: 1375, + `crdb_internal.replication_stream_progress(stream_id: int, frontier_ts: string) -> bytes`: 1549, + `crdb_internal.replication_stream_spec(stream_id: int) -> bytes`: 1551, + `crdb_internal.request_statement_bundle(stmtFingerprint: string, samplingProbability: float, minExecutionLatency: interval, expiresAfter: interval) -> bool`: 1388, + `crdb_internal.reset_index_usage_stats() -> bool`: 1367, + `crdb_internal.reset_multi_region_zone_configs_for_database(id: int) -> bool`: 1365, + `crdb_internal.reset_multi_region_zone_configs_for_table(id: int) -> bool`: 1364, + `crdb_internal.reset_sql_stats() -> bool`: 1368, + `crdb_internal.revalidate_unique_constraint(table_name: string, constraint_name: string) -> void`: 1381, + `crdb_internal.revalidate_unique_constraints_in_all_tables() -> void`: 1379, + `crdb_internal.revalidate_unique_constraints_in_table(table_name: string) -> void`: 1380, + `crdb_internal.round_decimal_values(val: decimal, scale: int) -> decimal`: 1342, + `crdb_internal.round_decimal_values(val: decimal[], scale: int) -> decimal[]`: 1343, + `crdb_internal.scan(start_key: bytes, end_key: bytes) -> tuple{bytes AS key, bytes AS value}`: 315, + `crdb_internal.scan(span: bytes[]) -> tuple{bytes AS key, bytes AS value}`: 316, + `crdb_internal.schedule_sql_stats_compaction() -> bool`: 1377, + `crdb_internal.serialize_session() -> bytes`: 1370, + `crdb_internal.set_compaction_concurrency(node_id: int, store_id: int, compaction_concurrency: int) -> bool`: 1389, + `crdb_internal.set_trace_verbose(trace_id: int, verbosity: bool) -> bool`: 1291, + `crdb_internal.set_vmodule(vmodule_string: string) -> int`: 1330, + `crdb_internal.show_create_all_schemas(database_name: string) -> string`: 351, + `crdb_internal.show_create_all_tables(database_name: string) -> string`: 352, + `crdb_internal.show_create_all_types(database_name: string) -> string`: 353, + `crdb_internal.sql_liveness_is_alive(session_id: bytes) -> bool`: 1353, + `crdb_internal.start_replication_stream(tenant_id: int) -> int`: 1548, + `crdb_internal.stream_ingestion_stats_json(job_id: int) -> jsonb`: 1546, + `crdb_internal.stream_ingestion_stats_pb(job_id: int) -> bytes`: 1547, + `crdb_internal.stream_partition(stream_id: int, partition_spec: bytes) -> bytes`: 1550, + `crdb_internal.table_span(table_id: int) -> bytes[]`: 1321, + `crdb_internal.tenant_span(tenant_id: int) -> bytes[]`: 1320, + `crdb_internal.testing_callback(name: string) -> int`: 321, + `crdb_internal.to_json_as_changefeed_with_flags(tuple, string...) -> bytes`: 470, + `crdb_internal.trace_id() -> int`: 1290, + `crdb_internal.trim_tenant_prefix(key: bytes) -> bytes`: 1318, + `crdb_internal.trim_tenant_prefix(keys: bytes[]) -> bytes[]`: 1319, + `crdb_internal.unary_table() -> tuple`: 329, + `crdb_internal.unsafe_clear_gossip_info(key: string) -> bool`: 1306, + `crdb_internal.unsafe_delete_descriptor(id: int) -> bool`: 1347, + `crdb_internal.unsafe_delete_descriptor(id: int, force: bool) -> bool`: 1348, + `crdb_internal.unsafe_delete_namespace_entry(parent_id: int, parent_schema_id: int, name: string, desc_id: int) -> bool`: 1351, + `crdb_internal.unsafe_delete_namespace_entry(parent_id: int, parent_schema_id: int, name: string, desc_id: int, force: bool) -> bool`: 1352, + `crdb_internal.unsafe_upsert_descriptor(id: int, desc: bytes) -> bool`: 1345, + `crdb_internal.unsafe_upsert_descriptor(id: int, desc: bytes, force: bool) -> bool`: 1346, + `crdb_internal.unsafe_upsert_namespace_entry(parent_id: int, parent_schema_id: int, name: string, desc_id: int, force: bool) -> bool`: 1349, + `crdb_internal.unsafe_upsert_namespace_entry(parent_id: int, parent_schema_id: int, name: string, desc_id: int) -> bool`: 1350, + `crdb_internal.update_tenant_resource_limits(tenant_id: int, available_request_units: float, refill_rate: float, max_burst_request_units: float, as_of: timestamp, as_of_consumed_request_units: float) -> int`: 1355, + `crdb_internal.validate_multi_region_zone_configs() -> bool`: 1363, + `crdb_internal.validate_session_revival_token(token: bytes) -> bool`: 1373, + `crdb_internal.validate_ttl_scheduled_jobs() -> void`: 1374, + `crdb_internal.void_func() -> void`: 1312, + `crdb_internal.write_file(data: bytes, uri: string) -> int`: 1275, + `crypt(password: string, salt: string) -> string`: 1537, + `cume_dist() -> float`: 1559, + `current_database() -> string`: 1285, + `current_date() -> date`: 1000, + `current_schema() -> string`: 1286, + `current_schemas(include_pg_catalog: bool) -> string[]`: 1287, + `current_setting(setting_name: string) -> string`: 1522, + `current_setting(setting_name: string, missing_ok: bool) -> string`: 1523, + `current_time() -> timetz`: 1004, + `current_time() -> time`: 1005, + `current_time(precision: int) -> timetz`: 1006, + `current_time(precision: int) -> time`: 1007, + `current_timestamp(precision: int) -> timestamptz`: 1008, + `current_timestamp(precision: int) -> timestamp`: 1009, + `current_timestamp(precision: int) -> date`: 1010, + `current_timestamp() -> timestamptz`: 1011, + `current_timestamp() -> timestamp`: 1012, + `current_timestamp() -> date`: 1013, + `current_user() -> string`: 1288, + `currval(sequence_name: string) -> int`: 972, + `currval(sequence_name: regclass) -> int`: 973, + `date_in(input: anyelement) -> date`: 1957, + `date_out(date: date) -> bytes`: 1956, + `date_part(element: string, input: timestamp) -> float`: 1046, + `date_part(element: string, input: interval) -> float`: 1047, + `date_part(element: string, input: date) -> float`: 1048, + `date_part(element: string, input: timestamptz) -> float`: 1049, + `date_part(element: string, input: time) -> float`: 1050, + `date_part(element: string, input: timetz) -> float`: 1051, + `date_recv(input: anyelement) -> date`: 1955, + `date_send(date: date) -> bytes`: 1954, + `date_trunc(element: string, input: timestamp) -> timestamp`: 1053, + `date_trunc(element: string, input: date) -> timestamptz`: 1054, + `date_trunc(element: string, input: time) -> interval`: 1055, + `date_trunc(element: string, input: timestamptz) -> timestamptz`: 1056, + `date_trunc(element: string, input: interval) -> interval`: 1057, + `decode(text: string, format: string) -> bytes`: 891, + `decompress(data: bytes, codec: string) -> bytes`: 893, + `default_to_database_primary_region(val: string) -> string`: 1361, + `degrees(val: float) -> float`: 765, + `dense_rank() -> int`: 1557, + `difference(source: string, target: string) -> string`: 1237, + `digest(data: string, type: string) -> bytes`: 1538, + `digest(data: bytes, type: string) -> bytes`: 1539, + `div(x: float, y: float) -> float`: 766, + `div(x: decimal, y: decimal) -> decimal`: 767, + `div(x: int, y: int) -> int`: 768, + `encode(data: bytes, format: string) -> string`: 890, + `enum_first(val: anyenum) -> anyelement`: 1280, + `enum_in(input: anyelement) -> anyenum`: 2021, + `enum_last(val: anyenum) -> anyelement`: 1281, + `enum_out(anyenum: anyenum) -> bytes`: 2020, + `enum_range(val: anyenum) -> anyelement`: 1282, + `enum_range(lower: anyenum, upper: anyenum) -> anyelement`: 1283, + `enum_recv(input: anyelement) -> anyenum`: 2019, + `enum_send(anyenum: anyenum) -> bytes`: 2018, + `every(arg1: bool) -> bool`: 167, + `exp(val: float) -> float`: 769, + `exp(val: decimal) -> decimal`: 770, + `experimental_follower_read_timestamp() -> timestamptz`: 1030, + `experimental_strftime(input: timestamp, extract_format: string) -> string`: 986, + `experimental_strftime(input: date, extract_format: string) -> string`: 987, + `experimental_strftime(input: timestamptz, extract_format: string) -> string`: 988, + `experimental_strptime(input: string, format: string) -> timestamptz`: 990, + `experimental_uuid_v4() -> bytes`: 979, + `extract(element: string, input: timestamp) -> float`: 1040, + `extract(element: string, input: interval) -> float`: 1041, + `extract(element: string, input: date) -> float`: 1042, + `extract(element: string, input: timestamptz) -> float`: 1043, + `extract(element: string, input: time) -> float`: 1044, + `extract(element: string, input: timetz) -> float`: 1045, + `extract_duration(element: string, input: interval) -> int`: 1052, + `family(val: inet) -> int`: 875, + `final_corr(arg1: decimal[]) -> float`: 62, + `final_covar_pop(arg1: decimal[]) -> float`: 53, + `final_covar_samp(arg1: decimal[]) -> float`: 63, + `final_regr_avgx(arg1: decimal[]) -> float`: 57, + `final_regr_avgy(arg1: decimal[]) -> float`: 58, + `final_regr_intercept(arg1: decimal[]) -> float`: 59, + `final_regr_r2(arg1: decimal[]) -> float`: 60, + `final_regr_slope(arg1: decimal[]) -> float`: 61, + `final_regr_sxx(arg1: decimal[]) -> float`: 54, + `final_regr_sxy(arg1: decimal[]) -> float`: 55, + `final_regr_syy(arg1: decimal[]) -> float`: 56, + `final_sqrdiff(arg1: decimal, arg2: decimal, arg3: int) -> decimal`: 64, + `final_sqrdiff(arg1: float, arg2: float, arg3: int) -> float`: 65, + `final_stddev(arg1: decimal, arg2: decimal, arg3: int) -> decimal`: 226, + `final_stddev(arg1: float, arg2: float, arg3: int) -> float`: 227, + `final_stddev_pop(arg1: decimal, arg2: decimal, arg3: int) -> decimal`: 228, + `final_stddev_pop(arg1: float, arg2: float, arg3: int) -> float`: 229, + `final_var_pop(arg1: decimal, arg2: decimal, arg3: int) -> decimal`: 224, + `final_var_pop(arg1: float, arg2: float, arg3: int) -> float`: 225, + `final_variance(arg1: decimal, arg2: decimal, arg3: int) -> decimal`: 222, + `final_variance(arg1: float, arg2: float, arg3: int) -> float`: 223, + `first_value(val: bool) -> bool`: 1681, + `first_value(val: box2d) -> box2d`: 1682, + `first_value(val: int) -> int`: 1683, + `first_value(val: float) -> float`: 1684, + `first_value(val: decimal) -> decimal`: 1685, + `first_value(val: date) -> date`: 1686, + `first_value(val: timestamp) -> timestamp`: 1687, + `first_value(val: interval) -> interval`: 1688, + `first_value(val: geography) -> geography`: 1689, + `first_value(val: geometry) -> geometry`: 1690, + `first_value(val: string) -> string`: 1691, + `first_value(val: bytes) -> bytes`: 1692, + `first_value(val: timestamptz) -> timestamptz`: 1693, + `first_value(val: oid) -> oid`: 1694, + `first_value(val: uuid) -> uuid`: 1695, + `first_value(val: inet) -> inet`: 1696, + `first_value(val: time) -> time`: 1697, + `first_value(val: timetz) -> timetz`: 1698, + `first_value(val: jsonb) -> jsonb`: 1699, + `first_value(val: varbit) -> varbit`: 1700, + `float4in(input: anyelement) -> float4`: 1913, + `float4out(float4: float4) -> bytes`: 1912, + `float4recv(input: anyelement) -> float4`: 1911, + `float4send(float4: float4) -> bytes`: 1910, + `float8in(input: anyelement) -> float`: 1977, + `float8out(float: float) -> bytes`: 1976, + `float8recv(input: anyelement) -> float`: 1975, + `float8send(float: float) -> bytes`: 1974, + `floor(val: float) -> float`: 771, + `floor(val: decimal) -> decimal`: 772, + `floor(val: int) -> float`: 773, + `fnv32(string...) -> int`: 908, + `fnv32(bytes...) -> int`: 909, + `fnv32a(string...) -> int`: 910, + `fnv32a(bytes...) -> int`: 911, + `fnv64(string...) -> int`: 912, + `fnv64(bytes...) -> int`: 913, + `fnv64a(string...) -> int`: 914, + `fnv64a(bytes...) -> int`: 915, + `follower_read_timestamp() -> timestamptz`: 1029, + `format(string, anyelement...) -> string`: 311, + `format_type(type_oid: oid, typemod: int) -> string`: 1421, + `from_ip(val: bytes) -> string`: 886, + `from_uuid(val: bytes) -> string`: 869, + `gateway_region() -> string`: 1360, + `gen_random_ulid() -> uuid`: 870, + `gen_random_uuid() -> uuid`: 1540, + `gen_salt(type: string) -> string`: 1541, + `gen_salt(type: string, iter_count: int) -> string`: 1542, + `generate_series(start: int, end: int) -> int`: 317, + `generate_series(start: int, end: int, step: int) -> int`: 318, + `generate_series(start: timestamp, end: timestamp, step: interval) -> timestamp`: 319, + `generate_series(start: timestamptz, end: timestamptz, step: interval) -> timestamptz`: 320, + `generate_subscripts(array: anyelement[]) -> int`: 330, + `generate_subscripts(array: anyelement[], dim: int) -> int`: 331, + `generate_subscripts(array: anyelement[], dim: int, reverse: bool) -> int`: 332, + `geography_in(input: anyelement) -> geography`: 1869, + `geography_out(geography: geography) -> bytes`: 1868, + `geography_recv(input: anyelement) -> geography`: 1867, + `geography_send(geography: geography) -> bytes`: 1866, + `geometry_in(input: anyelement) -> geometry`: 1877, + `geometry_out(geometry: geometry) -> bytes`: 1876, + `geometry_recv(input: anyelement) -> geometry`: 1875, + `geometry_send(geometry: geometry) -> bytes`: 1874, + `geometrytype(geometry: geometry) -> string`: 566, + `geomfromewkb(val: bytes) -> geometry`: 1742, + `geomfromewkt(val: string) -> geometry`: 1741, + `get_bit(bit_string: varbit, index: int) -> int`: 852, + `get_bit(byte_string: bytes, index: int) -> int`: 853, + `get_byte(byte_string: bytes, index: int) -> int`: 854, + `getdatabaseencoding() -> string`: 1402, + `greatest(anyelement...) -> anyelement`: 981, + `has_any_column_privilege(table: string, privilege: string) -> bool`: 1438, + `has_any_column_privilege(table: oid, privilege: string) -> bool`: 1439, + `has_any_column_privilege(user: string, table: string, privilege: string) -> bool`: 1440, + `has_any_column_privilege(user: string, table: oid, privilege: string) -> bool`: 1441, + `has_any_column_privilege(user: oid, table: string, privilege: string) -> bool`: 1442, + `has_any_column_privilege(user: oid, table: oid, privilege: string) -> bool`: 1443, + `has_column_privilege(table: string, column: string, privilege: string) -> bool`: 1444, + `has_column_privilege(table: string, column: int, privilege: string) -> bool`: 1445, + `has_column_privilege(table: oid, column: string, privilege: string) -> bool`: 1446, + `has_column_privilege(table: oid, column: int, privilege: string) -> bool`: 1447, + `has_column_privilege(user: string, table: string, column: string, privilege: string) -> bool`: 1448, + `has_column_privilege(user: string, table: string, column: int, privilege: string) -> bool`: 1449, + `has_column_privilege(user: string, table: oid, column: string, privilege: string) -> bool`: 1450, + `has_column_privilege(user: string, table: oid, column: int, privilege: string) -> bool`: 1451, + `has_column_privilege(user: oid, table: string, column: string, privilege: string) -> bool`: 1452, + `has_column_privilege(user: oid, table: string, column: int, privilege: string) -> bool`: 1453, + `has_column_privilege(user: oid, table: oid, column: string, privilege: string) -> bool`: 1454, + `has_column_privilege(user: oid, table: oid, column: int, privilege: string) -> bool`: 1455, + `has_database_privilege(database: string, privilege: string) -> bool`: 1456, + `has_database_privilege(database: oid, privilege: string) -> bool`: 1457, + `has_database_privilege(user: string, database: string, privilege: string) -> bool`: 1458, + `has_database_privilege(user: string, database: oid, privilege: string) -> bool`: 1459, + `has_database_privilege(user: oid, database: string, privilege: string) -> bool`: 1460, + `has_database_privilege(user: oid, database: oid, privilege: string) -> bool`: 1461, + `has_foreign_data_wrapper_privilege(fdw: string, privilege: string) -> bool`: 1462, + `has_foreign_data_wrapper_privilege(fdw: oid, privilege: string) -> bool`: 1463, + `has_foreign_data_wrapper_privilege(user: string, fdw: string, privilege: string) -> bool`: 1464, + `has_foreign_data_wrapper_privilege(user: string, fdw: oid, privilege: string) -> bool`: 1465, + `has_foreign_data_wrapper_privilege(user: oid, fdw: string, privilege: string) -> bool`: 1466, + `has_foreign_data_wrapper_privilege(user: oid, fdw: oid, privilege: string) -> bool`: 1467, + `has_function_privilege(function: string, privilege: string) -> bool`: 1468, + `has_function_privilege(function: oid, privilege: string) -> bool`: 1469, + `has_function_privilege(user: string, function: string, privilege: string) -> bool`: 1470, + `has_function_privilege(user: string, function: oid, privilege: string) -> bool`: 1471, + `has_function_privilege(user: oid, function: string, privilege: string) -> bool`: 1472, + `has_function_privilege(user: oid, function: oid, privilege: string) -> bool`: 1473, + `has_language_privilege(language: string, privilege: string) -> bool`: 1474, + `has_language_privilege(language: oid, privilege: string) -> bool`: 1475, + `has_language_privilege(user: string, language: string, privilege: string) -> bool`: 1476, + `has_language_privilege(user: string, language: oid, privilege: string) -> bool`: 1477, + `has_language_privilege(user: oid, language: string, privilege: string) -> bool`: 1478, + `has_language_privilege(user: oid, language: oid, privilege: string) -> bool`: 1479, + `has_schema_privilege(schema: string, privilege: string) -> bool`: 1480, + `has_schema_privilege(schema: oid, privilege: string) -> bool`: 1481, + `has_schema_privilege(user: string, schema: string, privilege: string) -> bool`: 1482, + `has_schema_privilege(user: string, schema: oid, privilege: string) -> bool`: 1483, + `has_schema_privilege(user: oid, schema: string, privilege: string) -> bool`: 1484, + `has_schema_privilege(user: oid, schema: oid, privilege: string) -> bool`: 1485, + `has_sequence_privilege(sequence: string, privilege: string) -> bool`: 1486, + `has_sequence_privilege(sequence: oid, privilege: string) -> bool`: 1487, + `has_sequence_privilege(user: string, sequence: string, privilege: string) -> bool`: 1488, + `has_sequence_privilege(user: string, sequence: oid, privilege: string) -> bool`: 1489, + `has_sequence_privilege(user: oid, sequence: string, privilege: string) -> bool`: 1490, + `has_sequence_privilege(user: oid, sequence: oid, privilege: string) -> bool`: 1491, + `has_server_privilege(server: string, privilege: string) -> bool`: 1492, + `has_server_privilege(server: oid, privilege: string) -> bool`: 1493, + `has_server_privilege(user: string, server: string, privilege: string) -> bool`: 1494, + `has_server_privilege(user: string, server: oid, privilege: string) -> bool`: 1495, + `has_server_privilege(user: oid, server: string, privilege: string) -> bool`: 1496, + `has_server_privilege(user: oid, server: oid, privilege: string) -> bool`: 1497, + `has_table_privilege(table: string, privilege: string) -> bool`: 1498, + `has_table_privilege(table: oid, privilege: string) -> bool`: 1499, + `has_table_privilege(user: string, table: string, privilege: string) -> bool`: 1500, + `has_table_privilege(user: string, table: oid, privilege: string) -> bool`: 1501, + `has_table_privilege(user: oid, table: string, privilege: string) -> bool`: 1502, + `has_table_privilege(user: oid, table: oid, privilege: string) -> bool`: 1503, + `has_tablespace_privilege(tablespace: string, privilege: string) -> bool`: 1504, + `has_tablespace_privilege(tablespace: oid, privilege: string) -> bool`: 1505, + `has_tablespace_privilege(user: string, tablespace: string, privilege: string) -> bool`: 1506, + `has_tablespace_privilege(user: string, tablespace: oid, privilege: string) -> bool`: 1507, + `has_tablespace_privilege(user: oid, tablespace: string, privilege: string) -> bool`: 1508, + `has_tablespace_privilege(user: oid, tablespace: oid, privilege: string) -> bool`: 1509, + `has_type_privilege(type: string, privilege: string) -> bool`: 1510, + `has_type_privilege(type: oid, privilege: string) -> bool`: 1511, + `has_type_privilege(user: string, type: string, privilege: string) -> bool`: 1512, + `has_type_privilege(user: string, type: oid, privilege: string) -> bool`: 1513, + `has_type_privilege(user: oid, type: string, privilege: string) -> bool`: 1514, + `has_type_privilege(user: oid, type: oid, privilege: string) -> bool`: 1515, + `hlc_to_timestamp(hlc: decimal) -> timestamptz`: 1036, + `hmac(data: string, key: string, type: string) -> bytes`: 1543, + `hmac(data: bytes, key: bytes, type: string) -> bytes`: 1544, + `host(val: inet) -> string`: 876, + `hostmask(val: inet) -> inet`: 877, + `ilike_escape(unescaped: string, pattern: string, escape: string) -> bool`: 949, + `inet(val: string) -> inet`: 885, + `inet_client_addr() -> inet`: 1525, + `inet_client_port() -> int`: 1526, + `inet_contained_by_or_equals(val: inet, container: inet) -> bool`: 883, + `inet_contains_or_equals(container: inet, val: inet) -> bool`: 884, + `inet_same_family(val: inet, val: inet) -> bool`: 882, + `inet_server_addr() -> inet`: 1527, + `inet_server_port() -> int`: 1528, + `inetin(input: anyelement) -> inet`: 1953, + `inetout(inet: inet) -> bytes`: 1952, + `inetrecv(input: anyelement) -> inet`: 1951, + `inetsend(inet: inet) -> bytes`: 1950, + `information_schema._pg_char_max_length(typid: oid, typmod: int4) -> int`: 1532, + `information_schema._pg_expandarray(input: anyelement[]) -> anyelement`: 328, + `information_schema._pg_index_position(oid: oid, col: int2) -> int`: 1533, + `information_schema._pg_numeric_precision(typid: oid, typmod: int4) -> int`: 1534, + `information_schema._pg_numeric_precision_radix(typid: oid, typmod: int4) -> int`: 1535, + `information_schema._pg_numeric_scale(typid: oid, typmod: int4) -> int`: 1536, + `information_schema._pg_truetypid(pg_attribute: tuple, pg_type: tuple) -> oid`: 1530, + `information_schema._pg_truetypmod(pg_attribute: tuple, pg_type: tuple) -> int4`: 1531, + `initcap(val: string) -> string`: 957, + `int2in(input: anyelement) -> int2`: 2001, + `int2out(int2: int2) -> bytes`: 2000, + `int2recv(input: anyelement) -> int2`: 1999, + `int2send(int2: int2) -> bytes`: 1998, + `int2vectorin(input: anyelement) -> int2vector`: 1881, + `int2vectorout(int2vector: int2vector) -> bytes`: 1880, + `int2vectorrecv(input: anyelement) -> int2vector`: 1879, + `int2vectorsend(int2vector: int2vector) -> bytes`: 1878, + `int4in(input: anyelement) -> int4`: 1981, + `int4out(int4: int4) -> bytes`: 1980, + `int4recv(input: anyelement) -> int4`: 1979, + `int4send(int4: int4) -> bytes`: 1978, + `int8in(input: anyelement) -> int`: 1901, + `int8out(int: int) -> bytes`: 1900, + `int8recv(input: anyelement) -> int`: 1899, + `int8send(int: int) -> bytes`: 1898, + `interval_in(input: anyelement) -> interval`: 2005, + `interval_out(interval: interval) -> bytes`: 2004, + `interval_recv(input: anyelement) -> interval`: 2003, + `interval_send(interval: interval) -> bytes`: 2002, + `isnan(val: float) -> bool`: 774, + `isnan(val: decimal) -> bool`: 775, + `json_agg(arg1: anyelement) -> jsonb`: 250, + `json_array_elements(input: jsonb) -> jsonb`: 333, + `json_array_elements_text(input: jsonb) -> string`: 335, + `json_array_length(json: jsonb) -> int`: 1266, + `json_build_array(anyelement...) -> jsonb`: 1256, + `json_build_object(anyelement...) -> jsonb`: 1258, + `json_each(input: jsonb) -> tuple{string AS key, jsonb AS value}`: 339, + `json_each_text(input: jsonb) -> tuple{string AS key, string AS value}`: 341, + `json_extract_path(jsonb, string...) -> jsonb`: 1241, + `json_extract_path_text(jsonb, string...) -> string`: 1243, + `json_object(texts: string[]) -> jsonb`: 1260, + `json_object(keys: string[], values: string[]) -> jsonb`: 1261, + `json_object_agg(arg1: string, arg2: anyelement) -> jsonb`: 252, + `json_object_keys(input: jsonb) -> string`: 337, + `json_populate_record(base: anyelement, from_json: jsonb) -> anyelement`: 343, + `json_populate_recordset(base: anyelement, from_json: jsonb) -> anyelement`: 345, + `json_remove_path(val: jsonb, path: string[]) -> jsonb`: 1240, + `json_set(val: jsonb, path: string[], to: jsonb) -> jsonb`: 1245, + `json_set(val: jsonb, path: string[], to: jsonb, create_missing: bool) -> jsonb`: 1246, + `json_strip_nulls(from_json: jsonb) -> jsonb`: 1264, + `json_to_record(input: jsonb) -> tuple`: 466, + `json_to_recordset(input: jsonb) -> tuple`: 467, + `json_typeof(val: jsonb) -> string`: 1252, + `json_valid(string: string) -> bool`: 1269, + `jsonb_agg(arg1: anyelement) -> jsonb`: 251, + `jsonb_array_elements(input: jsonb) -> jsonb`: 334, + `jsonb_array_elements_text(input: jsonb) -> string`: 336, + `jsonb_array_length(json: jsonb) -> int`: 1267, + `jsonb_build_array(anyelement...) -> jsonb`: 1257, + `jsonb_build_object(anyelement...) -> jsonb`: 1259, + `jsonb_each(input: jsonb) -> tuple{string AS key, jsonb AS value}`: 340, + `jsonb_each_text(input: jsonb) -> tuple{string AS key, string AS value}`: 342, + `jsonb_exists_any(json: jsonb, array: string[]) -> bool`: 1268, + `jsonb_extract_path(jsonb, string...) -> jsonb`: 1242, + `jsonb_extract_path_text(jsonb, string...) -> string`: 1244, + `jsonb_in(input: anyelement) -> jsonb`: 1861, + `jsonb_insert(target: jsonb, path: string[], new_val: jsonb) -> jsonb`: 1249, + `jsonb_insert(target: jsonb, path: string[], new_val: jsonb, insert_after: bool) -> jsonb`: 1250, + `jsonb_object(texts: string[]) -> jsonb`: 1262, + `jsonb_object(keys: string[], values: string[]) -> jsonb`: 1263, + `jsonb_object_agg(arg1: string, arg2: anyelement) -> jsonb`: 253, + `jsonb_object_keys(input: jsonb) -> string`: 338, + `jsonb_out(jsonb: jsonb) -> bytes`: 1860, + `jsonb_populate_record(base: anyelement, from_json: jsonb) -> anyelement`: 344, + `jsonb_populate_recordset(base: anyelement, from_json: jsonb) -> anyelement`: 346, + `jsonb_pretty(val: jsonb) -> string`: 1251, + `jsonb_recv(input: anyelement) -> jsonb`: 1859, + `jsonb_send(jsonb: jsonb) -> bytes`: 1858, + `jsonb_set(val: jsonb, path: string[], to: jsonb) -> jsonb`: 1247, + `jsonb_set(val: jsonb, path: string[], to: jsonb, create_missing: bool) -> jsonb`: 1248, + `jsonb_strip_nulls(from_json: jsonb) -> jsonb`: 1265, + `jsonb_to_record(input: jsonb) -> tuple`: 468, + `jsonb_to_recordset(input: jsonb) -> tuple`: 469, + `jsonb_typeof(val: jsonb) -> string`: 1253, + `lag(val: bool) -> bool`: 1561, + `lag(val: box2d) -> box2d`: 1562, + `lag(val: int) -> int`: 1563, + `lag(val: float) -> float`: 1564, + `lag(val: decimal) -> decimal`: 1565, + `lag(val: date) -> date`: 1566, + `lag(val: timestamp) -> timestamp`: 1567, + `lag(val: interval) -> interval`: 1568, + `lag(val: geography) -> geography`: 1569, + `lag(val: geometry) -> geometry`: 1570, + `lag(val: string) -> string`: 1571, + `lag(val: bytes) -> bytes`: 1572, + `lag(val: timestamptz) -> timestamptz`: 1573, + `lag(val: oid) -> oid`: 1574, + `lag(val: uuid) -> uuid`: 1575, + `lag(val: inet) -> inet`: 1576, + `lag(val: time) -> time`: 1577, + `lag(val: timetz) -> timetz`: 1578, + `lag(val: jsonb) -> jsonb`: 1579, + `lag(val: varbit) -> varbit`: 1580, + `lag(val: bool, n: int) -> bool`: 1581, + `lag(val: box2d, n: int) -> box2d`: 1582, + `lag(val: int, n: int) -> int`: 1583, + `lag(val: float, n: int) -> float`: 1584, + `lag(val: decimal, n: int) -> decimal`: 1585, + `lag(val: date, n: int) -> date`: 1586, + `lag(val: timestamp, n: int) -> timestamp`: 1587, + `lag(val: interval, n: int) -> interval`: 1588, + `lag(val: geography, n: int) -> geography`: 1589, + `lag(val: geometry, n: int) -> geometry`: 1590, + `lag(val: string, n: int) -> string`: 1591, + `lag(val: bytes, n: int) -> bytes`: 1592, + `lag(val: timestamptz, n: int) -> timestamptz`: 1593, + `lag(val: oid, n: int) -> oid`: 1594, + `lag(val: uuid, n: int) -> uuid`: 1595, + `lag(val: inet, n: int) -> inet`: 1596, + `lag(val: time, n: int) -> time`: 1597, + `lag(val: timetz, n: int) -> timetz`: 1598, + `lag(val: jsonb, n: int) -> jsonb`: 1599, + `lag(val: varbit, n: int) -> varbit`: 1600, + `lag(val: bool, n: int, default: bool) -> bool`: 1601, + `lag(val: box2d, n: int, default: box2d) -> box2d`: 1602, + `lag(val: int, n: int, default: int) -> int`: 1603, + `lag(val: float, n: int, default: float) -> float`: 1604, + `lag(val: decimal, n: int, default: decimal) -> decimal`: 1605, + `lag(val: date, n: int, default: date) -> date`: 1606, + `lag(val: timestamp, n: int, default: timestamp) -> timestamp`: 1607, + `lag(val: interval, n: int, default: interval) -> interval`: 1608, + `lag(val: geography, n: int, default: geography) -> geography`: 1609, + `lag(val: geometry, n: int, default: geometry) -> geometry`: 1610, + `lag(val: string, n: int, default: string) -> string`: 1611, + `lag(val: bytes, n: int, default: bytes) -> bytes`: 1612, + `lag(val: timestamptz, n: int, default: timestamptz) -> timestamptz`: 1613, + `lag(val: oid, n: int, default: oid) -> oid`: 1614, + `lag(val: uuid, n: int, default: uuid) -> uuid`: 1615, + `lag(val: inet, n: int, default: inet) -> inet`: 1616, + `lag(val: time, n: int, default: time) -> time`: 1617, + `lag(val: timetz, n: int, default: timetz) -> timetz`: 1618, + `lag(val: jsonb, n: int, default: jsonb) -> jsonb`: 1619, + `lag(val: varbit, n: int, default: varbit) -> varbit`: 1620, + `last_value(val: bool) -> bool`: 1701, + `last_value(val: box2d) -> box2d`: 1702, + `last_value(val: int) -> int`: 1703, + `last_value(val: float) -> float`: 1704, + `last_value(val: decimal) -> decimal`: 1705, + `last_value(val: date) -> date`: 1706, + `last_value(val: timestamp) -> timestamp`: 1707, + `last_value(val: interval) -> interval`: 1708, + `last_value(val: geography) -> geography`: 1709, + `last_value(val: geometry) -> geometry`: 1710, + `last_value(val: string) -> string`: 1711, + `last_value(val: bytes) -> bytes`: 1712, + `last_value(val: timestamptz) -> timestamptz`: 1713, + `last_value(val: oid) -> oid`: 1714, + `last_value(val: uuid) -> uuid`: 1715, + `last_value(val: inet) -> inet`: 1716, + `last_value(val: time) -> time`: 1717, + `last_value(val: timetz) -> timetz`: 1718, + `last_value(val: jsonb) -> jsonb`: 1719, + `last_value(val: varbit) -> varbit`: 1720, + `lastval() -> int`: 974, + `lead(val: bool) -> bool`: 1621, + `lead(val: box2d) -> box2d`: 1622, + `lead(val: int) -> int`: 1623, + `lead(val: float) -> float`: 1624, + `lead(val: decimal) -> decimal`: 1625, + `lead(val: date) -> date`: 1626, + `lead(val: timestamp) -> timestamp`: 1627, + `lead(val: interval) -> interval`: 1628, + `lead(val: geography) -> geography`: 1629, + `lead(val: geometry) -> geometry`: 1630, + `lead(val: string) -> string`: 1631, + `lead(val: bytes) -> bytes`: 1632, + `lead(val: timestamptz) -> timestamptz`: 1633, + `lead(val: oid) -> oid`: 1634, + `lead(val: uuid) -> uuid`: 1635, + `lead(val: inet) -> inet`: 1636, + `lead(val: time) -> time`: 1637, + `lead(val: timetz) -> timetz`: 1638, + `lead(val: jsonb) -> jsonb`: 1639, + `lead(val: varbit) -> varbit`: 1640, + `lead(val: bool, n: int) -> bool`: 1641, + `lead(val: box2d, n: int) -> box2d`: 1642, + `lead(val: int, n: int) -> int`: 1643, + `lead(val: float, n: int) -> float`: 1644, + `lead(val: decimal, n: int) -> decimal`: 1645, + `lead(val: date, n: int) -> date`: 1646, + `lead(val: timestamp, n: int) -> timestamp`: 1647, + `lead(val: interval, n: int) -> interval`: 1648, + `lead(val: geography, n: int) -> geography`: 1649, + `lead(val: geometry, n: int) -> geometry`: 1650, + `lead(val: string, n: int) -> string`: 1651, + `lead(val: bytes, n: int) -> bytes`: 1652, + `lead(val: timestamptz, n: int) -> timestamptz`: 1653, + `lead(val: oid, n: int) -> oid`: 1654, + `lead(val: uuid, n: int) -> uuid`: 1655, + `lead(val: inet, n: int) -> inet`: 1656, + `lead(val: time, n: int) -> time`: 1657, + `lead(val: timetz, n: int) -> timetz`: 1658, + `lead(val: jsonb, n: int) -> jsonb`: 1659, + `lead(val: varbit, n: int) -> varbit`: 1660, + `lead(val: bool, n: int, default: bool) -> bool`: 1661, + `lead(val: box2d, n: int, default: box2d) -> box2d`: 1662, + `lead(val: int, n: int, default: int) -> int`: 1663, + `lead(val: float, n: int, default: float) -> float`: 1664, + `lead(val: decimal, n: int, default: decimal) -> decimal`: 1665, + `lead(val: date, n: int, default: date) -> date`: 1666, + `lead(val: timestamp, n: int, default: timestamp) -> timestamp`: 1667, + `lead(val: interval, n: int, default: interval) -> interval`: 1668, + `lead(val: geography, n: int, default: geography) -> geography`: 1669, + `lead(val: geometry, n: int, default: geometry) -> geometry`: 1670, + `lead(val: string, n: int, default: string) -> string`: 1671, + `lead(val: bytes, n: int, default: bytes) -> bytes`: 1672, + `lead(val: timestamptz, n: int, default: timestamptz) -> timestamptz`: 1673, + `lead(val: oid, n: int, default: oid) -> oid`: 1674, + `lead(val: uuid, n: int, default: uuid) -> uuid`: 1675, + `lead(val: inet, n: int, default: inet) -> inet`: 1676, + `lead(val: time, n: int, default: time) -> time`: 1677, + `lead(val: timetz, n: int, default: timetz) -> timetz`: 1678, + `lead(val: jsonb, n: int, default: jsonb) -> jsonb`: 1679, + `lead(val: varbit, n: int, default: varbit) -> varbit`: 1680, + `least(anyelement...) -> anyelement`: 982, + `left(input: bytes, return_set: int) -> bytes`: 963, + `left(input: string, return_set: int) -> string`: 964, + `length(val: string) -> int`: 814, + `length(val: bytes) -> int`: 815, + `length(val: varbit) -> int`: 816, + `levenshtein(source: string, target: string) -> int`: 1238, + `levenshtein(source: string, target: string, ins_cost: int, del_cost: int, sub_cost: int) -> int`: 1239, + `like_escape(unescaped: string, pattern: string, escape: string) -> bool`: 947, + `ln(val: float) -> float`: 776, + `ln(val: decimal) -> decimal`: 777, + `localtime() -> timetz`: 1023, + `localtime() -> time`: 1024, + `localtime(precision: int) -> timetz`: 1025, + `localtime(precision: int) -> time`: 1026, + `localtimestamp(precision: int) -> timestamptz`: 1017, + `localtimestamp(precision: int) -> timestamp`: 1018, + `localtimestamp(precision: int) -> date`: 1019, + `localtimestamp() -> timestamptz`: 1020, + `localtimestamp() -> timestamp`: 1021, + `localtimestamp() -> date`: 1022, + `log(val: float) -> float`: 778, + `log(b: float, x: float) -> float`: 779, + `log(val: decimal) -> decimal`: 780, + `log(b: decimal, x: decimal) -> decimal`: 781, + `lower(val: string) -> string`: 827, + `lpad(string: string, length: int) -> string`: 929, + `lpad(string: string, length: int, fill: string) -> string`: 930, + `ltrim(input: string, trim_chars: string) -> string`: 935, + `ltrim(val: string) -> string`: 936, + `masklen(val: inet) -> int`: 878, + `max(arg1: collatedstring{*}) -> anyelement`: 168, + `max(arg1: anyenum) -> anyelement`: 169, + `max(arg1: bool) -> anyelement`: 170, + `max(arg1: box2d) -> anyelement`: 171, + `max(arg1: int) -> anyelement`: 172, + `max(arg1: float) -> anyelement`: 173, + `max(arg1: decimal) -> anyelement`: 174, + `max(arg1: date) -> anyelement`: 175, + `max(arg1: timestamp) -> anyelement`: 176, + `max(arg1: interval) -> anyelement`: 177, + `max(arg1: geography) -> anyelement`: 178, + `max(arg1: geometry) -> anyelement`: 179, + `max(arg1: string) -> anyelement`: 180, + `max(arg1: bytes) -> anyelement`: 181, + `max(arg1: timestamptz) -> anyelement`: 182, + `max(arg1: oid) -> anyelement`: 183, + `max(arg1: uuid) -> anyelement`: 184, + `max(arg1: inet) -> anyelement`: 185, + `max(arg1: time) -> anyelement`: 186, + `max(arg1: timetz) -> anyelement`: 187, + `max(arg1: jsonb) -> anyelement`: 188, + `max(arg1: varbit) -> anyelement`: 189, + `md5(string...) -> string`: 896, + `md5(bytes...) -> string`: 897, + `min(arg1: collatedstring{*}) -> anyelement`: 190, + `min(arg1: anyenum) -> anyelement`: 191, + `min(arg1: bool) -> anyelement`: 192, + `min(arg1: box2d) -> anyelement`: 193, + `min(arg1: int) -> anyelement`: 194, + `min(arg1: float) -> anyelement`: 195, + `min(arg1: decimal) -> anyelement`: 196, + `min(arg1: date) -> anyelement`: 197, + `min(arg1: timestamp) -> anyelement`: 198, + `min(arg1: interval) -> anyelement`: 199, + `min(arg1: geography) -> anyelement`: 200, + `min(arg1: geometry) -> anyelement`: 201, + `min(arg1: string) -> anyelement`: 202, + `min(arg1: bytes) -> anyelement`: 203, + `min(arg1: timestamptz) -> anyelement`: 204, + `min(arg1: oid) -> anyelement`: 205, + `min(arg1: uuid) -> anyelement`: 206, + `min(arg1: inet) -> anyelement`: 207, + `min(arg1: time) -> anyelement`: 208, + `min(arg1: timetz) -> anyelement`: 209, + `min(arg1: jsonb) -> anyelement`: 210, + `min(arg1: varbit) -> anyelement`: 211, + `mod(x: float, y: float) -> float`: 782, + `mod(x: decimal, y: decimal) -> decimal`: 783, + `mod(x: int, y: int) -> int`: 784, + `namein(input: anyelement) -> name`: 1985, + `nameout(name: name) -> bytes`: 1984, + `namerecv(input: anyelement) -> name`: 1983, + `namesend(name: name) -> bytes`: 1982, + `netmask(val: inet) -> inet`: 879, + `nextval(sequence_name: string) -> int`: 970, + `nextval(sequence_name: regclass) -> int`: 971, + `not_ilike_escape(unescaped: string, pattern: string, escape: string) -> bool`: 950, + `not_like_escape(unescaped: string, pattern: string, escape: string) -> bool`: 948, + `not_similar_to_escape(unescaped: string, pattern: string, escape: string) -> bool`: 956, + `now() -> timestamptz`: 1001, + `now() -> timestamp`: 1002, + `now() -> date`: 1003, + `nth_value(val: bool, n: int) -> bool`: 1721, + `nth_value(val: box2d, n: int) -> box2d`: 1722, + `nth_value(val: int, n: int) -> int`: 1723, + `nth_value(val: float, n: int) -> float`: 1724, + `nth_value(val: decimal, n: int) -> decimal`: 1725, + `nth_value(val: date, n: int) -> date`: 1726, + `nth_value(val: timestamp, n: int) -> timestamp`: 1727, + `nth_value(val: interval, n: int) -> interval`: 1728, + `nth_value(val: geography, n: int) -> geography`: 1729, + `nth_value(val: geometry, n: int) -> geometry`: 1730, + `nth_value(val: string, n: int) -> string`: 1731, + `nth_value(val: bytes, n: int) -> bytes`: 1732, + `nth_value(val: timestamptz, n: int) -> timestamptz`: 1733, + `nth_value(val: oid, n: int) -> oid`: 1734, + `nth_value(val: uuid, n: int) -> uuid`: 1735, + `nth_value(val: inet, n: int) -> inet`: 1736, + `nth_value(val: time, n: int) -> time`: 1737, + `nth_value(val: timetz, n: int) -> timetz`: 1738, + `nth_value(val: jsonb, n: int) -> jsonb`: 1739, + `nth_value(val: varbit, n: int) -> varbit`: 1740, + `ntile(n: int) -> int`: 1560, + `num_nonnulls(anyelement...) -> int`: 1359, + `num_nulls(anyelement...) -> int`: 1358, + `numeric_in(input: anyelement) -> decimal`: 1969, + `numeric_out(decimal: decimal) -> bytes`: 1968, + `numeric_recv(input: anyelement) -> decimal`: 1967, + `numeric_send(decimal: decimal) -> bytes`: 1966, + `obj_description(object_oid: oid) -> string`: 1423, + `obj_description(object_oid: oid, catalog_name: string) -> string`: 1424, + `octet_length(val: string) -> int`: 824, + `octet_length(val: bytes) -> int`: 825, + `octet_length(val: varbit) -> int`: 826, + `oid(int: int) -> oid`: 1425, + `oidin(input: anyelement) -> oid`: 1905, + `oidout(oid: oid) -> bytes`: 1904, + `oidrecv(input: anyelement) -> oid`: 1903, + `oidsend(oid: oid) -> bytes`: 1902, + `oidvectorin(input: anyelement) -> oidvector`: 1965, + `oidvectorout(oidvector: oidvector) -> bytes`: 1964, + `oidvectorrecv(input: anyelement) -> oidvector`: 1963, + `oidvectorsend(oidvector: oidvector) -> bytes`: 1962, + `overlaps(s1: timestamp, e1: timestamp, s1: timestamp, e2: timestamp) -> bool`: 1390, + `overlaps(s1: timestamp, e1: interval, s1: timestamp, e2: interval) -> bool`: 1391, + `overlaps(s1: timetz, e1: timetz, s1: timetz, e2: timetz) -> bool`: 1392, + `overlaps(s1: timetz, e1: interval, s1: timetz, e2: interval) -> bool`: 1393, + `overlaps(s1: time, e1: time, s1: time, e2: time) -> bool`: 1394, + `overlaps(s1: time, e1: interval, s1: time, e2: interval) -> bool`: 1395, + `overlaps(s1: timestamptz, e1: timestamptz, s1: timestamptz, e2: timestamptz) -> bool`: 1396, + `overlaps(s1: timestamptz, e1: interval, s1: timestamptz, e2: interval) -> bool`: 1397, + `overlaps(s1: date, e1: date, s1: date, e2: date) -> bool`: 1398, + `overlaps(s1: date, e1: interval, s1: date, e2: interval) -> bool`: 1399, + `overlay(input: string, overlay_val: string, start_pos: int) -> string`: 927, + `overlay(input: string, overlay_val: string, start_pos: int, end_pos: int) -> string`: 928, + `parse_date(val: string) -> date`: 1066, + `parse_date(string: string, datestyle: string) -> date`: 1067, + `parse_interval(val: string) -> interval`: 1070, + `parse_interval(string: string, style: string) -> interval`: 1071, + `parse_time(val: string) -> time`: 1068, + `parse_time(string: string, timestyle: string) -> time`: 1069, + `parse_timestamp(val: string) -> timestamp`: 1064, + `parse_timestamp(string: string, datestyle: string) -> timestamp`: 1065, + `parse_timetz(val: string) -> timetz`: 1072, + `parse_timetz(string: string, timestyle: string) -> timetz`: 1073, + `percent_rank() -> float`: 1558, + `percentile_cont(arg1: float) -> float`: 303, + `percentile_cont(arg1: float) -> interval`: 304, + `percentile_cont(arg1: float[]) -> float[]`: 305, + `percentile_cont(arg1: float[]) -> interval[]`: 306, + `percentile_cont_impl(arg1: float, arg2: float) -> float`: 307, + `percentile_cont_impl(arg1: float, arg2: interval) -> interval`: 308, + `percentile_cont_impl(arg1: float[], arg2: float) -> float[]`: 309, + `percentile_cont_impl(arg1: float[], arg2: interval) -> interval[]`: 310, + `percentile_disc(arg1: float) -> anyelement`: 261, + `percentile_disc(arg1: float[]) -> anyelement`: 262, + `percentile_disc_impl(arg1: float, arg2: bool) -> bool`: 263, + `percentile_disc_impl(arg1: float, arg2: box2d) -> box2d`: 264, + `percentile_disc_impl(arg1: float, arg2: int) -> int`: 265, + `percentile_disc_impl(arg1: float, arg2: float) -> float`: 266, + `percentile_disc_impl(arg1: float, arg2: decimal) -> decimal`: 267, + `percentile_disc_impl(arg1: float, arg2: date) -> date`: 268, + `percentile_disc_impl(arg1: float, arg2: timestamp) -> timestamp`: 269, + `percentile_disc_impl(arg1: float, arg2: interval) -> interval`: 270, + `percentile_disc_impl(arg1: float, arg2: geography) -> geography`: 271, + `percentile_disc_impl(arg1: float, arg2: geometry) -> geometry`: 272, + `percentile_disc_impl(arg1: float, arg2: string) -> string`: 273, + `percentile_disc_impl(arg1: float, arg2: bytes) -> bytes`: 274, + `percentile_disc_impl(arg1: float, arg2: timestamptz) -> timestamptz`: 275, + `percentile_disc_impl(arg1: float, arg2: oid) -> oid`: 276, + `percentile_disc_impl(arg1: float, arg2: uuid) -> uuid`: 277, + `percentile_disc_impl(arg1: float, arg2: inet) -> inet`: 278, + `percentile_disc_impl(arg1: float, arg2: time) -> time`: 279, + `percentile_disc_impl(arg1: float, arg2: timetz) -> timetz`: 280, + `percentile_disc_impl(arg1: float, arg2: jsonb) -> jsonb`: 281, + `percentile_disc_impl(arg1: float, arg2: varbit) -> varbit`: 282, + `percentile_disc_impl(arg1: float[], arg2: bool) -> bool[]`: 283, + `percentile_disc_impl(arg1: float[], arg2: box2d) -> box2d[]`: 284, + `percentile_disc_impl(arg1: float[], arg2: int) -> int[]`: 285, + `percentile_disc_impl(arg1: float[], arg2: float) -> float[]`: 286, + `percentile_disc_impl(arg1: float[], arg2: decimal) -> decimal[]`: 287, + `percentile_disc_impl(arg1: float[], arg2: date) -> date[]`: 288, + `percentile_disc_impl(arg1: float[], arg2: timestamp) -> timestamp[]`: 289, + `percentile_disc_impl(arg1: float[], arg2: interval) -> interval[]`: 290, + `percentile_disc_impl(arg1: float[], arg2: geography) -> geography[]`: 291, + `percentile_disc_impl(arg1: float[], arg2: geometry) -> geometry[]`: 292, + `percentile_disc_impl(arg1: float[], arg2: string) -> string[]`: 293, + `percentile_disc_impl(arg1: float[], arg2: bytes) -> bytes[]`: 294, + `percentile_disc_impl(arg1: float[], arg2: timestamptz) -> timestamptz[]`: 295, + `percentile_disc_impl(arg1: float[], arg2: oid) -> oid[]`: 296, + `percentile_disc_impl(arg1: float[], arg2: uuid) -> uuid[]`: 297, + `percentile_disc_impl(arg1: float[], arg2: inet) -> inet[]`: 298, + `percentile_disc_impl(arg1: float[], arg2: time) -> time[]`: 299, + `percentile_disc_impl(arg1: float[], arg2: timetz) -> timetz[]`: 300, + `percentile_disc_impl(arg1: float[], arg2: jsonb) -> jsonb[]`: 301, + `percentile_disc_impl(arg1: float[], arg2: varbit) -> varbit[]`: 302, + `pg_advisory_unlock(int: int) -> bool`: 1428, + `pg_backend_pid() -> int`: 1400, + `pg_client_encoding() -> string`: 1429, + `pg_collation_for(str: anyelement) -> string`: 1418, + `pg_column_is_updatable(reloid: oid, attnum: int2, include_triggers: bool) -> bool`: 1434, + `pg_column_size(anyelement...) -> int`: 1529, + `pg_encoding_to_char(encoding_id: int) -> string`: 1401, + `pg_function_is_visible(oid: oid) -> bool`: 1430, + `pg_get_constraintdef(constraint_oid: oid, pretty_bool: bool) -> string`: 1405, + `pg_get_constraintdef(constraint_oid: oid) -> string`: 1406, + `pg_get_expr(pg_node_tree: string, relation_oid: oid) -> string`: 1403, + `pg_get_expr(pg_node_tree: string, relation_oid: oid, pretty_bool: bool) -> string`: 1404, + `pg_get_functiondef(func_oid: oid) -> string`: 2035, + `pg_get_function_identity_arguments(func_oid: oid) -> string`: 1409, + `pg_get_function_result(func_oid: oid) -> string`: 1408, + `pg_get_indexdef(index_oid: oid) -> string`: 1410, + `pg_get_indexdef(index_oid: oid, column_no: int, pretty_bool: bool) -> string`: 1411, + `pg_get_keywords() -> tuple{string AS word, string AS catcode, string AS catdesc}`: 322, + `pg_get_partkeydef(oid: oid) -> string`: 1407, + `pg_get_serial_sequence(table_name: string, column_name: string) -> string`: 1414, + `pg_get_userbyid(role_oid: oid) -> string`: 1419, + `pg_get_viewdef(view_oid: oid) -> string`: 1412, + `pg_get_viewdef(view_oid: oid, pretty_bool: bool) -> string`: 1413, + `pg_has_role(role: string, privilege: string) -> bool`: 1516, + `pg_has_role(role: oid, privilege: string) -> bool`: 1517, + `pg_has_role(user: string, role: string, privilege: string) -> bool`: 1518, + `pg_has_role(user: string, role: oid, privilege: string) -> bool`: 1519, + `pg_has_role(user: oid, role: string, privilege: string) -> bool`: 1520, + `pg_has_role(user: oid, role: oid, privilege: string) -> bool`: 1521, + `pg_is_in_recovery() -> bool`: 1436, + `pg_is_other_temp_schema(oid: oid) -> bool`: 1416, + `pg_is_xlog_replay_paused() -> bool`: 1437, + `pg_my_temp_schema() -> oid`: 1415, + `pg_options_to_table(options: string[]) -> tuple{string AS option_name, string AS option_value}`: 323, + `pg_relation_is_updatable(reloid: oid, include_triggers: bool) -> int4`: 1433, + `pg_sequence_parameters(sequence_oid: oid) -> string`: 1420, + `pg_sleep(seconds: float) -> bool`: 1435, + `pg_table_is_visible(oid: oid) -> bool`: 1431, + `pg_try_advisory_lock(int: int) -> bool`: 1427, + `pg_type_is_visible(oid: oid) -> bool`: 1432, + `pg_typeof(val: anyelement) -> string`: 1417, + `pi() -> float`: 785, + `postgis_addbbox(geometry: geometry) -> geometry`: 357, + `postgis_dropbbox(geometry: geometry) -> geometry`: 358, + `postgis_extensions_upgrade() -> string`: 361, + `postgis_full_version() -> string`: 362, + `postgis_geos_version() -> string`: 363, + `postgis_getbbox(geometry: geometry) -> box2d`: 360, + `postgis_hasbbox(geometry: geometry) -> bool`: 359, + `postgis_lib_build_date() -> string`: 365, + `postgis_lib_version() -> string`: 366, + `postgis_liblwgeom_version() -> string`: 367, + `postgis_libxml_version() -> string`: 364, + `postgis_proj_version() -> string`: 368, + `postgis_scripts_build_date() -> string`: 369, + `postgis_scripts_installed() -> string`: 370, + `postgis_scripts_released() -> string`: 371, + `postgis_version() -> string`: 372, + `postgis_wagyu_version() -> string`: 373, + `pow(x: float, y: float) -> float`: 786, + `pow(x: decimal, y: decimal) -> decimal`: 787, + `pow(x: int, y: int) -> int`: 788, + `power(x: float, y: float) -> float`: 789, + `power(x: decimal, y: decimal) -> decimal`: 790, + `power(x: int, y: int) -> int`: 791, + `prettify_statement(val: string) -> string`: 830, + `prettify_statement(statement: string, line_width: int, align_mode: int, case_mode: int) -> string`: 831, + `quote_ident(val: string) -> string`: 958, + `quote_literal(val: string) -> string`: 959, + `quote_literal(val: anyelement) -> string`: 960, + `quote_nullable(val: string) -> string`: 961, + `quote_nullable(val: anyelement) -> string`: 962, + `radians(val: float) -> float`: 792, + `random() -> float`: 967, + `rank() -> int`: 1556, + `record_in(input: anyelement) -> anyelement`: 1933, + `record_out(tuple: tuple) -> bytes`: 1932, + `record_recv(input: anyelement) -> anyelement`: 1931, + `record_send(tuple: tuple) -> bytes`: 1930, + `regclassin(input: anyelement) -> regclass`: 1893, + `regclassout(regclass: regclass) -> bytes`: 1892, + `regclassrecv(input: anyelement) -> regclass`: 1891, + `regclasssend(regclass: regclass) -> bytes`: 1890, + `regexp_extract(input: string, regex: string) -> string`: 942, + `regexp_replace(input: string, regex: string, replace: string) -> string`: 943, + `regexp_replace(input: string, regex: string, replace: string, flags: string) -> string`: 944, + `regexp_split_to_array(string: string, pattern: string) -> string[]`: 945, + `regexp_split_to_array(string: string, pattern: string, flags: string) -> string[]`: 946, + `regexp_split_to_table(string: string, pattern: string) -> string`: 324, + `regexp_split_to_table(string: string, pattern: string, flags: string) -> string`: 325, + `regnamespacein(input: anyelement) -> regnamespace`: 1989, + `regnamespaceout(regnamespace: regnamespace) -> bytes`: 1988, + `regnamespacerecv(input: anyelement) -> regnamespace`: 1987, + `regnamespacesend(regnamespace: regnamespace) -> bytes`: 1986, + `regprocedurein(input: anyelement) -> regprocedure`: 1925, + `regprocedureout(regprocedure: regprocedure) -> bytes`: 1924, + `regprocedurerecv(input: anyelement) -> regprocedure`: 1923, + `regproceduresend(regprocedure: regprocedure) -> bytes`: 1922, + `regprocin(input: anyelement) -> regproc`: 1937, + `regprocout(regproc: regproc) -> bytes`: 1936, + `regprocrecv(input: anyelement) -> regproc`: 1935, + `regprocsend(regproc: regproc) -> bytes`: 1934, + `regr_avgx(arg1: float, arg2: float) -> float`: 84, + `regr_avgx(arg1: int, arg2: int) -> float`: 85, + `regr_avgx(arg1: decimal, arg2: decimal) -> float`: 86, + `regr_avgx(arg1: float, arg2: int) -> float`: 87, + `regr_avgx(arg1: float, arg2: decimal) -> float`: 88, + `regr_avgx(arg1: int, arg2: float) -> float`: 89, + `regr_avgx(arg1: int, arg2: decimal) -> float`: 90, + `regr_avgx(arg1: decimal, arg2: float) -> float`: 91, + `regr_avgx(arg1: decimal, arg2: int) -> float`: 92, + `regr_avgy(arg1: float, arg2: float) -> float`: 93, + `regr_avgy(arg1: int, arg2: int) -> float`: 94, + `regr_avgy(arg1: decimal, arg2: decimal) -> float`: 95, + `regr_avgy(arg1: float, arg2: int) -> float`: 96, + `regr_avgy(arg1: float, arg2: decimal) -> float`: 97, + `regr_avgy(arg1: int, arg2: float) -> float`: 98, + `regr_avgy(arg1: int, arg2: decimal) -> float`: 99, + `regr_avgy(arg1: decimal, arg2: float) -> float`: 100, + `regr_avgy(arg1: decimal, arg2: int) -> float`: 101, + `regr_count(arg1: float, arg2: float) -> int`: 156, + `regr_count(arg1: int, arg2: int) -> int`: 157, + `regr_count(arg1: decimal, arg2: decimal) -> int`: 158, + `regr_count(arg1: float, arg2: int) -> int`: 159, + `regr_count(arg1: float, arg2: decimal) -> int`: 160, + `regr_count(arg1: int, arg2: float) -> int`: 161, + `regr_count(arg1: int, arg2: decimal) -> int`: 162, + `regr_count(arg1: decimal, arg2: float) -> int`: 163, + `regr_count(arg1: decimal, arg2: int) -> int`: 164, + `regr_intercept(arg1: float, arg2: float) -> float`: 102, + `regr_intercept(arg1: int, arg2: int) -> float`: 103, + `regr_intercept(arg1: decimal, arg2: decimal) -> float`: 104, + `regr_intercept(arg1: float, arg2: int) -> float`: 105, + `regr_intercept(arg1: float, arg2: decimal) -> float`: 106, + `regr_intercept(arg1: int, arg2: float) -> float`: 107, + `regr_intercept(arg1: int, arg2: decimal) -> float`: 108, + `regr_intercept(arg1: decimal, arg2: float) -> float`: 109, + `regr_intercept(arg1: decimal, arg2: int) -> float`: 110, + `regr_r2(arg1: float, arg2: float) -> float`: 111, + `regr_r2(arg1: int, arg2: int) -> float`: 112, + `regr_r2(arg1: decimal, arg2: decimal) -> float`: 113, + `regr_r2(arg1: float, arg2: int) -> float`: 114, + `regr_r2(arg1: float, arg2: decimal) -> float`: 115, + `regr_r2(arg1: int, arg2: float) -> float`: 116, + `regr_r2(arg1: int, arg2: decimal) -> float`: 117, + `regr_r2(arg1: decimal, arg2: float) -> float`: 118, + `regr_r2(arg1: decimal, arg2: int) -> float`: 119, + `regr_slope(arg1: float, arg2: float) -> float`: 120, + `regr_slope(arg1: int, arg2: int) -> float`: 121, + `regr_slope(arg1: decimal, arg2: decimal) -> float`: 122, + `regr_slope(arg1: float, arg2: int) -> float`: 123, + `regr_slope(arg1: float, arg2: decimal) -> float`: 124, + `regr_slope(arg1: int, arg2: float) -> float`: 125, + `regr_slope(arg1: int, arg2: decimal) -> float`: 126, + `regr_slope(arg1: decimal, arg2: float) -> float`: 127, + `regr_slope(arg1: decimal, arg2: int) -> float`: 128, + `regr_sxx(arg1: float, arg2: float) -> float`: 129, + `regr_sxx(arg1: int, arg2: int) -> float`: 130, + `regr_sxx(arg1: decimal, arg2: decimal) -> float`: 131, + `regr_sxx(arg1: float, arg2: int) -> float`: 132, + `regr_sxx(arg1: float, arg2: decimal) -> float`: 133, + `regr_sxx(arg1: int, arg2: float) -> float`: 134, + `regr_sxx(arg1: int, arg2: decimal) -> float`: 135, + `regr_sxx(arg1: decimal, arg2: float) -> float`: 136, + `regr_sxx(arg1: decimal, arg2: int) -> float`: 137, + `regr_sxy(arg1: float, arg2: float) -> float`: 138, + `regr_sxy(arg1: int, arg2: int) -> float`: 139, + `regr_sxy(arg1: decimal, arg2: decimal) -> float`: 140, + `regr_sxy(arg1: float, arg2: int) -> float`: 141, + `regr_sxy(arg1: float, arg2: decimal) -> float`: 142, + `regr_sxy(arg1: int, arg2: float) -> float`: 143, + `regr_sxy(arg1: int, arg2: decimal) -> float`: 144, + `regr_sxy(arg1: decimal, arg2: float) -> float`: 145, + `regr_sxy(arg1: decimal, arg2: int) -> float`: 146, + `regr_syy(arg1: float, arg2: float) -> float`: 147, + `regr_syy(arg1: int, arg2: int) -> float`: 148, + `regr_syy(arg1: decimal, arg2: decimal) -> float`: 149, + `regr_syy(arg1: float, arg2: int) -> float`: 150, + `regr_syy(arg1: float, arg2: decimal) -> float`: 151, + `regr_syy(arg1: int, arg2: float) -> float`: 152, + `regr_syy(arg1: int, arg2: decimal) -> float`: 153, + `regr_syy(arg1: decimal, arg2: float) -> float`: 154, + `regr_syy(arg1: decimal, arg2: int) -> float`: 155, + `regrolein(input: anyelement) -> regrole`: 1853, + `regroleout(regrole: regrole) -> bytes`: 1852, + `regrolerecv(input: anyelement) -> regrole`: 1851, + `regrolesend(regrole: regrole) -> bytes`: 1850, + `regtypein(input: anyelement) -> regtype`: 1917, + `regtypeout(regtype: regtype) -> bytes`: 1916, + `regtyperecv(input: anyelement) -> regtype`: 1915, + `regtypesend(regtype: regtype) -> bytes`: 1914, + `rehome_row() -> string`: 1362, + `repeat(input: string, repeat_counter: int) -> string`: 889, + `replace(input: string, find: string, replace: string) -> string`: 940, + `reverse(val: string) -> string`: 939, + `right(input: bytes, return_set: int) -> bytes`: 965, + `right(input: string, return_set: int) -> string`: 966, + `round(val: float) -> float`: 793, + `round(val: decimal) -> decimal`: 794, + `round(input: float, decimal_accuracy: int) -> float`: 795, + `round(input: decimal, decimal_accuracy: int) -> decimal`: 796, + `row_number() -> int`: 1555, + `row_to_json(row: tuple) -> jsonb`: 1058, + `rpad(string: string, length: int) -> string`: 931, + `rpad(string: string, length: int, fill: string) -> string`: 932, + `rtrim(input: string, trim_chars: string) -> string`: 937, + `rtrim(val: string) -> string`: 938, + `session_user() -> string`: 1289, + `set_bit(bit_string: varbit, index: int, to_set: int) -> varbit`: 855, + `set_bit(byte_string: bytes, index: int, to_set: int) -> bytes`: 856, + `set_byte(byte_string: bytes, index: int, to_set: int) -> bytes`: 857, + `set_config(setting_name: string, new_value: string, is_local: bool) -> string`: 1524, + `set_masklen(val: inet, prefixlen: int) -> inet`: 880, + `setval(sequence_name: string, value: int) -> int`: 975, + `setval(sequence_name: regclass, value: int) -> int`: 976, + `setval(sequence_name: string, value: int, is_called: bool) -> int`: 977, + `setval(sequence_name: regclass, value: int, is_called: bool) -> int`: 978, + `sha1(string...) -> string`: 898, + `sha1(bytes...) -> string`: 899, + `sha224(string...) -> string`: 900, + `sha224(bytes...) -> string`: 901, + `sha256(string...) -> string`: 902, + `sha256(bytes...) -> string`: 903, + `sha384(string...) -> string`: 904, + `sha384(bytes...) -> string`: 905, + `sha512(string...) -> string`: 906, + `sha512(bytes...) -> string`: 907, + `shobj_description(object_oid: oid, catalog_name: string) -> string`: 1426, + `show_trgm(input: string) -> string[]`: 1554, + `sign(val: float) -> float`: 800, + `sign(val: decimal) -> decimal`: 801, + `sign(val: int) -> int`: 802, + `similar_escape(pattern: string) -> string`: 951, + `similar_escape(pattern: string, escape: string) -> string`: 952, + `similar_to_escape(pattern: string) -> string`: 953, + `similar_to_escape(pattern: string, escape: string) -> string`: 954, + `similar_to_escape(unescaped: string, pattern: string, escape: string) -> bool`: 955, + `similarity(left: string, right: string) -> float`: 1553, + `sin(val: float) -> float`: 797, + `sind(val: float) -> float`: 798, + `sinh(val: float) -> float`: 799, + `soundex(source: string) -> string`: 1236, + `split_part(input: string, delimiter: string, return_index_pos: int) -> string`: 888, + `sqrdiff(arg1: int) -> decimal`: 219, + `sqrdiff(arg1: decimal) -> decimal`: 220, + `sqrdiff(arg1: float) -> float`: 221, + `sqrt(val: float) -> float`: 803, + `sqrt(val: decimal) -> decimal`: 804, + `st_addmeasure(geometry: geometry, start: float, end: float) -> geometry`: 568, + `st_addpoint(line_string: geometry, point: geometry, index: int) -> geometry`: 663, + `st_addpoint(line_string: geometry, point: geometry) -> geometry`: 664, + `st_affine(geometry: geometry, a: float, b: float, d: float, e: float, x_off: float, y_off: float) -> geometry`: 652, + `st_affine(geometry: geometry, a: float, b: float, c: float, d: float, e: float, f: float, g: float, h: float, i: float, x_off: float, y_off: float, z_off: float) -> geometry`: 653, + `st_angle(point1: geometry, point2: geometry, point3: geometry, point4: geometry) -> float`: 691, + `st_angle(point1: geometry, point2: geometry, point3: geometry) -> float`: 692, + `st_angle(line1: geometry, line2: geometry) -> float`: 693, + `st_area(geography: geography) -> float`: 1773, + `st_area(geography: geography, use_spheroid: bool) -> float`: 1774, + `st_area(geometry: geometry) -> float`: 1775, + `st_area(geometry_str: string) -> float`: 1776, + `st_area2d(geometry: geometry) -> float`: 555, + `st_asbinary(geometry: geometry) -> bytes`: 474, + `st_asbinary(geography: geography) -> bytes`: 475, + `st_asbinary(geometry: geometry, xdr_or_ndr: string) -> bytes`: 476, + `st_asbinary(geography: geography, xdr_or_ndr: string) -> bytes`: 477, + `st_asencodedpolyline(geometry: geometry) -> string`: 694, + `st_asencodedpolyline(geometry: geometry, precision: int4) -> string`: 695, + `st_asewkb(geometry: geometry) -> bytes`: 478, + `st_asewkb(geography: geography) -> bytes`: 479, + `st_asewkt(geometry: geometry) -> string`: 1777, + `st_asewkt(geometry: geometry, max_decimal_digits: int) -> string`: 1778, + `st_asewkt(geography: geography) -> string`: 1779, + `st_asewkt(geography: geography, max_decimal_digits: int) -> string`: 1780, + `st_asewkt(geometry_str: string) -> string`: 1781, + `st_asewkt(geometry_str: string, max_decimal_digits: int) -> string`: 1782, + `st_asgeojson(row: tuple) -> string`: 1783, + `st_asgeojson(row: tuple, geo_column: string) -> string`: 1784, + `st_asgeojson(row: tuple, geo_column: string, max_decimal_digits: int) -> string`: 1785, + `st_asgeojson(row: tuple, geo_column: string, max_decimal_digits: int, pretty: bool) -> string`: 1786, + `st_asgeojson(geometry: geometry) -> string`: 1787, + `st_asgeojson(geometry: geometry, max_decimal_digits: int) -> string`: 1788, + `st_asgeojson(geometry: geometry, max_decimal_digits: int, options: int) -> string`: 1789, + `st_asgeojson(geography: geography) -> string`: 1790, + `st_asgeojson(geography: geography, max_decimal_digits: int) -> string`: 1791, + `st_asgeojson(geography: geography, max_decimal_digits: int, options: int) -> string`: 1792, + `st_asgeojson(geometry_str: string) -> string`: 1793, + `st_asgeojson(geometry_str: string, max_decimal_digits: int) -> string`: 1794, + `st_asgeojson(geometry_str: string, max_decimal_digits: int, options: int) -> string`: 1795, + `st_ashexewkb(geometry: geometry) -> string`: 482, + `st_ashexewkb(geography: geography) -> string`: 483, + `st_ashexewkb(geometry: geometry, xdr_or_ndr: string) -> string`: 484, + `st_ashexewkb(geography: geography, xdr_or_ndr: string) -> string`: 485, + `st_ashexwkb(geometry: geometry) -> string`: 480, + `st_ashexwkb(geography: geography) -> string`: 481, + `st_askml(geometry: geometry) -> string`: 1796, + `st_askml(geography: geography) -> string`: 1797, + `st_askml(geometry_str: string) -> string`: 1798, + `st_astext(geometry: geometry) -> string`: 1799, + `st_astext(geometry: geometry, max_decimal_digits: int) -> string`: 1800, + `st_astext(geography: geography) -> string`: 1801, + `st_astext(geography: geography, max_decimal_digits: int) -> string`: 1802, + `st_astext(geometry_str: string) -> string`: 1803, + `st_astext(geometry_str: string, max_decimal_digits: int) -> string`: 1804, + `st_astwkb(geometry: geometry, precision_xy: int) -> bytes`: 486, + `st_astwkb(geometry: geometry, precision_xy: int, precision_z: int) -> bytes`: 487, + `st_astwkb(geometry: geometry, precision_xy: int, precision_z: int, precision_m: int) -> bytes`: 488, + `st_azimuth(geometry_a: geometry, geometry_b: geometry) -> float`: 584, + `st_azimuth(geography_a: geography, geography_b: geography) -> float`: 585, + `st_boundary(geometry: geometry) -> geometry`: 628, + `st_box2dfromgeohash(geohash: string, precision: int) -> box2d`: 464, + `st_box2dfromgeohash(geohash: string) -> box2d`: 465, + `st_buffer(geometry: geometry, distance: int) -> geometry`: 1805, + `st_buffer(geometry: geometry, distance: float) -> geometry`: 1806, + `st_buffer(geometry: geometry, distance: decimal) -> geometry`: 1807, + `st_buffer(geometry: geometry, distance: float, quad_segs: int) -> geometry`: 1808, + `st_buffer(geometry: geometry, distance: float, buffer_style_params: string) -> geometry`: 1809, + `st_buffer(geography: geography, distance: float) -> geography`: 1810, + `st_buffer(geography: geography, distance: float, quad_segs: int) -> geography`: 1811, + `st_buffer(geography: geography, distance: float, buffer_style_params: string) -> geography`: 1812, + `st_buffer(geometry_str: string, distance: int) -> geometry`: 1813, + `st_buffer(geometry_str: string, distance: float) -> geometry`: 1814, + `st_buffer(geometry_str: string, distance: decimal) -> geometry`: 1815, + `st_buffer(geometry_str: string, distance: float, quad_segs: int) -> geometry`: 1816, + `st_buffer(geometry_str: string, distance: float, buffer_style_params: string) -> geometry`: 1817, + `st_centroid(geography: geography) -> geography`: 1818, + `st_centroid(geography: geography, use_spheroid: bool) -> geography`: 1819, + `st_centroid(geometry: geometry) -> geometry`: 1820, + `st_centroid(geometry_str: string) -> geometry`: 1821, + `st_clipbybox2d(geometry: geometry, box2d: box2d) -> geometry`: 632, + `st_closestpoint(geometry_a: geometry, geometry_b: geometry) -> geometry`: 639, + `st_collect(arg1: geometry) -> geometry`: 258, + `st_collectionextract(geometry: geometry, type: int) -> geometry`: 573, + `st_collectionhomogenize(geometry: geometry) -> geometry`: 574, + `st_combinebbox(box2d: box2d, geometry: geometry) -> box2d`: 703, + `st_contains(geometry_a: geometry, geometry_b: geometry) -> bool`: 602, + `st_containsproperly(geometry_a: geometry, geometry_b: geometry) -> bool`: 603, + `st_convexhull(geometry: geometry) -> geometry`: 633, + `st_coorddim(geometry: geometry) -> int`: 1743, + `st_coveredby(geometry_a: geometry, geometry_b: geometry) -> bool`: 1822, + `st_coveredby(geography_a: geography, geography_b: geography) -> bool`: 1823, + `st_coveredby(geometry_a_str: string, geometry_b_str: string) -> bool`: 1824, + `st_covers(geometry_a: geometry, geometry_b: geometry) -> bool`: 1825, + `st_covers(geography_a: geography, geography_b: geography) -> bool`: 1826, + `st_covers(geometry_a_str: string, geometry_b_str: string) -> bool`: 1827, + `st_crosses(geometry_a: geometry, geometry_b: geometry) -> bool`: 604, + `st_dfullywithin(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 606, + `st_dfullywithinexclusive(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 720, + `st_difference(geometry_a: geometry, geometry_b: geometry) -> geometry`: 634, + `st_dimension(geometry: geometry) -> int`: 507, + `st_disjoint(geometry_a: geometry, geometry_b: geometry) -> bool`: 605, + `st_distance(geometry_a: geometry, geometry_b: geometry) -> float`: 1828, + `st_distance(geography_a: geography, geography_b: geography) -> float`: 1829, + `st_distance(geography_a: geography, geography_b: geography, use_spheroid: bool) -> float`: 1830, + `st_distance(geometry_a_str: string, geometry_b_str: string) -> float`: 1831, + `st_distancesphere(geometry_a: geometry, geometry_b: geometry) -> float`: 589, + `st_distancespheroid(geometry_a: geometry, geometry_b: geometry) -> float`: 590, + `st_dwithin(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1832, + `st_dwithin(geography_a: geography, geography_b: geography, distance: float) -> bool`: 1833, + `st_dwithin(geography_a: geography, geography_b: geography, distance: float, use_spheroid: bool) -> bool`: 1834, + `st_dwithin(geometry_a_str: string, geometry_b_str: string, distance: float) -> bool`: 1835, + `st_dwithinexclusive(geometry_a: geometry, geometry_b: geometry, distance: float) -> bool`: 1846, + `st_dwithinexclusive(geography_a: geography, geography_b: geography, distance: float) -> bool`: 1847, + `st_dwithinexclusive(geography_a: geography, geography_b: geography, distance: float, use_spheroid: bool) -> bool`: 1848, + `st_dwithinexclusive(geometry_a_str: string, geometry_b_str: string, distance: float) -> bool`: 1849, + `st_endpoint(geometry: geometry) -> geometry`: 511, + `st_envelope(geometry: geometry) -> geometry`: 685, + `st_envelope(box2d: box2d) -> geometry`: 686, + `st_equals(geometry_a: geometry, geometry_b: geometry) -> bool`: 610, + `st_estimatedextent(schema_name: string, table_name: string, geocolumn_name: string, parent_only: bool) -> box2d`: 708, + `st_estimatedextent(schema_name: string, table_name: string, geocolumn_name: string) -> box2d`: 709, + `st_estimatedextent(table_name: string, geocolumn_name: string) -> box2d`: 710, + `st_expand(box2d: box2d, delta: float) -> box2d`: 704, + `st_expand(box2d: box2d, delta_x: float, delta_y: float) -> box2d`: 705, + `st_expand(geometry: geometry, delta: float) -> geometry`: 706, + `st_expand(geometry: geometry, delta_x: float, delta_y: float) -> geometry`: 707, + `st_extent(arg1: geometry) -> box2d`: 255, + `st_exteriorring(geometry: geometry) -> geometry`: 518, + `st_flipcoordinates(geometry: geometry) -> geometry`: 689, + `st_force2d(geometry: geometry) -> geometry`: 526, + `st_force3d(geometry: geometry) -> geometry`: 1750, + `st_force3d(geometry: geometry, defaultZ: float) -> geometry`: 1751, + `st_force3dm(geometry: geometry) -> geometry`: 529, + `st_force3dm(geometry: geometry, defaultM: float) -> geometry`: 530, + `st_force3dz(geometry: geometry) -> geometry`: 527, + `st_force3dz(geometry: geometry, defaultZ: float) -> geometry`: 528, + `st_force4d(geometry: geometry) -> geometry`: 531, + `st_force4d(geometry: geometry, defaultZ: float) -> geometry`: 532, + `st_force4d(geometry: geometry, defaultZ: float, defaultM: float) -> geometry`: 533, + `st_forcecollection(geometry: geometry) -> geometry`: 575, + `st_forcepolygonccw(geometry: geometry) -> geometry`: 535, + `st_forcepolygoncw(geometry: geometry) -> geometry`: 534, + `st_frechetdistance(geometry_a: geometry, geometry_b: geometry) -> float`: 591, + `st_frechetdistance(geometry_a: geometry, geometry_b: geometry, densify_frac: float) -> float`: 592, + `st_generatepoints(geometry: geometry, npoints: int4) -> geometry`: 512, + `st_generatepoints(geometry: geometry, npoints: int4, seed: int4) -> geometry`: 513, + `st_geogfromewkb(val: bytes) -> geography`: 456, + `st_geogfromewkt(val: string) -> geography`: 453, + `st_geogfromgeojson(val: string) -> geography`: 457, + `st_geogfromgeojson(val: jsonb) -> geography`: 458, + `st_geogfromtext(val: string) -> geography`: 1744, + `st_geogfromtext(str: string, srid: int) -> geography`: 1745, + `st_geogfromwkb(val: bytes) -> geography`: 454, + `st_geogfromwkb(bytes: bytes, srid: int) -> geography`: 455, + `st_geographyfromtext(val: string) -> geography`: 451, + `st_geographyfromtext(str: string, srid: int) -> geography`: 452, + `st_geohash(geometry: geometry) -> string`: 491, + `st_geohash(geometry: geometry, precision: int) -> string`: 492, + `st_geohash(geography: geography) -> string`: 493, + `st_geohash(geography: geography, precision: int) -> string`: 494, + `st_geomcollfromtext(val: string) -> geometry`: 395, + `st_geomcollfromtext(str: string, srid: int) -> geometry`: 396, + `st_geomcollfromwkb(val: bytes) -> geometry`: 397, + `st_geomcollfromwkb(wkb: bytes, srid: int) -> geometry`: 398, + `st_geometryfromtext(val: string) -> geometry`: 378, + `st_geometryfromtext(str: string, srid: int) -> geometry`: 379, + `st_geometryn(geometry: geometry, n: int) -> geometry`: 521, + `st_geometrytype(geometry: geometry) -> string`: 567, + `st_geomfromewkb(val: bytes) -> geometry`: 385, + `st_geomfromewkt(val: string) -> geometry`: 380, + `st_geomfromgeohash(geohash: string, precision: int) -> geometry`: 462, + `st_geomfromgeohash(geohash: string) -> geometry`: 463, + `st_geomfromgeojson(val: string) -> geometry`: 386, + `st_geomfromgeojson(val: jsonb) -> geometry`: 387, + `st_geomfromtext(val: string) -> geometry`: 1746, + `st_geomfromtext(str: string, srid: int) -> geometry`: 1747, + `st_geomfromwkb(val: bytes) -> geometry`: 383, + `st_geomfromwkb(bytes: bytes, srid: int) -> geometry`: 384, + `st_hasarc(geometry: geometry) -> bool`: 515, + `st_hausdorffdistance(geometry_a: geometry, geometry_b: geometry) -> float`: 593, + `st_hausdorffdistance(geometry_a: geometry, geometry_b: geometry, densify_frac: float) -> float`: 594, + `st_interiorringn(geometry: geometry, n: int) -> geometry`: 519, + `st_intersection(geometry_a: geometry, geometry_b: geometry) -> geometry`: 1839, + `st_intersection(geography_a: geography, geography_b: geography) -> geography`: 1840, + `st_intersection(geometry_a_str: string, geometry_b_str: string) -> geometry`: 1841, + `st_intersects(geometry_a: geometry, geometry_b: geometry) -> bool`: 1836, + `st_intersects(geography_a: geography, geography_b: geography) -> bool`: 1837, + `st_intersects(geometry_a_str: string, geometry_b_str: string) -> bool`: 1838, + `st_isclosed(geometry: geometry) -> bool`: 579, + `st_iscollection(geometry: geometry) -> bool`: 580, + `st_isempty(geometry: geometry) -> bool`: 581, + `st_ispolygonccw(geometry: geometry) -> bool`: 537, + `st_ispolygoncw(geometry: geometry) -> bool`: 536, + `st_isring(geometry: geometry) -> bool`: 582, + `st_issimple(geometry: geometry) -> bool`: 583, + `st_isvalid(geometry: geometry) -> bool`: 622, + `st_isvalid(geometry: geometry, flags: int) -> bool`: 623, + `st_isvalidreason(geometry: geometry) -> string`: 624, + `st_isvalidreason(geometry: geometry, flags: int) -> string`: 625, + `st_isvalidtrajectory(geometry: geometry) -> bool`: 626, + `st_length(geography: geography) -> float`: 1842, + `st_length(geography: geography, use_spheroid: bool) -> float`: 1843, + `st_length(geometry: geometry) -> float`: 1844, + `st_length(geometry_str: string) -> float`: 1845, + `st_length2d(geometry: geometry) -> float`: 559, + `st_linecrossingdirection(linestring_a: geometry, linestring_b: geometry) -> int`: 737, + `st_linefromencodedpolyline(encoded_polyline: string) -> geometry`: 696, + `st_linefromencodedpolyline(encoded_polyline: string, precision: int4) -> geometry`: 697, + `st_linefrommultipoint(geometry: geometry) -> geometry`: 576, + `st_linefromtext(val: string) -> geometry`: 399, + `st_linefromtext(str: string, srid: int) -> geometry`: 400, + `st_linefromwkb(val: bytes) -> geometry`: 401, + `st_linefromwkb(wkb: bytes, srid: int) -> geometry`: 402, + `st_lineinterpolatepoint(geometry: geometry, fraction: float) -> geometry`: 569, + `st_lineinterpolatepoints(geometry: geometry, fraction: float) -> geometry`: 570, + `st_lineinterpolatepoints(geometry: geometry, fraction: float, repeat: bool) -> geometry`: 571, + `st_linelocatepoint(line: geometry, point: geometry) -> float`: 723, + `st_linemerge(geometry: geometry) -> geometry`: 577, + `st_linestringfromtext(val: string) -> geometry`: 403, + `st_linestringfromtext(str: string, srid: int) -> geometry`: 404, + `st_linestringfromwkb(val: bytes) -> geometry`: 405, + `st_linestringfromwkb(wkb: bytes, srid: int) -> geometry`: 406, + `st_linesubstring(linestring: geometry, start_fraction: float, end_fraction: float) -> geometry`: 735, + `st_linesubstring(linestring: geometry, start_fraction: decimal, end_fraction: decimal) -> geometry`: 736, + `st_longestline(geometry_a: geometry, geometry_b: geometry) -> geometry`: 596, + `st_m(geometry: geometry) -> float`: 550, + `st_makebox2d(geometry_a: geometry, geometry_b: geometry) -> box2d`: 702, + `st_makeenvelope(xmin: float, ymin: float, xmax: float, ymax: float, srid: int) -> geometry`: 687, + `st_makeenvelope(xmin: float, ymin: float, xmax: float, ymax: float) -> geometry`: 688, + `st_makeline(arg1: geometry) -> geometry`: 254, + `st_makepoint(x: float, y: float) -> geometry`: 388, + `st_makepoint(x: float, y: float, z: float) -> geometry`: 389, + `st_makepoint(x: float, y: float, z: float, m: float) -> geometry`: 390, + `st_makepointm(x: float, y: float, m: float) -> geometry`: 391, + `st_makepolygon(geometry: geometry) -> geometry`: 392, + `st_makepolygon(outer: geometry, interior: anyelement[]) -> geometry`: 393, + `st_makevalid(geometry: geometry) -> geometry`: 627, + `st_maxdistance(geometry_a: geometry, geometry_b: geometry) -> float`: 595, + `st_memcollect(arg1: geometry) -> geometry`: 259, + `st_memsize(geometry: geometry) -> int`: 722, + `st_memunion(arg1: geometry) -> geometry`: 257, + `st_minimumboundingcircle(geometry: geometry) -> geometry`: 725, + `st_minimumboundingcircle(geometry: geometry, num_segs: int) -> geometry`: 726, + `st_minimumboundingradius(geometry: geometry) -> tuple{geometry AS center, float AS radius}`: 724, + `st_minimumclearance(geometry: geometry) -> float`: 522, + `st_minimumclearanceline(geometry: geometry) -> geometry`: 523, + `st_mlinefromtext(val: string) -> geometry`: 407, + `st_mlinefromtext(str: string, srid: int) -> geometry`: 408, + `st_mlinefromwkb(val: bytes) -> geometry`: 409, + `st_mlinefromwkb(wkb: bytes, srid: int) -> geometry`: 410, + `st_mpointfromtext(val: string) -> geometry`: 411, + `st_mpointfromtext(str: string, srid: int) -> geometry`: 412, + `st_mpointfromwkb(val: bytes) -> geometry`: 413, + `st_mpointfromwkb(wkb: bytes, srid: int) -> geometry`: 414, + `st_mpolyfromtext(val: string) -> geometry`: 415, + `st_mpolyfromtext(str: string, srid: int) -> geometry`: 416, + `st_mpolyfromwkb(val: bytes) -> geometry`: 417, + `st_mpolyfromwkb(wkb: bytes, srid: int) -> geometry`: 418, + `st_multi(geometry: geometry) -> geometry`: 572, + `st_multilinefromtext(val: string) -> geometry`: 419, + `st_multilinefromtext(str: string, srid: int) -> geometry`: 420, + `st_multilinefromwkb(val: bytes) -> geometry`: 421, + `st_multilinefromwkb(wkb: bytes, srid: int) -> geometry`: 422, + `st_multilinestringfromtext(val: string) -> geometry`: 423, + `st_multilinestringfromtext(str: string, srid: int) -> geometry`: 424, + `st_multilinestringfromwkb(val: bytes) -> geometry`: 425, + `st_multilinestringfromwkb(wkb: bytes, srid: int) -> geometry`: 426, + `st_multipointfromtext(val: string) -> geometry`: 427, + `st_multipointfromtext(str: string, srid: int) -> geometry`: 428, + `st_multipointfromwkb(val: bytes) -> geometry`: 429, + `st_multipointfromwkb(wkb: bytes, srid: int) -> geometry`: 430, + `st_multipolyfromtext(val: string) -> geometry`: 431, + `st_multipolyfromtext(str: string, srid: int) -> geometry`: 432, + `st_multipolyfromwkb(val: bytes) -> geometry`: 433, + `st_multipolyfromwkb(wkb: bytes, srid: int) -> geometry`: 434, + `st_multipolygonfromtext(val: string) -> geometry`: 435, + `st_multipolygonfromtext(str: string, srid: int) -> geometry`: 436, + `st_multipolygonfromwkb(val: bytes) -> geometry`: 437, + `st_multipolygonfromwkb(wkb: bytes, srid: int) -> geometry`: 438, + `st_ndims(geometry: geometry) -> int`: 506, + `st_node(geometry: geometry) -> geometry`: 699, + `st_normalize(geometry: geometry) -> geometry`: 612, + `st_npoints(geometry: geometry) -> int`: 516, + `st_nrings(geometry: geometry) -> int`: 525, + `st_numgeometries(geometry: geometry) -> int`: 538, + `st_numinteriorring(geometry: geometry) -> int`: 1748, + `st_numinteriorrings(geometry: geometry) -> int`: 524, + `st_numpoints(geometry: geometry) -> int`: 514, + `st_orderingequals(geometry_a: geometry, geometry_b: geometry) -> bool`: 611, + `st_orientedenvelope(geometry: geometry) -> geometry`: 734, + `st_overlaps(geometry_a: geometry, geometry_b: geometry) -> bool`: 615, + `st_perimeter(geography: geography) -> float`: 560, + `st_perimeter(geography: geography, use_spheroid: bool) -> float`: 561, + `st_perimeter(geometry: geometry) -> float`: 562, + `st_perimeter2d(geometry: geometry) -> float`: 563, + `st_point(x: float, y: float) -> geometry`: 459, + `st_pointfromgeohash(geohash: string, precision: int) -> geometry`: 460, + `st_pointfromgeohash(geohash: string) -> geometry`: 461, + `st_pointfromtext(val: string) -> geometry`: 439, + `st_pointfromtext(str: string, srid: int) -> geometry`: 440, + `st_pointfromwkb(val: bytes) -> geometry`: 441, + `st_pointfromwkb(wkb: bytes, srid: int) -> geometry`: 442, + `st_pointinsidecircle(geometry: geometry, x_coord: float, y_coord: float, radius: float) -> bool`: 721, + `st_pointn(geometry: geometry, n: int) -> geometry`: 520, + `st_pointonsurface(geometry: geometry) -> geometry`: 635, + `st_points(geometry: geometry) -> geometry`: 517, + `st_polyfromtext(val: string) -> geometry`: 443, + `st_polyfromtext(str: string, srid: int) -> geometry`: 444, + `st_polyfromwkb(val: bytes) -> geometry`: 445, + `st_polyfromwkb(wkb: bytes, srid: int) -> geometry`: 446, + `st_polygon(geometry: geometry, srid: int) -> geometry`: 394, + `st_polygonfromtext(val: string) -> geometry`: 447, + `st_polygonfromtext(str: string, srid: int) -> geometry`: 448, + `st_polygonfromwkb(val: bytes) -> geometry`: 449, + `st_polygonfromwkb(wkb: bytes, srid: int) -> geometry`: 450, + `st_project(geography: geography, distance: float, azimuth: float) -> geography`: 505, + `st_relate(geometry_a: geometry, geometry_b: geometry) -> string`: 618, + `st_relate(geometry_a: geometry, geometry_b: geometry, pattern: string) -> bool`: 619, + `st_relate(geometry_a: geometry, geometry_b: geometry, bnr: int) -> string`: 620, + `st_relatematch(intersection_matrix: string, pattern: string) -> bool`: 621, + `st_removepoint(line_string: geometry, index: int) -> geometry`: 666, + `st_removerepeatedpoints(geometry: geometry, tolerance: float) -> geometry`: 667, + `st_removerepeatedpoints(geometry: geometry) -> geometry`: 668, + `st_reverse(geometry: geometry) -> geometry`: 669, + `st_rotate(g: geometry, angle_radians: float) -> geometry`: 657, + `st_rotate(g: geometry, angle_radians: float, origin_x: float, origin_y: float) -> geometry`: 658, + `st_rotate(g: geometry, angle_radians: float, origin_point: geometry) -> geometry`: 659, + `st_rotatex(g: geometry, angle_radians: float) -> geometry`: 660, + `st_rotatey(g: geometry, angle_radians: float) -> geometry`: 661, + `st_rotatez(g: geometry, angle_radians: float) -> geometry`: 662, + `st_s2covering(geometry: geometry) -> geometry`: 374, + `st_s2covering(geometry: geometry, settings: string) -> geometry`: 375, + `st_s2covering(geography: geography) -> geography`: 376, + `st_s2covering(geography: geography, settings: string) -> geography`: 377, + `st_scale(geometry: geometry, x_factor: float, y_factor: float) -> geometry`: 654, + `st_scale(g: geometry, factor: geometry) -> geometry`: 655, + `st_scale(g: geometry, factor: geometry, origin: geometry) -> geometry`: 656, + `st_segmentize(geography: geography, max_segment_length_meters: float) -> geography`: 670, + `st_segmentize(geometry: geometry, max_segment_length: float) -> geometry`: 671, + `st_setpoint(line_string: geometry, index: int, point: geometry) -> geometry`: 665, + `st_setsrid(geometry: geometry, srid: int) -> geometry`: 644, + `st_setsrid(geography: geography, srid: int) -> geography`: 645, + `st_sharedpaths(geometry_a: geometry, geometry_b: geometry) -> geometry`: 638, + `st_shiftlongitude(geometry: geometry) -> geometry`: 578, + `st_shortestline(geometry_a: geometry, geometry_b: geometry) -> geometry`: 597, + `st_simplify(geometry: geometry, tolerance: float) -> geometry`: 641, + `st_simplify(geometry: geometry, tolerance: float, preserve_collapsed: bool) -> geometry`: 642, + `st_simplifypreservetopology(geometry: geometry, tolerance: float) -> geometry`: 643, + `st_snap(input: geometry, target: geometry, tolerance: float) -> geometry`: 676, + `st_snaptogrid(geometry: geometry, size: float) -> geometry`: 672, + `st_snaptogrid(geometry: geometry, size_x: float, size_y: float) -> geometry`: 673, + `st_snaptogrid(geometry: geometry, origin_x: float, origin_y: float, size_x: float, size_y: float) -> geometry`: 674, + `st_snaptogrid(geometry: geometry, origin: geometry, size_x: float, size_y: float, size_z: float, size_m: float) -> geometry`: 675, + `st_srid(geography: geography) -> int`: 564, + `st_srid(geometry: geometry) -> int`: 565, + `st_startpoint(geometry: geometry) -> geometry`: 508, + `st_subdivide(geometry: geometry) -> geometry`: 700, + `st_subdivide(geometry: geometry, max_vertices: int4) -> geometry`: 701, + `st_summary(geometry: geometry) -> string`: 509, + `st_summary(geography: geography) -> string`: 510, + `st_swapordinates(geometry: geometry, swap_ordinate_string: string) -> geometry`: 690, + `st_symdifference(geometry_a: geometry, geometry_b: geometry) -> geometry`: 640, + `st_symmetricdifference(geometry_a: geometry, geometry_b: geometry) -> geometry`: 1749, + `st_touches(geometry_a: geometry, geometry_b: geometry) -> bool`: 616, + `st_transform(geometry: geometry, srid: int) -> geometry`: 646, + `st_transform(geometry: geometry, to_proj_text: string) -> geometry`: 647, + `st_transform(geometry: geometry, from_proj_text: string, to_proj_text: string) -> geometry`: 648, + `st_transform(geometry: geometry, from_proj_text: string, srid: int) -> geometry`: 649, + `st_translate(g: geometry, delta_x: float, delta_y: float) -> geometry`: 650, + `st_translate(g: geometry, delta_x: float, delta_y: float, delta_z: float) -> geometry`: 651, + `st_transscale(geometry: geometry, delta_x: float, delta_y: float, x_factor: float, y_factor: float) -> geometry`: 727, + `st_unaryunion(geometry: geometry) -> geometry`: 698, + `st_union(arg1: geometry) -> geometry`: 256, + `st_voronoilines(geometry: geometry) -> geometry`: 731, + `st_voronoilines(geometry: geometry, tolerance: float) -> geometry`: 732, + `st_voronoilines(geometry: geometry, tolerance: float, extend_to: geometry) -> geometry`: 733, + `st_voronoipolygons(geometry: geometry) -> geometry`: 728, + `st_voronoipolygons(geometry: geometry, tolerance: float) -> geometry`: 729, + `st_voronoipolygons(geometry: geometry, tolerance: float, extend_to: geometry) -> geometry`: 730, + `st_within(geometry_a: geometry, geometry_b: geometry) -> bool`: 617, + `st_wkbtosql(val: bytes) -> geometry`: 381, + `st_wkttosql(val: string) -> geometry`: 382, + `st_x(geometry: geometry) -> float`: 539, + `st_xmax(geometry: geometry) -> float`: 542, + `st_xmax(box2d: box2d) -> float`: 543, + `st_xmin(geometry: geometry) -> float`: 540, + `st_xmin(box2d: box2d) -> float`: 541, + `st_y(geometry: geometry) -> float`: 544, + `st_ymax(geometry: geometry) -> float`: 547, + `st_ymax(box2d: box2d) -> float`: 548, + `st_ymin(geometry: geometry) -> float`: 545, + `st_ymin(box2d: box2d) -> float`: 546, + `st_z(geometry: geometry) -> float`: 549, + `st_zmflag(geometry: geometry) -> int2`: 551, + `statement_timestamp() -> timestamptz`: 1027, + `statement_timestamp() -> timestamp`: 1028, + `stddev(arg1: int) -> decimal`: 239, + `stddev(arg1: decimal) -> decimal`: 240, + `stddev(arg1: float) -> float`: 241, + `stddev_pop(arg1: int) -> decimal`: 245, + `stddev_pop(arg1: decimal) -> decimal`: 246, + `stddev_pop(arg1: float) -> float`: 247, + `stddev_samp(arg1: int) -> decimal`: 242, + `stddev_samp(arg1: decimal) -> decimal`: 243, + `stddev_samp(arg1: float) -> float`: 244, + `strftime(input: timestamp, extract_format: string) -> string`: 983, + `strftime(input: date, extract_format: string) -> string`: 984, + `strftime(input: timestamptz, extract_format: string) -> string`: 985, + `string_agg(arg1: string, arg2: string) -> string`: 212, + `string_agg(arg1: bytes, arg2: bytes) -> bytes`: 213, + `string_to_array(str: string, delimiter: string) -> string[]`: 1074, + `string_to_array(str: string, delimiter: string, null: string) -> string[]`: 1075, + `strpos(input: string, find: string) -> int`: 924, + `strpos(input: varbit, find: varbit) -> int`: 925, + `strpos(input: bytes, find: bytes) -> int`: 926, + `strptime(input: string, format: string) -> timestamptz`: 989, + `substr(input: string, start_pos: int) -> string`: 832, + `substr(input: string, start_pos: int, length: int) -> string`: 833, + `substr(input: string, regex: string) -> string`: 834, + `substr(input: string, regex: string, escape_char: string) -> string`: 835, + `substr(input: varbit, start_pos: int) -> varbit`: 836, + `substr(input: varbit, start_pos: int, length: int) -> varbit`: 837, + `substr(input: bytes, start_pos: int) -> bytes`: 838, + `substr(input: bytes, start_pos: int, length: int) -> bytes`: 839, + `substring(input: string, start_pos: int) -> string`: 840, + `substring(input: string, start_pos: int, length: int) -> string`: 841, + `substring(input: string, regex: string) -> string`: 842, + `substring(input: string, regex: string, escape_char: string) -> string`: 843, + `substring(input: varbit, start_pos: int) -> varbit`: 844, + `substring(input: varbit, start_pos: int, length: int) -> varbit`: 845, + `substring(input: bytes, start_pos: int) -> bytes`: 846, + `substring(input: bytes, start_pos: int, length: int) -> bytes`: 847, + `sum(arg1: int) -> decimal`: 215, + `sum(arg1: float) -> float`: 216, + `sum(arg1: decimal) -> decimal`: 217, + `sum(arg1: interval) -> interval`: 218, + `sum_int(arg1: int) -> int`: 214, + `tan(val: float) -> float`: 805, + `tand(val: float) -> float`: 806, + `tanh(val: float) -> float`: 807, + `text(val: inet) -> string`: 881, + `textin(input: anyelement) -> string`: 1885, + `textout(string: string) -> bytes`: 1884, + `textrecv(input: anyelement) -> string`: 1883, + `textsend(string: string) -> bytes`: 1882, + `time_in(input: anyelement) -> time`: 2009, + `time_out(time: time) -> bytes`: 2008, + `time_recv(input: anyelement) -> time`: 2007, + `time_send(time: time) -> bytes`: 2006, + `timeofday() -> string`: 1039, + `timestamp_in(input: anyelement) -> timestamp`: 1993, + `timestamp_out(timestamp: timestamp) -> bytes`: 1992, + `timestamp_recv(input: anyelement) -> timestamp`: 1991, + `timestamp_send(timestamp: timestamp) -> bytes`: 1990, + `timestamptz_in(input: anyelement) -> timestamptz`: 1865, + `timestamptz_out(timestamptz: timestamptz) -> bytes`: 1864, + `timestamptz_recv(input: anyelement) -> timestamptz`: 1863, + `timestamptz_send(timestamptz: timestamptz) -> bytes`: 1862, + `timetz_in(input: anyelement) -> timetz`: 1909, + `timetz_out(timetz: timetz) -> bytes`: 1908, + `timetz_recv(input: anyelement) -> timetz`: 1907, + `timetz_send(timetz: timetz) -> bytes`: 1906, + `timezone(timezone: string, timestamptz_string: string) -> timestamp`: 1059, + `timezone(timezone: string, timestamp: timestamp) -> timestamptz`: 1060, + `timezone(timezone: string, timestamptz: timestamptz) -> timestamp`: 1061, + `timezone(timezone: string, time: time) -> timetz`: 1062, + `timezone(timezone: string, timetz: timetz) -> timetz`: 1063, + `to_char(interval: interval) -> string`: 991, + `to_char(timestamp: timestamp) -> string`: 992, + `to_char(date: date) -> string`: 993, + `to_char_with_style(interval: interval, style: string) -> string`: 994, + `to_char_with_style(timestamp: timestamp, datestyle: string) -> string`: 995, + `to_char_with_style(date: date, datestyle: string) -> string`: 996, + `to_english(val: int) -> string`: 923, + `to_hex(val: int) -> string`: 920, + `to_hex(val: bytes) -> string`: 921, + `to_hex(val: string) -> string`: 922, + `to_ip(val: string) -> bytes`: 887, + `to_json(val: anyelement) -> jsonb`: 1254, + `to_jsonb(val: anyelement) -> jsonb`: 1255, + `to_regclass(text: string) -> regtype`: 2023, + `to_regnamespace(text: string) -> regtype`: 2025, + `to_regproc(text: string) -> regtype`: 2027, + `to_regprocedure(text: string) -> regtype`: 2029, + `to_regrole(text: string) -> regtype`: 2031, + `to_regtype(text: string) -> regtype`: 2033, + `to_timestamp(timestamp: float) -> timestamptz`: 997, + `to_uuid(val: string) -> bytes`: 868, + `transaction_timestamp() -> timestamptz`: 1014, + `transaction_timestamp() -> timestamp`: 1015, + `transaction_timestamp() -> date`: 1016, + `transition_regression_aggregate(arg1: float, arg2: float) -> decimal[]`: 66, + `transition_regression_aggregate(arg1: int, arg2: int) -> decimal[]`: 67, + `transition_regression_aggregate(arg1: decimal, arg2: decimal) -> decimal[]`: 68, + `transition_regression_aggregate(arg1: float, arg2: int) -> decimal[]`: 69, + `transition_regression_aggregate(arg1: float, arg2: decimal) -> decimal[]`: 70, + `transition_regression_aggregate(arg1: int, arg2: float) -> decimal[]`: 71, + `transition_regression_aggregate(arg1: int, arg2: decimal) -> decimal[]`: 72, + `transition_regression_aggregate(arg1: decimal, arg2: float) -> decimal[]`: 73, + `transition_regression_aggregate(arg1: decimal, arg2: int) -> decimal[]`: 74, + `translate(input: string, find: string, replace: string) -> string`: 941, + `trunc(val: float) -> float`: 808, + `trunc(val: decimal) -> decimal`: 809, + `trunc(val: decimal, scale: int) -> decimal`: 810, + `ulid_to_uuid(val: string) -> uuid`: 872, + `unaccent(val: string) -> string`: 828, + `unique_rowid() -> int`: 968, + `unknownin(input: anyelement) -> unknown`: 1897, + `unknownout(unknown: unknown) -> bytes`: 1896, + `unknownrecv(input: anyelement) -> unknown`: 1895, + `unknownsend(unknown: unknown) -> bytes`: 1894, + `unnest(input: anyelement[]) -> anyelement`: 326, + `unnest(anyelement[], anyelement[], anyelement[]...) -> tuple`: 327, + `unordered_unique_rowid() -> int`: 969, + `upper(val: string) -> string`: 829, + `uuid_generate_v1() -> uuid`: 864, + `uuid_generate_v1mc() -> uuid`: 865, + `uuid_generate_v3(namespace: uuid, name: string) -> uuid`: 866, + `uuid_generate_v4() -> uuid`: 858, + `uuid_generate_v5(namespace: uuid, name: string) -> uuid`: 867, + `uuid_in(input: anyelement) -> uuid`: 1961, + `uuid_nil() -> uuid`: 859, + `uuid_ns_dns() -> uuid`: 860, + `uuid_ns_oid() -> uuid`: 862, + `uuid_ns_url() -> uuid`: 861, + `uuid_ns_x500() -> uuid`: 863, + `uuid_out(uuid: uuid) -> bytes`: 1960, + `uuid_recv(input: anyelement) -> uuid`: 1959, + `uuid_send(uuid: uuid) -> bytes`: 1958, + `uuid_to_ulid(val: uuid) -> string`: 871, + `uuid_v4() -> bytes`: 980, + `var_pop(arg1: int) -> decimal`: 236, + `var_pop(arg1: decimal) -> decimal`: 237, + `var_pop(arg1: float) -> float`: 238, + `var_samp(arg1: int) -> decimal`: 233, + `var_samp(arg1: decimal) -> decimal`: 234, + `var_samp(arg1: float) -> float`: 235, + `varbit_in(input: anyelement) -> varbit`: 1941, + `varbit_out(varbit: varbit) -> bytes`: 1940, + `varbit_recv(input: anyelement) -> varbit`: 1939, + `varbit_send(varbit: varbit) -> bytes`: 1938, + `varcharin(input: anyelement) -> varchar`: 1889, + `varcharout(varchar: varchar) -> bytes`: 1888, + `varcharrecv(input: anyelement) -> varchar`: 1887, + `varcharsend(varchar: varchar) -> bytes`: 1886, + `variance(arg1: int) -> decimal`: 230, + `variance(arg1: decimal) -> decimal`: 231, + `variance(arg1: float) -> float`: 232, + `version() -> string`: 1284, + `voidin(input: anyelement) -> void`: 1997, + `voidout(void: void) -> bytes`: 1996, + `voidrecv(input: anyelement) -> void`: 1995, + `voidsend(void: void) -> bytes`: 1994, + `width_bucket(operand: decimal, b1: decimal, b2: decimal, count: int) -> int`: 811, + `width_bucket(operand: int, b1: int, b2: int, count: int) -> int`: 812, + `width_bucket(operand: anyelement, thresholds: anyelement[]) -> int`: 813, + `with_max_staleness(max_staleness: interval) -> timestamptz`: 1033, + `with_max_staleness(max_staleness: interval, nearest_only: bool) -> timestamptz`: 1034, + `with_min_timestamp(min_timestamp: timestamptz) -> timestamptz`: 1031, + `with_min_timestamp(min_timestamp: timestamptz, nearest_only: bool) -> timestamptz`: 1032, + `xor_agg(arg1: bytes) -> bytes`: 248, + `xor_agg(arg1: int) -> int`: 249, +} + +func signatureMustHaveHardcodedOID(sig string) oid.Oid { + oid, ok := builtinOidsBySignature[sig] + if !ok { + panic(fmt.Sprintf("Missing an entry for %s in builtins.BuiltinOidsBySignature. "+ + "Update the signature there if it's changed, or if it's new, add the following entry:"+ + "`%s`: %d", sig, sig, nextUnusedOid())) + } + return oid +} + +func oidsMustBeUnique() error { + sigsByOid := make(map[oid.Oid][]string, len(builtinOidsBySignature)) + for n, oid := range builtinOidsBySignature { + sigsByOid[oid] = append(sigsByOid[oid], n) + } + for oid, sigs := range sigsByOid { + if len(sigs) > 1 { + return errors.Newf("Multiple signatures have oid %d: %v", oid, sigs) + } + } + return nil +} + +// nextUnusedOid is used to construct the error message when a signature is missing +// from the map. Note that it doesn't check for collisions elsewhere. +func nextUnusedOid() oid.Oid { + var max oid.Oid + for _, oid := range builtinOidsBySignature { + if oid > max { + max = oid + } + } + return max + 1 +} + +func init() { + if err := oidsMustBeUnique(); err != nil { + panic(err) + } +} diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index 16fbc20a4867..6cb368df72f0 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -46,7 +46,7 @@ import ( var _ eval.ValueGenerator = &seriesValueGenerator{} var _ eval.ValueGenerator = &arrayValueGenerator{} -func initGeneratorBuiltins() { +func init() { // Add all windows to the builtins map after a few sanity checks. for k, v := range generators { if v.props.Class != tree.GeneratorClass { diff --git a/pkg/sql/sem/builtins/generator_probe_ranges.go b/pkg/sql/sem/builtins/generator_probe_ranges.go index 6105ebc76603..9c353cb7890c 100644 --- a/pkg/sql/sem/builtins/generator_probe_ranges.go +++ b/pkg/sql/sem/builtins/generator_probe_ranges.go @@ -34,7 +34,7 @@ import ( "github.com/cockroachdb/errors" ) -func initProbeRangesBuiltins() { +func init() { // Add all windows to the Builtins map after a few sanity checks. for k, v := range probeRangesGenerators { if v.props.Class != tree.GeneratorClass { diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 3bec85b9f3b0..7e84f30985e2 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -7223,7 +7223,7 @@ func toUseSphereOrSpheroid(useSpheroid tree.DBool) geogfn.UseSphereOrSpheroid { return geogfn.UseSphere } -func initGeoBuiltins() { +func init() { // Some functions have exactly the same definition - in which case, // we can just copy them. for _, alias := range []struct { diff --git a/pkg/sql/sem/builtins/help_test.go b/pkg/sql/sem/builtins/help_test.go index bb71447a3a0f..b41a4c145d09 100644 --- a/pkg/sql/sem/builtins/help_test.go +++ b/pkg/sql/sem/builtins/help_test.go @@ -25,12 +25,14 @@ import ( func TestHelpFunctions(t *testing.T) { defer leaktest.AfterTest(t)() + numTestsRun := 0 // This test checks that all the built-in functions receive contextual help. - builtinsregistry.Iterate(func(f string, _ *tree.FunctionProperties, _ []tree.Overload) { + builtinsregistry.AddSubscription(func(f string, _ *tree.FunctionProperties, _ []tree.Overload) { if unicode.IsUpper(rune(f[0])) { return } t.Run(f, func(t *testing.T) { + numTestsRun++ _, err := parser.Parse("select " + f + "(??") if err == nil { t.Errorf("parser didn't trigger error") @@ -55,4 +57,8 @@ func TestHelpFunctions(t *testing.T) { } }) }) + + if numTestsRun < 1000 { + t.Errorf("Test saw %d builtins, probably load order is wrong", numTestsRun) + } } diff --git a/pkg/sql/sem/builtins/math_builtins.go b/pkg/sql/sem/builtins/math_builtins.go index e390731eda95..7ef1360f7e29 100644 --- a/pkg/sql/sem/builtins/math_builtins.go +++ b/pkg/sql/sem/builtins/math_builtins.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/errors" ) -func initMathBuiltins() { +func init() { // Add all mathBuiltins to the builtins map after a sanity check. for k, v := range mathBuiltins { registerBuiltin(k, v) diff --git a/pkg/sql/sem/builtins/overlaps_builtins.go b/pkg/sql/sem/builtins/overlaps_builtins.go index 44fb61ad68ef..f4a8c4a4b22f 100644 --- a/pkg/sql/sem/builtins/overlaps_builtins.go +++ b/pkg/sql/sem/builtins/overlaps_builtins.go @@ -45,7 +45,7 @@ var ( } ) -func initOverlapsBuiltins() { +func init() { // Add all overlapsBuiltins to the builtins map after a sanity check. for k, v := range overlapsBuiltins { registerBuiltin(k, v) diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 5296bef3d99b..04511730a9d1 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -96,7 +96,7 @@ func PGIOBuiltinPrefix(typ *types.T) string { } // initPGBuiltins adds all of the postgres builtins to the builtins map. -func initPGBuiltins() { +func init() { for k, v := range pgBuiltins { v.props.Category = builtinconstants.CategoryCompatibility registerBuiltin(k, v) diff --git a/pkg/sql/sem/builtins/pgcrypto_builtins.go b/pkg/sql/sem/builtins/pgcrypto_builtins.go index 7fbe5d9e7a21..e1825c96fb78 100644 --- a/pkg/sql/sem/builtins/pgcrypto_builtins.go +++ b/pkg/sql/sem/builtins/pgcrypto_builtins.go @@ -34,7 +34,7 @@ import ( _ "golang.org/x/crypto/bcrypt" // linked to by go:linkname ) -func initPgcryptoBuiltins() { +func init() { // Add all pgcryptoBuiltins to the builtins map after a sanity check. for k, v := range pgcryptoBuiltins { registerBuiltin(k, v) diff --git a/pkg/sql/sem/builtins/replication_builtins.go b/pkg/sql/sem/builtins/replication_builtins.go index 8c65437a506f..363e353e316e 100644 --- a/pkg/sql/sem/builtins/replication_builtins.go +++ b/pkg/sql/sem/builtins/replication_builtins.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/errors" ) -func initReplicationBuiltins() { +func init() { // Add all replicationBuiltins to the builtins map after a sanity check. for k, v := range replicationBuiltins { registerBuiltin(k, v) diff --git a/pkg/sql/sem/builtins/trigram_builtins.go b/pkg/sql/sem/builtins/trigram_builtins.go index 95b1ee3a808d..6e7581e85a43 100644 --- a/pkg/sql/sem/builtins/trigram_builtins.go +++ b/pkg/sql/sem/builtins/trigram_builtins.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/trigram" ) -func initTrigramBuiltins() { +func init() { for k, v := range trigramBuiltins { v.props.Category = builtinconstants.CategoryTrigram v.props.AvailableOnPublicSchema = true diff --git a/pkg/sql/sem/builtins/window_builtins.go b/pkg/sql/sem/builtins/window_builtins.go index 9c404a01ba08..1b849a5d649a 100644 --- a/pkg/sql/sem/builtins/window_builtins.go +++ b/pkg/sql/sem/builtins/window_builtins.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/errors" ) -func initWindowBuiltins() { +func init() { // Add all windows to the builtins map after a few sanity checks. for k, v := range windows { if v.props.Class != tree.WindowClass { diff --git a/pkg/sql/sem/tree/object_name.go b/pkg/sql/sem/tree/object_name.go index 1a7797bf270f..2af07caede27 100644 --- a/pkg/sql/sem/tree/object_name.go +++ b/pkg/sql/sem/tree/object_name.go @@ -22,6 +22,7 @@ type ObjectName interface { var _ ObjectName = &TableName{} var _ ObjectName = &TypeName{} +var _ ObjectName = &FunctionName{} // objName is the internal type for a qualified object. type objName struct { diff --git a/pkg/sql/sem/tree/overload.go b/pkg/sql/sem/tree/overload.go index dd897addce8b..8deaf6925b89 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -135,8 +135,10 @@ type Overload struct { // statement which will be executed as a common table expression in the query. SQLFn SQLFnOverload - // OnTypeCheck is incremented every time this overload is type checked. - OnTypeCheck func() + // OnTypeCheck is called every time this overload is type checked. + // This is a pointer so that it can be set in a builtinsregistry hook, which + // gets a copy of the overload struct. + OnTypeCheck *func() // SpecializedVecBuiltin is used to let the vectorized engine // know when an Overload has a specialized vectorized operator. diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index f76ff7416820..560e211ec3e1 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -1231,8 +1231,8 @@ func (expr *FuncExpr) TypeCheck( if err := semaCtx.checkVolatility(overloadImpl.Volatility); err != nil { return nil, pgerror.Wrapf(err, pgcode.InvalidParameterValue, "%s()", def.Name) } - if overloadImpl.OnTypeCheck != nil { - overloadImpl.OnTypeCheck() + if overloadImpl.OnTypeCheck != nil && *overloadImpl.OnTypeCheck != nil { + (*overloadImpl.OnTypeCheck)() } return expr, nil } diff --git a/pkg/sql/sem/tree/udf.go b/pkg/sql/sem/tree/udf.go index 88f550de9c54..7de236932f5a 100644 --- a/pkg/sql/sem/tree/udf.go +++ b/pkg/sql/sem/tree/udf.go @@ -66,6 +66,20 @@ func (f *FunctionName) Format(ctx *FmtCtx) { func (f *FunctionName) String() string { return AsString(f) } +// FQString renders the function name in full, not omitting the prefix +// schema and catalog names. Suitable for logging, etc. +func (f *FunctionName) FQString() string { + ctx := NewFmtCtx(FmtSimple) + ctx.FormatNode(&f.CatalogName) + ctx.WriteByte('.') + ctx.FormatNode(&f.SchemaName) + ctx.WriteByte('.') + ctx.FormatNode(&f.ObjectName) + return ctx.CloseAndGetString() +} + +func (f *FunctionName) objectName() {} + // CreateFunction represents a CREATE FUNCTION statement. type CreateFunction struct { IsProcedure bool diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index 532c7bd5a0a2..751c77995546 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -167,7 +167,7 @@ func formatViewQueryForDisplay( } // Convert sequences referenced by ID in the view back to their names. - sequenceReplacedViewQuery, err := formatViewQuerySequencesForDisplay(ctx, semaCtx, typeReplacedViewQuery) + sequenceReplacedViewQuery, err := formatQuerySequencesForDisplay(ctx, semaCtx, typeReplacedViewQuery, false /* multiStmt */) if err != nil { log.Warningf(ctx, "error converting sequence IDs to names for view %s (%v): %+v", desc.GetName(), desc.GetID(), err) @@ -177,11 +177,11 @@ func formatViewQueryForDisplay( return sequenceReplacedViewQuery } -// formatViewQuerySequencesForDisplay walks the view query and +// formatQuerySequencesForDisplay walks the view query and // looks for sequence IDs in the statement. If it finds any, // it will replace the IDs with the descriptor's fully qualified name. -func formatViewQuerySequencesForDisplay( - ctx context.Context, semaCtx *tree.SemaContext, viewQuery string, +func formatQuerySequencesForDisplay( + ctx context.Context, semaCtx *tree.SemaContext, queries string, multiStmt bool, ) (string, error) { replaceFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { newExpr, err = schemaexpr.ReplaceIDsWithFQNames(ctx, expr, semaCtx) @@ -191,16 +191,39 @@ func formatViewQuerySequencesForDisplay( return false, newExpr, nil } - stmt, err := parser.ParseOne(viewQuery) - if err != nil { - return "", err + var stmts tree.Statements + if multiStmt { + parsedStmts, err := parser.Parse(queries) + if err != nil { + return "", err + } + stmts = make(tree.Statements, len(parsedStmts)) + for i, stmt := range parsedStmts { + stmts[i] = stmt.AST + } + } else { + stmt, err := parser.ParseOne(queries) + if err != nil { + return "", err + } + stmts = tree.Statements{stmt.AST} } - newStmt, err := tree.SimpleStmtVisit(stmt.AST, replaceFunc) - if err != nil { - return "", err + fmtCtx := tree.NewFmtCtx(tree.FmtSimple) + for i, stmt := range stmts { + newStmt, err := tree.SimpleStmtVisit(stmt, replaceFunc) + if err != nil { + return "", err + } + if i > 0 { + fmtCtx.WriteString("\n") + } + fmtCtx.FormatNode(newStmt) + if multiStmt { + fmtCtx.WriteString(";") + } } - return newStmt.String(), nil + return fmtCtx.CloseAndGetString(), nil } // formatViewQueryTypesForDisplay walks the view query and @@ -258,6 +281,77 @@ func formatViewQueryTypesForDisplay( return newStmt.String(), nil } +// formatFunctionQueryTypesForDisplay is similar to +// formatViewQueryTypesForDisplay but can only be used for function. +// nil is used as the table descriptor for schemaexpr.FormatExprForDisplay call. +// This is fine assuming that UDFs cannot be created with expression casting a +// column/var to an enum in function body. This is super rare case for now, and +// it's tracked with issue #87475. We should also unify this function with +// formatViewQueryTypesForDisplay. +func formatFunctionQueryTypesForDisplay( + ctx context.Context, + semaCtx *tree.SemaContext, + sessionData *sessiondata.SessionData, + queries string, +) (string, error) { + replaceFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { + // We need to resolve the type to check if it's user-defined. If not, + // no other work is needed. + var typRef tree.ResolvableTypeReference + switch n := expr.(type) { + case *tree.CastExpr: + typRef = n.Type + case *tree.AnnotateTypeExpr: + typRef = n.Type + default: + return true, expr, nil + } + var typ *types.T + typ, err = tree.ResolveType(ctx, typRef, semaCtx.TypeResolver) + if err != nil { + return false, expr, err + } + if !typ.UserDefined() { + return true, expr, nil + } + formattedExpr, err := schemaexpr.FormatExprForDisplay( + ctx, nil, expr.String(), semaCtx, sessionData, tree.FmtParsable, + ) + if err != nil { + return false, expr, err + } + newExpr, err = parser.ParseExpr(formattedExpr) + if err != nil { + return false, expr, err + } + return false, newExpr, nil + } + + var stmts tree.Statements + parsedStmts, err := parser.Parse(queries) + if err != nil { + return "", errors.Wrap(err, "failed to parse query") + } + stmts = make(tree.Statements, len(parsedStmts)) + for i, stmt := range parsedStmts { + stmts[i] = stmt.AST + } + + fmtCtx := tree.NewFmtCtx(tree.FmtSimple) + for i, stmt := range stmts { + newStmt, err := tree.SimpleStmtVisit(stmt, replaceFunc) + if err != nil { + return "", err + } + if i > 0 { + fmtCtx.WriteString("\n") + } + fmtCtx.FormatNode(newStmt) + fmtCtx.WriteString(";") + } + return fmtCtx.CloseAndGetString(), nil +} + // showComments prints out the COMMENT statements sufficient to populate a // table's comments, including its index and column comments. func showComments( diff --git a/pkg/sql/sqltelemetry/BUILD.bazel b/pkg/sql/sqltelemetry/BUILD.bazel index 73d9b99476cd..192b3002ac32 100644 --- a/pkg/sql/sqltelemetry/BUILD.bazel +++ b/pkg/sql/sqltelemetry/BUILD.bazel @@ -35,6 +35,8 @@ go_library( "//pkg/settings", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/sem/builtins/builtinsregistry", + "//pkg/sql/sem/tree", "//pkg/util/log", "//pkg/util/log/logcrash", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/sqltelemetry/scalar.go b/pkg/sql/sqltelemetry/scalar.go index f7144cd5187c..56536fdce763 100644 --- a/pkg/sql/sqltelemetry/scalar.go +++ b/pkg/sql/sqltelemetry/scalar.go @@ -14,6 +14,8 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) // BuiltinCounter creates a telemetry counter for a built-in function. @@ -22,6 +24,17 @@ func BuiltinCounter(name, signature string) telemetry.Counter { return telemetry.GetCounterOnce(fmt.Sprintf("sql.plan.builtins.%s%s", name, signature)) } +func init() { + builtinsregistry.AddSubscription(func(name string, _ *tree.FunctionProperties, os []tree.Overload) { + for _, o := range os { + c := BuiltinCounter(name, o.Signature(false)) + *o.OnTypeCheck = func() { + telemetry.Inc(c) + } + } + }) +} + // UnaryOpCounter creates a telemetry counter for a scalar unary operator. // This is to be incremented upon type checking of this type of scalar operation. func UnaryOpCounter(op, typ string) telemetry.Counter { diff --git a/pkg/sql/tests/rsg_test.go b/pkg/sql/tests/rsg_test.go index ee61a1e460c2..27d610f9022c 100644 --- a/pkg/sql/tests/rsg_test.go +++ b/pkg/sql/tests/rsg_test.go @@ -341,7 +341,7 @@ func TestRandomSyntaxFunctions(t *testing.T) { namedBuiltinChan := make(chan namedBuiltin) go func() { for { - for _, name := range builtins.AllBuiltinNames { + for _, name := range builtins.AllBuiltinNames() { lower := strings.ToLower(name) if strings.HasPrefix(lower, "crdb_internal.force_") { continue diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index f13838028284..26326371cbb5 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -129,7 +129,7 @@ func TestLint(t *testing.T) { } var names []string - for _, name := range builtins.AllBuiltinNames { + for _, name := range builtins.AllBuiltinNames() { switch name { case "extract", "trim", "overlay", "position", "substring", "st_x", "st_y": // Exempt special forms: EXTRACT(... FROM ...), etc.