From 8b20e5daa7d87b9d09754f9cc1fa3572415309a2 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Thu, 4 Aug 2022 17:26:21 -0400 Subject: [PATCH 1/4] psql: populate pg_proc with udf This commit extends the pg_proc virtual table with user-defined functions. Release note (sql change): Previously pg_proc table was only populated with builtin functions. With createing UDFs supported, pg_proc table is extended to be populated with UDFs data as well. --- pkg/sql/catalog/descriptor.go | 3 + pkg/sql/catalog/funcdesc/func_desc.go | 5 + pkg/sql/catalog/schema.go | 4 + pkg/sql/catalog/schemadesc/schema_desc.go | 14 +++ .../schemadesc/synthetic_schema_desc.go | 7 ++ pkg/sql/logictest/testdata/logic_test/udf | 22 +++++ pkg/sql/pg_catalog.go | 98 ++++++++++++++++++- 7 files changed, 152 insertions(+), 1 deletion(-) diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 4bad937abfe7..0f04ee44722f 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -875,6 +875,9 @@ type FunctionDescriptor interface { // ToOverload converts the function descriptor to tree.Overload object which // can be used for execution. ToOverload() (ret *tree.Overload, err error) + + // GetLanguage returns the language of this function. + GetLanguage() catpb.Function_Language } // FilterDescriptorState inspects the state of a given descriptor and returns an diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index ea15b8b0e9df..2783b86737d5 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -457,6 +457,11 @@ func (desc *immutable) FuncDesc() *descpb.FunctionDescriptor { return &desc.FunctionDescriptor } +// GetLanguage implements the FunctionDescriptor interface. +func (desc *immutable) GetLanguage() catpb.Function_Language { + return desc.Lang +} + // ContainsUserDefinedTypes implements the catalog.HydratableDescriptor interface. func (desc *immutable) ContainsUserDefinedTypes() bool { for i := range desc.Args { diff --git a/pkg/sql/catalog/schema.go b/pkg/sql/catalog/schema.go index 99edb117ba14..714fc2493d2a 100644 --- a/pkg/sql/catalog/schema.go +++ b/pkg/sql/catalog/schema.go @@ -41,6 +41,10 @@ type SchemaDescriptor interface { // returns a collection of overloads with the same function name, each // overload is prefixed with the same schema name. GetResolvedFuncDefinition(name string) (*tree.ResolvedFunctionDefinition, bool) + + // ForEachFunctionOverload iterates through all function overloads within the + // schema and calls fn on each overload. + ForEachFunctionOverload(fn func(overload descpb.SchemaDescriptor_FunctionOverload) error) error } // ResolvedSchemaKind is an enum that represents what kind of schema diff --git a/pkg/sql/catalog/schemadesc/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go index 04194eaaba45..41731906406a 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc.go +++ b/pkg/sql/catalog/schemadesc/schema_desc.go @@ -469,6 +469,20 @@ func (desc *immutable) GetResolvedFuncDefinition( return funcDef, true } +// ForEachFunctionOverload implements the SchemaDescriptor interface. +func (desc *immutable) ForEachFunctionOverload( + fn func(overload descpb.SchemaDescriptor_FunctionOverload) error, +) error { + for _, function := range desc.Functions { + for i := range function.Overloads { + if err := fn(function.Overloads[i]); err != nil { + return err + } + } + } + return nil +} + // IsSchemaNameValid returns whether the input name is valid for a user defined // schema. func IsSchemaNameValid(name string) error { diff --git a/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go index 761faa459154..4f32bac2a41e 100644 --- a/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go +++ b/pkg/sql/catalog/schemadesc/synthetic_schema_desc.go @@ -153,6 +153,13 @@ func (p synthetic) GetFunction(name string) (descpb.SchemaDescriptor_Function, b return descpb.SchemaDescriptor_Function{}, false } +// ForEachFunctionOverload implements the SchemaDescriptor interface. +func (p synthetic) ForEachFunctionOverload( + fn func(overload descpb.SchemaDescriptor_FunctionOverload) error, +) error { + return nil +} + func (p synthetic) ContainsUserDefinedTypes() bool { return false } diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 89754cd5bc7f..ee08dd9a7a45 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -380,3 +380,25 @@ SELECT * FROM fetch_one_then_two() ---- fetch_one_then_two 2 + +subtest udf_pg_proc + +statement ok +CREATE FUNCTION proc_f(INT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +statement +CREATE FUNCTION proc_f(STRING, b INT) RETURNS SETOF STRING STRICT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$ SELECT 'hello' $$; + +statement ok +CREATE SCHEMA sc; + +statement +CREATE FUNCTION sc.proc_f_2(STRING) RETURNS STRING LANGUAGE SQL AS $$ SELECT 'hello' $$; + +query TTTTTBBBTITTTTT +SELECT oid, proname, pronamespace, proowner, prolang, proleakproof, proisstrict, proretset, provolatile, pronargs, prorettype, proargtypes, proargmodes, proargnames, prosrc +FROM pg_catalog.pg_proc WHERE proname IN ('proc_f', 'proc_f_2'); +---- +100115 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; +100116 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; +100118 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 093ec12f716b..09d0568cddaa 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -2290,7 +2290,12 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, schema: vtable.PGCatalogProc, populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { h := makeOidHasher() - return forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */ + // Build rows for builtin function. Normally, dbContext is not nil. So only + // dbContext is looked at and used to generate the NamespaceOid. However, + // the downside is that the NamespaceOid would change if pg_catalog.pg_proc + // is selected from a different database. But this is probably fine for + // builtin function since they don't really belong to any database. + err := forEachDatabaseDesc(ctx, p, dbContext, false, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { nspOid := h.NamespaceOid(db.GetID(), pgCatalogName) for _, name := range builtins.AllBuiltinNames { @@ -2414,6 +2419,84 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, } return nil }) + if err != nil { + return err + } + return forEachDatabaseDesc(ctx, p, nil /* dbContext */, false, /* requiresPrivileges */ + func(dbDesc catalog.DatabaseDescriptor) error { + return forEachSchema(ctx, p, dbDesc, func(scDesc catalog.SchemaDescriptor) error { + return scDesc.ForEachFunctionOverload(func(overload descpb.SchemaDescriptor_FunctionOverload) error { + fnDesc, err := p.Descriptors().GetImmutableFunctionByID(ctx, p.Txn(), overload.ID, tree.ObjectLookupFlags{}) + if err != nil { + return err + } + isStrict := fnDesc.GetNullInputBehavior() != catpb.Function_CALLED_ON_NULL_INPUT + argTypes := tree.NewDArray(types.Oid) + argModes := tree.NewDArray(types.String) + var argNames tree.Datum + argNamesArray := tree.NewDArray(types.String) + foundAnyArgNames := false + for _, arg := range fnDesc.GetArgs() { + if err := argTypes.Append(tree.NewDOid(arg.Type.Oid())); err != nil { + return err + } + // We only support IN arguments at the moment. + if err := argModes.Append(tree.NewDString("i")); err != nil { + return err + } + if len(arg.Name) > 0 { + foundAnyArgNames = true + } + if err := argNamesArray.Append(tree.NewDString(arg.Name)); err != nil { + return err + } + } + argNames = tree.DNull + if foundAnyArgNames { + argNames = argNamesArray + } + + return addRow( + // TODO (chengxiong) fix this when we have the ID to OID thing + tree.NewDOid(catid.FuncIDToOID(fnDesc.GetID())), // oid + tree.NewDName(fnDesc.GetName()), // proname + h.NamespaceOid(dbDesc.GetID(), scDesc.GetName()), // pronamespace + h.UserOid(fnDesc.GetPrivileges().Owner()), // proowner + // In postgres oid of sql language is 14, need to add a mapping if + // we are going to support more languages. + tree.NewDOid(14), // prolang + tree.DNull, // procost + tree.DNull, // prorows + oidZero, // provariadic + tree.DNull, // protransform + tree.DBoolFalse, // proisagg + tree.DBoolFalse, // proiswindow + tree.DBoolFalse, // prosecdef + tree.MakeDBool(tree.DBool(fnDesc.GetLeakProof())), // proleakproof + tree.MakeDBool(tree.DBool(isStrict)), // proisstrict + tree.MakeDBool(tree.DBool(fnDesc.GetReturnType().ReturnSet)), // proretset + tree.NewDString(funcVolatility(fnDesc.GetVolatility())), // provolatile + tree.DNull, // proparallel + tree.NewDInt(tree.DInt(len(fnDesc.GetArgs()))), // pronargs + tree.NewDInt(tree.DInt(0)), // pronargdefaults + tree.NewDOid(fnDesc.GetReturnType().Type.Oid()), // prorettype + tree.NewDOidVectorFromDArray(argTypes), // proargtypes + tree.DNull, // proallargtypes + argModes, // proargmodes + argNames, // proargnames + tree.DNull, // proargdefaults + tree.DNull, // protrftypes + tree.NewDString(fnDesc.GetFunctionBody()), // prosrc + tree.DNull, // probin + tree.DNull, // proconfig + tree.DNull, // proacl + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // prokind + tree.DNull, // prosupport + ) + }) + }) + }) }, } @@ -4572,3 +4655,16 @@ func (h oidHasher) CastOid(srcID oid.Oid, tgtID oid.Oid) *tree.DOid { h.writeUInt32(uint32(tgtID)) return h.getOid() } + +func funcVolatility(v catpb.Function_Volatility) string { + switch v { + case catpb.Function_IMMUTABLE: + return "i" + case catpb.Function_STABLE: + return "s" + case catpb.Function_VOLATILE: + return "v" + default: + return "" + } +} From 9a50c6befbbc452b57497536da70860451ce8411 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Fri, 5 Aug 2022 00:37:57 -0400 Subject: [PATCH 2/4] sql: add crdb_internal.create_function_statements virtual table Release note (sql change): a new virtual table crdb_internal.create_function_statements is added, so that users can use to query create statements of user defined functions, as well as parent db and schema ids. --- .../testdata/logic_test/crdb_internal_tenant | 1 + pkg/cli/testdata/zip/partial1 | 1 + pkg/cli/testdata/zip/partial1_excluded | 1 + pkg/cli/testdata/zip/partial2 | 1 + pkg/cli/testdata/zip/testzip | 1 + pkg/cli/testdata/zip/testzip_concurrent | 3 + pkg/cli/testdata/zip/testzip_tenant | 1 + pkg/cli/zip_cluster_wide.go | 1 + pkg/sql/catalog/descriptor.go | 4 + pkg/sql/catalog/funcdesc/BUILD.bazel | 1 + pkg/sql/catalog/funcdesc/func_desc.go | 81 + pkg/sql/catalog/typedesc/type_desc.go | 1 - pkg/sql/crdb_internal.go | 57 + pkg/sql/function_resolver_test.go | 24 +- .../testdata/logic_test/crdb_internal | 1 + .../testdata/logic_test/create_statements | 17 + .../logictest/testdata/logic_test/grant_table | 1 + .../testdata/logic_test/information_schema | 5 + .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 3322 +++++++++-------- pkg/sql/logictest/testdata/logic_test/table | 1 + pkg/sql/logictest/testdata/logic_test/udf | 96 +- .../opt/optbuilder/testdata/create_function | 25 +- pkg/sql/parser/testdata/create_function | 176 +- pkg/sql/sem/catconstants/constants.go | 1 + pkg/sql/sem/tree/udf.go | 6 +- 26 files changed, 2166 insertions(+), 1735 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 0e700207604d..07ea6185c0f9 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -52,6 +52,7 @@ crdb_internal cluster_settings table admin NULL NULL crdb_internal cluster_statement_statistics table admin NULL NULL crdb_internal cluster_transaction_statistics table admin NULL NULL crdb_internal cluster_transactions table admin NULL NULL +crdb_internal create_function_statements table admin NULL NULL crdb_internal create_schema_statements table admin NULL NULL crdb_internal create_statements table admin NULL NULL crdb_internal create_type_statements table admin NULL NULL diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 0cbb3ff8cca4..25b3daea2f51 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -24,6 +24,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/crdb_internal.create_function_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 3d8d58896d38..b94b6d7f76ba 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -24,6 +24,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/crdb_internal.create_function_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index aab22367bcf1..fd8a0d36d5f3 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -24,6 +24,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/crdb_internal.create_function_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 4fd94a90cb86..5da5334e4762 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -24,6 +24,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/crdb_internal.create_function_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index e74618f3087b..3ae03e4a985b 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -46,6 +46,9 @@ zip [cluster] requesting tenant ranges: done [cluster] requesting tenant ranges: last request failed: rpc error: ... [cluster] requesting tenant ranges: received response... +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements: done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements: writing output: debug/crdb_internal.create_function_statements.txt... [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements: done [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements: writing output: debug/crdb_internal.create_schema_statements.txt... diff --git a/pkg/cli/testdata/zip/testzip_tenant b/pkg/cli/testdata/zip/testzip_tenant index a966e66d6d49..66975cc521b2 100644 --- a/pkg/cli/testdata/zip/testzip_tenant +++ b/pkg/cli/testdata/zip/testzip_tenant @@ -32,6 +32,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/crdb_internal.create_function_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... [cluster] retrieving SQL data for crdb_internal.kv_node_liveness: last request failed: ERROR: unimplemented: operation is unsupported in multi-tenancy mode (SQLSTATE 0A000) [cluster] retrieving SQL data for crdb_internal.kv_node_liveness: creating error output: debug/crdb_internal.kv_node_liveness.txt.err.txt... done diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index aa7cf0bbf8e1..e9ce7581a1c8 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -94,6 +94,7 @@ var debugZipTablesPerCluster = []string{ `"".crdb_internal.create_statements`, // Ditto, for CREATE TYPE. `"".crdb_internal.create_type_statements`, + `"".crdb_internal.create_function_statements`, "crdb_internal.kv_node_liveness", "crdb_internal.kv_node_status", diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 0f04ee44722f..d21dfff97262 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -878,6 +878,10 @@ type FunctionDescriptor interface { // GetLanguage returns the language of this function. GetLanguage() catpb.Function_Language + + // ToCreateExpr converts a function descriptor back to a CREATE FUNCTION + // statement. This is mainly used for formatting, e.g. SHOW CREATE FUNCTION. + ToCreateExpr() (*tree.CreateFunction, error) } // FilterDescriptorState inspects the state of a given descriptor and returns an diff --git a/pkg/sql/catalog/funcdesc/BUILD.bazel b/pkg/sql/catalog/funcdesc/BUILD.bazel index 510a749e0268..92aee3e8b8ef 100644 --- a/pkg/sql/catalog/funcdesc/BUILD.bazel +++ b/pkg/sql/catalog/funcdesc/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/catprivilege", "//pkg/sql/catalog/descpb", + "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/privilege", diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index 2783b86737d5..7ab3b1ef086b 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" @@ -536,6 +537,86 @@ func (desc *immutable) calledOnNullInput() (bool, error) { } } +// ToCreateExpr implements the FunctionDescriptor interface. +func (desc *immutable) ToCreateExpr() (ret *tree.CreateFunction, err error) { + ret = &tree.CreateFunction{ + FuncName: tree.MakeFunctionNameFromPrefix(tree.ObjectNamePrefix{}, tree.Name(desc.Name)), + ReturnType: tree.FuncReturnType{ + Type: desc.ReturnType.Type, + IsSet: desc.ReturnType.ReturnSet, + }, + } + ret.Args = make(tree.FuncArgs, len(desc.Args)) + for i := range desc.Args { + ret.Args[i] = tree.FuncArg{ + Name: tree.Name(desc.Args[i].Name), + Type: desc.Args[i].Type, + Class: toTreeNodeArgClass(desc.Args[i].Class), + } + if desc.Args[i].DefaultExpr != nil { + ret.Args[i].DefaultVal, err = parser.ParseExpr(*desc.Args[i].DefaultExpr) + if err != nil { + return nil, err + } + } + } + // We only store 5 function attributes at the moment. We may extend the + // pre-allocated capacity in the future. + ret.Options = make(tree.FunctionOptions, 0, 5) + ret.Options = append(ret.Options, desc.getCreateExprVolatility()) + ret.Options = append(ret.Options, tree.FunctionLeakproof(desc.LeakProof)) + ret.Options = append(ret.Options, desc.getCreateExprNullInputBehavior()) + ret.Options = append(ret.Options, tree.FunctionBodyStr(desc.FunctionBody)) + ret.Options = append(ret.Options, desc.getCreateExprLang()) + return ret, nil +} + +func (desc *immutable) getCreateExprLang() tree.FunctionLanguage { + switch desc.Lang { + case catpb.Function_SQL: + return tree.FunctionLangSQL + } + return 0 +} + +func (desc *immutable) getCreateExprVolatility() tree.FunctionVolatility { + switch desc.Volatility { + case catpb.Function_IMMUTABLE: + return tree.FunctionImmutable + case catpb.Function_STABLE: + return tree.FunctionStable + case catpb.Function_VOLATILE: + return tree.FunctionVolatile + } + return 0 +} + +func (desc *immutable) getCreateExprNullInputBehavior() tree.FunctionNullInputBehavior { + switch desc.NullInputBehavior { + case catpb.Function_CALLED_ON_NULL_INPUT: + return tree.FunctionCalledOnNullInput + case catpb.Function_RETURNS_NULL_ON_NULL_INPUT: + return tree.FunctionReturnsNullOnNullInput + case catpb.Function_STRICT: + return tree.FunctionStrict + } + return 0 +} + +func toTreeNodeArgClass(class catpb.Function_Arg_Class) tree.FuncArgClass { + switch class { + case catpb.Function_Arg_IN: + return tree.FunctionArgIn + case catpb.Function_Arg_OUT: + return tree.FunctionArgOut + case catpb.Function_Arg_IN_OUT: + return tree.FunctionArgInOut + case catpb.Function_Arg_VARIADIC: + return tree.FunctionArgVariadic + } + return 0 +} + // UserDefinedFunctionOIDToID converts a UDF OID into a descriptor ID. OID of a // UDF must be greater CockroachPredefinedOIDMax. The function returns an error // if the given OID is less than or equal to CockroachPredefinedOIDMax. diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 065d900eb875..02e77e5a40d4 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -796,7 +796,6 @@ func EnsureTypeIsHydrated( return err } } - return nil } if !t.UserDefined() || t.IsHydrated() { return nil diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 419e4cd559ef..babac7687bd5 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -116,6 +116,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalClusterSessionsTableID: crdbInternalClusterSessionsTable, catconstants.CrdbInternalClusterSettingsTableID: crdbInternalClusterSettingsTable, catconstants.CrdbInternalClusterStmtStatsTableID: crdbInternalClusterStmtStatsTable, + catconstants.CrdbInternalCreateFunctionStmtsTableID: crdbInternalCreateFunctionStmtsTable, catconstants.CrdbInternalCreateSchemaStmtsTableID: crdbInternalCreateSchemaStmtsTable, catconstants.CrdbInternalCreateStmtsTableID: crdbInternalCreateStmtsTable, catconstants.CrdbInternalCreateTypeStmtsTableID: crdbInternalCreateTypeStmtsTable, @@ -2576,6 +2577,62 @@ CREATE TABLE crdb_internal.create_schema_statements ( }, } +var crdbInternalCreateFunctionStmtsTable = virtualSchemaTable{ + comment: "CREATE statements for all user-defined functions", + schema: ` +CREATE TABLE crdb_internal.create_function_statements ( + database_id INT, + database_name STRING, + schema_id INT, + schema_name STRING, + function_id INT, + function_name STRING, + create_statement STRING +) +`, + populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + return forEachSchema(ctx, p, db, func(sc catalog.SchemaDescriptor) error { + return sc.ForEachFunctionOverload(func(overload descpb.SchemaDescriptor_FunctionOverload) error { + fnDesc, err := p.Descriptors().GetImmutableFunctionByID(ctx, p.txn, overload.ID, tree.ObjectLookupFlags{}) + if err != nil { + return err + } + treeNode, err := fnDesc.ToCreateExpr() + treeNode.FuncName.ObjectNamePrefix = tree.ObjectNamePrefix{ + ExplicitSchema: true, + SchemaName: tree.Name(sc.GetName()), + } + if err != nil { + return err + } + for i := range treeNode.Options { + if body, ok := treeNode.Options[i].(tree.FunctionBodyStr); ok { + stmtStrs := strings.Split(string(body), "\n") + for i := range stmtStrs { + stmtStrs[i] = "\t" + stmtStrs[i] + } + + p := &treeNode.Options[i] + // Add two new lines just for better formatting. + *p = "\n" + tree.FunctionBodyStr(strings.Join(stmtStrs, "\n")) + "\n" + } + } + + return addRow( + tree.NewDInt(tree.DInt(db.GetID())), // database_id + tree.NewDString(db.GetName()), // database_name + tree.NewDInt(tree.DInt(sc.GetID())), // schema_id + tree.NewDString(sc.GetName()), // schema_name + tree.NewDInt(tree.DInt(fnDesc.GetID())), // function_id + tree.NewDString(fnDesc.GetName()), //function_name + tree.NewDString(tree.AsString(treeNode)), // create_statement + ) + }) + }) + return nil + }, +} + // Prepare the row populate function. var typeView = tree.NewDString("view") var typeTable = tree.NewDString("table") diff --git a/pkg/sql/function_resolver_test.go b/pkg/sql/function_resolver_test.go index 78d1df646baf..79f07c9a44bc 100644 --- a/pkg/sql/function_resolver_test.go +++ b/pkg/sql/function_resolver_test.go @@ -62,7 +62,9 @@ CREATE FUNCTION f(a notmyworkday) RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT a FROM v; SELECT nextval('sq1'); $$; -CREATE FUNCTION f() RETURNS VOID IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$;`) +CREATE FUNCTION f() RETURNS VOID IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE FUNCTION f() RETURNS t IMMUTABLE LANGUAGE SQL AS $$ SELECT a, b, c FROM t $$; +`) var sessionData sessiondatapb.SessionData { @@ -88,7 +90,7 @@ CREATE FUNCTION f() RETURNS VOID IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$;`) path := sessiondata.MakeSearchPath(searchPathArray) funcDef, err := funcResolver.ResolveFunction(ctx, &fname, &path) require.NoError(t, err) - require.Equal(t, 2, len(funcDef.Overloads)) + require.Equal(t, 3, len(funcDef.Overloads)) // Verify Function Signature looks good sort.Slice(funcDef.Overloads, func(i, j int) bool { @@ -98,7 +100,7 @@ CREATE FUNCTION f() RETURNS VOID IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$;`) require.True(t, funcDef.Overloads[0].UDFContainsOnlySignature) require.True(t, funcDef.Overloads[0].IsUDF) require.Equal(t, 1, len(funcDef.Overloads[0].Types.Types())) - require.NotEqual(t, funcDef.Overloads[0].Types.Types()[0].TypeMeta, types.UserDefinedTypeMetadata{}) + require.NotZero(t, funcDef.Overloads[0].Types.Types()[0].TypeMeta) require.Equal(t, types.EnumFamily, funcDef.Overloads[0].Types.Types()[0].Family()) require.Equal(t, types.Int, funcDef.Overloads[0].ReturnType([]tree.TypedExpr{})) @@ -108,6 +110,13 @@ CREATE FUNCTION f() RETURNS VOID IMMUTABLE LANGUAGE SQL AS $$ SELECT 1 $$;`) require.Equal(t, 0, len(funcDef.Overloads[1].Types.Types())) require.Equal(t, types.Void, funcDef.Overloads[1].ReturnType([]tree.TypedExpr{})) + require.Equal(t, 100112, int(funcDef.Overloads[2].Oid)) + require.True(t, funcDef.Overloads[2].UDFContainsOnlySignature) + require.True(t, funcDef.Overloads[2].IsUDF) + require.Equal(t, 0, len(funcDef.Overloads[2].Types.Types())) + require.Equal(t, types.TupleFamily, funcDef.Overloads[2].ReturnType([]tree.TypedExpr{}).Family()) + require.NotZero(t, funcDef.Overloads[2].ReturnType([]tree.TypedExpr{}).TypeMeta) + overload, err := funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[0].Oid) require.NoError(t, err) require.Equal(t, `SELECT a FROM defaultdb.public.t; @@ -130,6 +139,15 @@ SELECT nextval(105:::REGCLASS);`, overload.Body) require.Equal(t, 0, len(overload.Types.Types())) require.Equal(t, types.Void, overload.ReturnType([]tree.TypedExpr{})) + overload, err = funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[2].Oid) + require.NoError(t, err) + require.Equal(t, `SELECT a, b, c FROM defaultdb.public.t;`, overload.Body) + require.True(t, overload.IsUDF) + require.False(t, overload.UDFContainsOnlySignature) + require.Equal(t, 0, len(overload.Types.Types())) + require.Equal(t, types.TupleFamily, overload.ReturnType([]tree.TypedExpr{}).Family()) + require.NotZero(t, overload.ReturnType([]tree.TypedExpr{}).TypeMeta) + return nil }) require.NoError(t, err) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 0f0fabc69a1b..57a4de5d088c 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -31,6 +31,7 @@ crdb_internal cluster_settings table admin NULL NULL crdb_internal cluster_statement_statistics table admin NULL NULL crdb_internal cluster_transaction_statistics table admin NULL NULL crdb_internal cluster_transactions table admin NULL NULL +crdb_internal create_function_statements table admin NULL NULL crdb_internal create_schema_statements table admin NULL NULL crdb_internal create_statements table admin NULL NULL crdb_internal create_type_statements table admin NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 8a903e66ca85..63c43a23ebc4 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -433,6 +433,23 @@ CREATE TABLE crdb_internal.cluster_transactions ( num_auto_retries INT8 NULL, last_auto_retry_reason STRING NULL ) {} {} +CREATE TABLE crdb_internal.create_function_statements ( + database_id INT8 NULL, + database_name STRING NULL, + schema_id INT8 NULL, + schema_name STRING NULL, + function_id INT8 NULL, + function_name STRING NULL, + create_statement STRING NULL +) CREATE TABLE crdb_internal.create_function_statements ( + database_id INT8 NULL, + database_name STRING NULL, + schema_id INT8 NULL, + schema_name STRING NULL, + function_id INT8 NULL, + function_name STRING NULL, + create_statement STRING NULL +) {} {} CREATE TABLE crdb_internal.create_schema_statements ( database_id INT8 NULL, database_name STRING NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 35c033eee115..8b57fbe584b7 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -44,6 +44,7 @@ test crdb_internal cluster_settings public test crdb_internal cluster_statement_statistics public SELECT false test crdb_internal cluster_transaction_statistics public SELECT false test crdb_internal cluster_transactions public SELECT false +test crdb_internal create_function_statements public SELECT false test crdb_internal create_schema_statements public SELECT false test crdb_internal create_statements public SELECT false test crdb_internal create_type_statements public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 4688f856689a..00270a1ff308 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -409,6 +409,7 @@ crdb_internal cluster_settings crdb_internal cluster_statement_statistics crdb_internal cluster_transaction_statistics crdb_internal cluster_transactions +crdb_internal create_function_statements crdb_internal create_schema_statements crdb_internal create_statements crdb_internal create_type_statements @@ -733,6 +734,7 @@ cluster_settings cluster_statement_statistics cluster_transaction_statistics cluster_transactions +create_function_statements create_schema_statements create_statements create_type_statements @@ -1096,6 +1098,7 @@ system crdb_internal cluster_settings SYSTEM system crdb_internal cluster_statement_statistics SYSTEM VIEW NO 1 system crdb_internal cluster_transaction_statistics SYSTEM VIEW NO 1 system crdb_internal cluster_transactions SYSTEM VIEW NO 1 +system crdb_internal create_function_statements SYSTEM VIEW NO 1 system crdb_internal create_schema_statements SYSTEM VIEW NO 1 system crdb_internal create_statements SYSTEM VIEW NO 1 system crdb_internal create_type_statements SYSTEM VIEW NO 1 @@ -2749,6 +2752,7 @@ NULL public system crdb_internal cluster_settings NULL public system crdb_internal cluster_statement_statistics SELECT NO YES NULL public system crdb_internal cluster_transaction_statistics SELECT NO YES NULL public system crdb_internal cluster_transactions SELECT NO YES +NULL public system crdb_internal create_function_statements SELECT NO YES NULL public system crdb_internal create_schema_statements SELECT NO YES NULL public system crdb_internal create_statements SELECT NO YES NULL public system crdb_internal create_type_statements SELECT NO YES @@ -3319,6 +3323,7 @@ NULL public system crdb_internal cluster_settings NULL public system crdb_internal cluster_statement_statistics SELECT NO YES NULL public system crdb_internal cluster_transaction_statistics SELECT NO YES NULL public system crdb_internal cluster_transactions SELECT NO YES +NULL public system crdb_internal create_function_statements SELECT NO YES NULL public system crdb_internal create_schema_statements SELECT NO YES NULL public system crdb_internal create_statements SELECT NO YES NULL public system crdb_internal create_type_statements SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index e3206df9417b..78b05e746555 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -177,42 +177,42 @@ is_updatable b 120 2 28 is_updatable c 120 3 28 false is_updatable_view a 121 1 0 false is_updatable_view b 121 2 0 false -pg_class oid 4294967125 1 0 false -pg_class relname 4294967125 2 0 false -pg_class relnamespace 4294967125 3 0 false -pg_class reltype 4294967125 4 0 false -pg_class reloftype 4294967125 5 0 false -pg_class relowner 4294967125 6 0 false -pg_class relam 4294967125 7 0 false -pg_class relfilenode 4294967125 8 0 false -pg_class reltablespace 4294967125 9 0 false -pg_class relpages 4294967125 10 0 false -pg_class reltuples 4294967125 11 0 false -pg_class relallvisible 4294967125 12 0 false -pg_class reltoastrelid 4294967125 13 0 false -pg_class relhasindex 4294967125 14 0 false -pg_class relisshared 4294967125 15 0 false -pg_class relpersistence 4294967125 16 0 false -pg_class relistemp 4294967125 17 0 false -pg_class relkind 4294967125 18 0 false -pg_class relnatts 4294967125 19 0 false -pg_class relchecks 4294967125 20 0 false -pg_class relhasoids 4294967125 21 0 false -pg_class relhaspkey 4294967125 22 0 false -pg_class relhasrules 4294967125 23 0 false -pg_class relhastriggers 4294967125 24 0 false -pg_class relhassubclass 4294967125 25 0 false -pg_class relfrozenxid 4294967125 26 0 false -pg_class relacl 4294967125 27 0 false -pg_class reloptions 4294967125 28 0 false -pg_class relforcerowsecurity 4294967125 29 0 false -pg_class relispartition 4294967125 30 0 false -pg_class relispopulated 4294967125 31 0 false -pg_class relreplident 4294967125 32 0 false -pg_class relrewrite 4294967125 33 0 false -pg_class relrowsecurity 4294967125 34 0 false -pg_class relpartbound 4294967125 35 0 false -pg_class relminmxid 4294967125 36 0 false +pg_class oid 4294967124 1 0 false +pg_class relname 4294967124 2 0 false +pg_class relnamespace 4294967124 3 0 false +pg_class reltype 4294967124 4 0 false +pg_class reloftype 4294967124 5 0 false +pg_class relowner 4294967124 6 0 false +pg_class relam 4294967124 7 0 false +pg_class relfilenode 4294967124 8 0 false +pg_class reltablespace 4294967124 9 0 false +pg_class relpages 4294967124 10 0 false +pg_class reltuples 4294967124 11 0 false +pg_class relallvisible 4294967124 12 0 false +pg_class reltoastrelid 4294967124 13 0 false +pg_class relhasindex 4294967124 14 0 false +pg_class relisshared 4294967124 15 0 false +pg_class relpersistence 4294967124 16 0 false +pg_class relistemp 4294967124 17 0 false +pg_class relkind 4294967124 18 0 false +pg_class relnatts 4294967124 19 0 false +pg_class relchecks 4294967124 20 0 false +pg_class relhasoids 4294967124 21 0 false +pg_class relhaspkey 4294967124 22 0 false +pg_class relhasrules 4294967124 23 0 false +pg_class relhastriggers 4294967124 24 0 false +pg_class relhassubclass 4294967124 25 0 false +pg_class relfrozenxid 4294967124 26 0 false +pg_class relacl 4294967124 27 0 false +pg_class reloptions 4294967124 28 0 false +pg_class relforcerowsecurity 4294967124 29 0 false +pg_class relispartition 4294967124 30 0 false +pg_class relispopulated 4294967124 31 0 false +pg_class relreplident 4294967124 32 0 false +pg_class relrewrite 4294967124 33 0 false +pg_class relrowsecurity 4294967124 34 0 false +pg_class relpartbound 4294967124 35 0 false +pg_class relminmxid 4294967124 36 0 false # Check that the oid does not exist. If this test fail, change the oid here and in diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 52f969f482be..b495f4e71197 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1479,16 +1479,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967122 111 0 4294967125 110 14 a -4294967122 112 0 4294967125 110 15 a -4294967122 192087236 0 4294967125 0 0 n -4294967079 842401391 0 4294967125 110 1 n -4294967079 842401391 0 4294967125 110 2 n -4294967079 842401391 0 4294967125 110 3 n -4294967079 842401391 0 4294967125 110 4 n -4294967122 2061447344 0 4294967125 3687884464 0 n -4294967122 3764151187 0 4294967125 0 0 n -4294967122 3836426375 0 4294967125 3687884465 0 n +4294967121 111 0 4294967124 110 14 a +4294967121 112 0 4294967124 110 15 a +4294967121 192087236 0 4294967124 0 0 n +4294967078 842401391 0 4294967124 110 1 n +4294967078 842401391 0 4294967124 110 2 n +4294967078 842401391 0 4294967124 110 3 n +4294967078 842401391 0 4294967124 110 4 n +4294967121 2061447344 0 4294967124 3687884464 0 n +4294967121 3764151187 0 4294967124 0 0 n +4294967121 3836426375 0 4294967124 3687884465 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1501,8 +1501,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967079 4294967125 pg_rewrite pg_class -4294967122 4294967125 pg_constraint pg_class +4294967078 4294967124 pg_rewrite pg_class +4294967121 4294967124 pg_constraint pg_class # Some entries in pg_depend are foreign key constraints that reference an index # in pg_class. Other entries are table-view dependencies @@ -1697,279 +1697,280 @@ oid typname typnamespace typowner typ 100132 _newtype1 3082627813 1546506610 -1 false b 100133 newtype2 3082627813 1546506610 -1 false e 100134 _newtype2 3082627813 1546506610 -1 false b -4294967004 spatial_ref_sys 1700435119 2310524507 -1 false c -4294967005 geometry_columns 1700435119 2310524507 -1 false c -4294967006 geography_columns 1700435119 2310524507 -1 false c -4294967008 pg_views 591606261 2310524507 -1 false c -4294967009 pg_user 591606261 2310524507 -1 false c -4294967010 pg_user_mappings 591606261 2310524507 -1 false c -4294967011 pg_user_mapping 591606261 2310524507 -1 false c -4294967012 pg_type 591606261 2310524507 -1 false c -4294967013 pg_ts_template 591606261 2310524507 -1 false c -4294967014 pg_ts_parser 591606261 2310524507 -1 false c -4294967015 pg_ts_dict 591606261 2310524507 -1 false c -4294967016 pg_ts_config 591606261 2310524507 -1 false c -4294967017 pg_ts_config_map 591606261 2310524507 -1 false c -4294967018 pg_trigger 591606261 2310524507 -1 false c -4294967019 pg_transform 591606261 2310524507 -1 false c -4294967020 pg_timezone_names 591606261 2310524507 -1 false c -4294967021 pg_timezone_abbrevs 591606261 2310524507 -1 false c -4294967022 pg_tablespace 591606261 2310524507 -1 false c -4294967023 pg_tables 591606261 2310524507 -1 false c -4294967024 pg_subscription 591606261 2310524507 -1 false c -4294967025 pg_subscription_rel 591606261 2310524507 -1 false c -4294967026 pg_stats 591606261 2310524507 -1 false c -4294967027 pg_stats_ext 591606261 2310524507 -1 false c -4294967028 pg_statistic 591606261 2310524507 -1 false c -4294967029 pg_statistic_ext 591606261 2310524507 -1 false c -4294967030 pg_statistic_ext_data 591606261 2310524507 -1 false c -4294967031 pg_statio_user_tables 591606261 2310524507 -1 false c -4294967032 pg_statio_user_sequences 591606261 2310524507 -1 false c -4294967033 pg_statio_user_indexes 591606261 2310524507 -1 false c -4294967034 pg_statio_sys_tables 591606261 2310524507 -1 false c -4294967035 pg_statio_sys_sequences 591606261 2310524507 -1 false c -4294967036 pg_statio_sys_indexes 591606261 2310524507 -1 false c -4294967037 pg_statio_all_tables 591606261 2310524507 -1 false c -4294967038 pg_statio_all_sequences 591606261 2310524507 -1 false c -4294967039 pg_statio_all_indexes 591606261 2310524507 -1 false c -4294967040 pg_stat_xact_user_tables 591606261 2310524507 -1 false c -4294967041 pg_stat_xact_user_functions 591606261 2310524507 -1 false c -4294967042 pg_stat_xact_sys_tables 591606261 2310524507 -1 false c -4294967043 pg_stat_xact_all_tables 591606261 2310524507 -1 false c -4294967044 pg_stat_wal_receiver 591606261 2310524507 -1 false c -4294967045 pg_stat_user_tables 591606261 2310524507 -1 false c -4294967046 pg_stat_user_indexes 591606261 2310524507 -1 false c -4294967047 pg_stat_user_functions 591606261 2310524507 -1 false c -4294967048 pg_stat_sys_tables 591606261 2310524507 -1 false c -4294967049 pg_stat_sys_indexes 591606261 2310524507 -1 false c -4294967050 pg_stat_subscription 591606261 2310524507 -1 false c -4294967051 pg_stat_ssl 591606261 2310524507 -1 false c -4294967052 pg_stat_slru 591606261 2310524507 -1 false c -4294967053 pg_stat_replication 591606261 2310524507 -1 false c -4294967054 pg_stat_progress_vacuum 591606261 2310524507 -1 false c -4294967055 pg_stat_progress_create_index 591606261 2310524507 -1 false c -4294967056 pg_stat_progress_cluster 591606261 2310524507 -1 false c -4294967057 pg_stat_progress_basebackup 591606261 2310524507 -1 false c -4294967058 pg_stat_progress_analyze 591606261 2310524507 -1 false c -4294967059 pg_stat_gssapi 591606261 2310524507 -1 false c -4294967060 pg_stat_database 591606261 2310524507 -1 false c -4294967061 pg_stat_database_conflicts 591606261 2310524507 -1 false c -4294967062 pg_stat_bgwriter 591606261 2310524507 -1 false c -4294967063 pg_stat_archiver 591606261 2310524507 -1 false c -4294967064 pg_stat_all_tables 591606261 2310524507 -1 false c -4294967065 pg_stat_all_indexes 591606261 2310524507 -1 false c -4294967066 pg_stat_activity 591606261 2310524507 -1 false c -4294967067 pg_shmem_allocations 591606261 2310524507 -1 false c -4294967068 pg_shdepend 591606261 2310524507 -1 false c -4294967069 pg_shseclabel 591606261 2310524507 -1 false c -4294967070 pg_shdescription 591606261 2310524507 -1 false c -4294967071 pg_shadow 591606261 2310524507 -1 false c -4294967072 pg_settings 591606261 2310524507 -1 false c -4294967073 pg_sequences 591606261 2310524507 -1 false c -4294967074 pg_sequence 591606261 2310524507 -1 false c -4294967075 pg_seclabel 591606261 2310524507 -1 false c -4294967076 pg_seclabels 591606261 2310524507 -1 false c -4294967077 pg_rules 591606261 2310524507 -1 false c -4294967078 pg_roles 591606261 2310524507 -1 false c -4294967079 pg_rewrite 591606261 2310524507 -1 false c -4294967080 pg_replication_slots 591606261 2310524507 -1 false c -4294967081 pg_replication_origin 591606261 2310524507 -1 false c -4294967082 pg_replication_origin_status 591606261 2310524507 -1 false c -4294967083 pg_range 591606261 2310524507 -1 false c -4294967084 pg_publication_tables 591606261 2310524507 -1 false c -4294967085 pg_publication 591606261 2310524507 -1 false c -4294967086 pg_publication_rel 591606261 2310524507 -1 false c -4294967087 pg_proc 591606261 2310524507 -1 false c -4294967088 pg_prepared_xacts 591606261 2310524507 -1 false c -4294967089 pg_prepared_statements 591606261 2310524507 -1 false c -4294967090 pg_policy 591606261 2310524507 -1 false c -4294967091 pg_policies 591606261 2310524507 -1 false c -4294967092 pg_partitioned_table 591606261 2310524507 -1 false c -4294967093 pg_opfamily 591606261 2310524507 -1 false c -4294967094 pg_operator 591606261 2310524507 -1 false c -4294967095 pg_opclass 591606261 2310524507 -1 false c -4294967096 pg_namespace 591606261 2310524507 -1 false c -4294967097 pg_matviews 591606261 2310524507 -1 false c -4294967098 pg_locks 591606261 2310524507 -1 false c -4294967099 pg_largeobject 591606261 2310524507 -1 false c -4294967100 pg_largeobject_metadata 591606261 2310524507 -1 false c -4294967101 pg_language 591606261 2310524507 -1 false c -4294967102 pg_init_privs 591606261 2310524507 -1 false c -4294967103 pg_inherits 591606261 2310524507 -1 false c -4294967104 pg_indexes 591606261 2310524507 -1 false c -4294967105 pg_index 591606261 2310524507 -1 false c -4294967106 pg_hba_file_rules 591606261 2310524507 -1 false c -4294967107 pg_group 591606261 2310524507 -1 false c -4294967108 pg_foreign_table 591606261 2310524507 -1 false c -4294967109 pg_foreign_server 591606261 2310524507 -1 false c -4294967110 pg_foreign_data_wrapper 591606261 2310524507 -1 false c -4294967111 pg_file_settings 591606261 2310524507 -1 false c -4294967112 pg_extension 591606261 2310524507 -1 false c -4294967113 pg_event_trigger 591606261 2310524507 -1 false c -4294967114 pg_enum 591606261 2310524507 -1 false c -4294967115 pg_description 591606261 2310524507 -1 false c -4294967116 pg_depend 591606261 2310524507 -1 false c -4294967117 pg_default_acl 591606261 2310524507 -1 false c -4294967118 pg_db_role_setting 591606261 2310524507 -1 false c -4294967119 pg_database 591606261 2310524507 -1 false c -4294967120 pg_cursors 591606261 2310524507 -1 false c -4294967121 pg_conversion 591606261 2310524507 -1 false c -4294967122 pg_constraint 591606261 2310524507 -1 false c -4294967123 pg_config 591606261 2310524507 -1 false c -4294967124 pg_collation 591606261 2310524507 -1 false c -4294967125 pg_class 591606261 2310524507 -1 false c -4294967126 pg_cast 591606261 2310524507 -1 false c -4294967127 pg_available_extensions 591606261 2310524507 -1 false c -4294967128 pg_available_extension_versions 591606261 2310524507 -1 false c -4294967129 pg_auth_members 591606261 2310524507 -1 false c -4294967130 pg_authid 591606261 2310524507 -1 false c -4294967131 pg_attribute 591606261 2310524507 -1 false c -4294967132 pg_attrdef 591606261 2310524507 -1 false c -4294967133 pg_amproc 591606261 2310524507 -1 false c -4294967134 pg_amop 591606261 2310524507 -1 false c -4294967135 pg_am 591606261 2310524507 -1 false c -4294967136 pg_aggregate 591606261 2310524507 -1 false c -4294967138 views 198834802 2310524507 -1 false c -4294967139 view_table_usage 198834802 2310524507 -1 false c -4294967140 view_routine_usage 198834802 2310524507 -1 false c -4294967141 view_column_usage 198834802 2310524507 -1 false c -4294967142 user_privileges 198834802 2310524507 -1 false c -4294967143 user_mappings 198834802 2310524507 -1 false c -4294967144 user_mapping_options 198834802 2310524507 -1 false c -4294967145 user_defined_types 198834802 2310524507 -1 false c -4294967146 user_attributes 198834802 2310524507 -1 false c -4294967147 usage_privileges 198834802 2310524507 -1 false c -4294967148 udt_privileges 198834802 2310524507 -1 false c -4294967149 type_privileges 198834802 2310524507 -1 false c -4294967150 triggers 198834802 2310524507 -1 false c -4294967151 triggered_update_columns 198834802 2310524507 -1 false c -4294967152 transforms 198834802 2310524507 -1 false c -4294967153 tablespaces 198834802 2310524507 -1 false c -4294967154 tablespaces_extensions 198834802 2310524507 -1 false c -4294967155 tables 198834802 2310524507 -1 false c -4294967156 tables_extensions 198834802 2310524507 -1 false c -4294967157 table_privileges 198834802 2310524507 -1 false c -4294967158 table_constraints_extensions 198834802 2310524507 -1 false c -4294967159 table_constraints 198834802 2310524507 -1 false c -4294967160 statistics 198834802 2310524507 -1 false c -4294967161 st_units_of_measure 198834802 2310524507 -1 false c -4294967162 st_spatial_reference_systems 198834802 2310524507 -1 false c -4294967163 st_geometry_columns 198834802 2310524507 -1 false c -4294967164 session_variables 198834802 2310524507 -1 false c -4294967165 sequences 198834802 2310524507 -1 false c -4294967166 schema_privileges 198834802 2310524507 -1 false c -4294967167 schemata 198834802 2310524507 -1 false c -4294967168 schemata_extensions 198834802 2310524507 -1 false c -4294967169 sql_sizing 198834802 2310524507 -1 false c -4294967170 sql_parts 198834802 2310524507 -1 false c -4294967171 sql_implementation_info 198834802 2310524507 -1 false c -4294967172 sql_features 198834802 2310524507 -1 false c -4294967173 routines 198834802 2310524507 -1 false c -4294967174 routine_privileges 198834802 2310524507 -1 false c -4294967175 role_usage_grants 198834802 2310524507 -1 false c -4294967176 role_udt_grants 198834802 2310524507 -1 false c -4294967177 role_table_grants 198834802 2310524507 -1 false c -4294967178 role_routine_grants 198834802 2310524507 -1 false c -4294967179 role_column_grants 198834802 2310524507 -1 false c -4294967180 resource_groups 198834802 2310524507 -1 false c -4294967181 referential_constraints 198834802 2310524507 -1 false c -4294967182 profiling 198834802 2310524507 -1 false c -4294967183 processlist 198834802 2310524507 -1 false c -4294967184 plugins 198834802 2310524507 -1 false c -4294967185 partitions 198834802 2310524507 -1 false c -4294967186 parameters 198834802 2310524507 -1 false c -4294967187 optimizer_trace 198834802 2310524507 -1 false c -4294967188 keywords 198834802 2310524507 -1 false c -4294967189 key_column_usage 198834802 2310524507 -1 false c -4294967190 information_schema_catalog_name 198834802 2310524507 -1 false c -4294967191 foreign_tables 198834802 2310524507 -1 false c -4294967192 foreign_table_options 198834802 2310524507 -1 false c -4294967193 foreign_servers 198834802 2310524507 -1 false c -4294967194 foreign_server_options 198834802 2310524507 -1 false c -4294967195 foreign_data_wrappers 198834802 2310524507 -1 false c -4294967196 foreign_data_wrapper_options 198834802 2310524507 -1 false c -4294967197 files 198834802 2310524507 -1 false c -4294967198 events 198834802 2310524507 -1 false c -4294967199 engines 198834802 2310524507 -1 false c -4294967200 enabled_roles 198834802 2310524507 -1 false c -4294967201 element_types 198834802 2310524507 -1 false c -4294967202 domains 198834802 2310524507 -1 false c -4294967203 domain_udt_usage 198834802 2310524507 -1 false c -4294967204 domain_constraints 198834802 2310524507 -1 false c -4294967205 data_type_privileges 198834802 2310524507 -1 false c -4294967206 constraint_table_usage 198834802 2310524507 -1 false c -4294967207 constraint_column_usage 198834802 2310524507 -1 false c -4294967208 columns 198834802 2310524507 -1 false c -4294967209 columns_extensions 198834802 2310524507 -1 false c -4294967210 column_udt_usage 198834802 2310524507 -1 false c -4294967211 column_statistics 198834802 2310524507 -1 false c -4294967212 column_privileges 198834802 2310524507 -1 false c -4294967213 column_options 198834802 2310524507 -1 false c -4294967214 column_domain_usage 198834802 2310524507 -1 false c -4294967215 column_column_usage 198834802 2310524507 -1 false c -4294967216 collations 198834802 2310524507 -1 false c -4294967217 collation_character_set_applicability 198834802 2310524507 -1 false c -4294967218 check_constraints 198834802 2310524507 -1 false c -4294967219 check_constraint_routine_usage 198834802 2310524507 -1 false c -4294967220 character_sets 198834802 2310524507 -1 false c -4294967221 attributes 198834802 2310524507 -1 false c -4294967222 applicable_roles 198834802 2310524507 -1 false c -4294967223 administrable_role_authorizations 198834802 2310524507 -1 false c -4294967225 super_regions 194902141 2310524507 -1 false c -4294967226 pg_catalog_table_is_implemented 194902141 2310524507 -1 false c -4294967227 tenant_usage_details 194902141 2310524507 -1 false c -4294967228 active_range_feeds 194902141 2310524507 -1 false c -4294967229 default_privileges 194902141 2310524507 -1 false c -4294967230 regions 194902141 2310524507 -1 false c -4294967231 cluster_inflight_traces 194902141 2310524507 -1 false c -4294967232 lost_descriptors_with_data 194902141 2310524507 -1 false c -4294967233 cross_db_references 194902141 2310524507 -1 false c -4294967234 cluster_database_privileges 194902141 2310524507 -1 false c -4294967235 invalid_objects 194902141 2310524507 -1 false c -4294967236 zones 194902141 2310524507 -1 false c -4294967237 transaction_statistics 194902141 2310524507 -1 false c -4294967238 node_transaction_statistics 194902141 2310524507 -1 false c -4294967239 table_row_statistics 194902141 2310524507 -1 false c -4294967240 tables 194902141 2310524507 -1 false c -4294967241 table_indexes 194902141 2310524507 -1 false c -4294967242 table_columns 194902141 2310524507 -1 false c -4294967243 statement_statistics 194902141 2310524507 -1 false c -4294967244 session_variables 194902141 2310524507 -1 false c -4294967245 session_trace 194902141 2310524507 -1 false c -4294967246 schema_changes 194902141 2310524507 -1 false c -4294967247 node_runtime_info 194902141 2310524507 -1 false c -4294967248 ranges 194902141 2310524507 -1 false c -4294967249 ranges_no_leases 194902141 2310524507 -1 false c -4294967250 predefined_comments 194902141 2310524507 -1 false c -4294967251 partitions 194902141 2310524507 -1 false c -4294967252 node_txn_stats 194902141 2310524507 -1 false c -4294967253 node_statement_statistics 194902141 2310524507 -1 false c -4294967254 node_metrics 194902141 2310524507 -1 false c -4294967255 node_sessions 194902141 2310524507 -1 false c -4294967256 node_transactions 194902141 2310524507 -1 false c -4294967257 node_queries 194902141 2310524507 -1 false c -4294967258 node_execution_insights 194902141 2310524507 -1 false c -4294967259 node_distsql_flows 194902141 2310524507 -1 false c -4294967260 node_contention_events 194902141 2310524507 -1 false c -4294967261 leases 194902141 2310524507 -1 false c -4294967262 kv_store_status 194902141 2310524507 -1 false c -4294967263 kv_node_status 194902141 2310524507 -1 false c -4294967264 jobs 194902141 2310524507 -1 false c -4294967265 node_inflight_trace_spans 194902141 2310524507 -1 false c -4294967266 index_usage_statistics 194902141 2310524507 -1 false c -4294967267 index_columns 194902141 2310524507 -1 false c -4294967268 transaction_contention_events 194902141 2310524507 -1 false c -4294967269 gossip_network 194902141 2310524507 -1 false c -4294967270 gossip_liveness 194902141 2310524507 -1 false c -4294967271 gossip_alerts 194902141 2310524507 -1 false c -4294967272 gossip_nodes 194902141 2310524507 -1 false c -4294967273 kv_node_liveness 194902141 2310524507 -1 false c -4294967274 forward_dependencies 194902141 2310524507 -1 false c -4294967275 feature_usage 194902141 2310524507 -1 false c -4294967276 databases 194902141 2310524507 -1 false c -4294967277 create_type_statements 194902141 2310524507 -1 false c -4294967278 create_statements 194902141 2310524507 -1 false c -4294967279 create_schema_statements 194902141 2310524507 -1 false c +4294967003 spatial_ref_sys 1700435119 2310524507 -1 false c +4294967004 geometry_columns 1700435119 2310524507 -1 false c +4294967005 geography_columns 1700435119 2310524507 -1 false c +4294967007 pg_views 591606261 2310524507 -1 false c +4294967008 pg_user 591606261 2310524507 -1 false c +4294967009 pg_user_mappings 591606261 2310524507 -1 false c +4294967010 pg_user_mapping 591606261 2310524507 -1 false c +4294967011 pg_type 591606261 2310524507 -1 false c +4294967012 pg_ts_template 591606261 2310524507 -1 false c +4294967013 pg_ts_parser 591606261 2310524507 -1 false c +4294967014 pg_ts_dict 591606261 2310524507 -1 false c +4294967015 pg_ts_config 591606261 2310524507 -1 false c +4294967016 pg_ts_config_map 591606261 2310524507 -1 false c +4294967017 pg_trigger 591606261 2310524507 -1 false c +4294967018 pg_transform 591606261 2310524507 -1 false c +4294967019 pg_timezone_names 591606261 2310524507 -1 false c +4294967020 pg_timezone_abbrevs 591606261 2310524507 -1 false c +4294967021 pg_tablespace 591606261 2310524507 -1 false c +4294967022 pg_tables 591606261 2310524507 -1 false c +4294967023 pg_subscription 591606261 2310524507 -1 false c +4294967024 pg_subscription_rel 591606261 2310524507 -1 false c +4294967025 pg_stats 591606261 2310524507 -1 false c +4294967026 pg_stats_ext 591606261 2310524507 -1 false c +4294967027 pg_statistic 591606261 2310524507 -1 false c +4294967028 pg_statistic_ext 591606261 2310524507 -1 false c +4294967029 pg_statistic_ext_data 591606261 2310524507 -1 false c +4294967030 pg_statio_user_tables 591606261 2310524507 -1 false c +4294967031 pg_statio_user_sequences 591606261 2310524507 -1 false c +4294967032 pg_statio_user_indexes 591606261 2310524507 -1 false c +4294967033 pg_statio_sys_tables 591606261 2310524507 -1 false c +4294967034 pg_statio_sys_sequences 591606261 2310524507 -1 false c +4294967035 pg_statio_sys_indexes 591606261 2310524507 -1 false c +4294967036 pg_statio_all_tables 591606261 2310524507 -1 false c +4294967037 pg_statio_all_sequences 591606261 2310524507 -1 false c +4294967038 pg_statio_all_indexes 591606261 2310524507 -1 false c +4294967039 pg_stat_xact_user_tables 591606261 2310524507 -1 false c +4294967040 pg_stat_xact_user_functions 591606261 2310524507 -1 false c +4294967041 pg_stat_xact_sys_tables 591606261 2310524507 -1 false c +4294967042 pg_stat_xact_all_tables 591606261 2310524507 -1 false c +4294967043 pg_stat_wal_receiver 591606261 2310524507 -1 false c +4294967044 pg_stat_user_tables 591606261 2310524507 -1 false c +4294967045 pg_stat_user_indexes 591606261 2310524507 -1 false c +4294967046 pg_stat_user_functions 591606261 2310524507 -1 false c +4294967047 pg_stat_sys_tables 591606261 2310524507 -1 false c +4294967048 pg_stat_sys_indexes 591606261 2310524507 -1 false c +4294967049 pg_stat_subscription 591606261 2310524507 -1 false c +4294967050 pg_stat_ssl 591606261 2310524507 -1 false c +4294967051 pg_stat_slru 591606261 2310524507 -1 false c +4294967052 pg_stat_replication 591606261 2310524507 -1 false c +4294967053 pg_stat_progress_vacuum 591606261 2310524507 -1 false c +4294967054 pg_stat_progress_create_index 591606261 2310524507 -1 false c +4294967055 pg_stat_progress_cluster 591606261 2310524507 -1 false c +4294967056 pg_stat_progress_basebackup 591606261 2310524507 -1 false c +4294967057 pg_stat_progress_analyze 591606261 2310524507 -1 false c +4294967058 pg_stat_gssapi 591606261 2310524507 -1 false c +4294967059 pg_stat_database 591606261 2310524507 -1 false c +4294967060 pg_stat_database_conflicts 591606261 2310524507 -1 false c +4294967061 pg_stat_bgwriter 591606261 2310524507 -1 false c +4294967062 pg_stat_archiver 591606261 2310524507 -1 false c +4294967063 pg_stat_all_tables 591606261 2310524507 -1 false c +4294967064 pg_stat_all_indexes 591606261 2310524507 -1 false c +4294967065 pg_stat_activity 591606261 2310524507 -1 false c +4294967066 pg_shmem_allocations 591606261 2310524507 -1 false c +4294967067 pg_shdepend 591606261 2310524507 -1 false c +4294967068 pg_shseclabel 591606261 2310524507 -1 false c +4294967069 pg_shdescription 591606261 2310524507 -1 false c +4294967070 pg_shadow 591606261 2310524507 -1 false c +4294967071 pg_settings 591606261 2310524507 -1 false c +4294967072 pg_sequences 591606261 2310524507 -1 false c +4294967073 pg_sequence 591606261 2310524507 -1 false c +4294967074 pg_seclabel 591606261 2310524507 -1 false c +4294967075 pg_seclabels 591606261 2310524507 -1 false c +4294967076 pg_rules 591606261 2310524507 -1 false c +4294967077 pg_roles 591606261 2310524507 -1 false c +4294967078 pg_rewrite 591606261 2310524507 -1 false c +4294967079 pg_replication_slots 591606261 2310524507 -1 false c +4294967080 pg_replication_origin 591606261 2310524507 -1 false c +4294967081 pg_replication_origin_status 591606261 2310524507 -1 false c +4294967082 pg_range 591606261 2310524507 -1 false c +4294967083 pg_publication_tables 591606261 2310524507 -1 false c +4294967084 pg_publication 591606261 2310524507 -1 false c +4294967085 pg_publication_rel 591606261 2310524507 -1 false c +4294967086 pg_proc 591606261 2310524507 -1 false c +4294967087 pg_prepared_xacts 591606261 2310524507 -1 false c +4294967088 pg_prepared_statements 591606261 2310524507 -1 false c +4294967089 pg_policy 591606261 2310524507 -1 false c +4294967090 pg_policies 591606261 2310524507 -1 false c +4294967091 pg_partitioned_table 591606261 2310524507 -1 false c +4294967092 pg_opfamily 591606261 2310524507 -1 false c +4294967093 pg_operator 591606261 2310524507 -1 false c +4294967094 pg_opclass 591606261 2310524507 -1 false c +4294967095 pg_namespace 591606261 2310524507 -1 false c +4294967096 pg_matviews 591606261 2310524507 -1 false c +4294967097 pg_locks 591606261 2310524507 -1 false c +4294967098 pg_largeobject 591606261 2310524507 -1 false c +4294967099 pg_largeobject_metadata 591606261 2310524507 -1 false c +4294967100 pg_language 591606261 2310524507 -1 false c +4294967101 pg_init_privs 591606261 2310524507 -1 false c +4294967102 pg_inherits 591606261 2310524507 -1 false c +4294967103 pg_indexes 591606261 2310524507 -1 false c +4294967104 pg_index 591606261 2310524507 -1 false c +4294967105 pg_hba_file_rules 591606261 2310524507 -1 false c +4294967106 pg_group 591606261 2310524507 -1 false c +4294967107 pg_foreign_table 591606261 2310524507 -1 false c +4294967108 pg_foreign_server 591606261 2310524507 -1 false c +4294967109 pg_foreign_data_wrapper 591606261 2310524507 -1 false c +4294967110 pg_file_settings 591606261 2310524507 -1 false c +4294967111 pg_extension 591606261 2310524507 -1 false c +4294967112 pg_event_trigger 591606261 2310524507 -1 false c +4294967113 pg_enum 591606261 2310524507 -1 false c +4294967114 pg_description 591606261 2310524507 -1 false c +4294967115 pg_depend 591606261 2310524507 -1 false c +4294967116 pg_default_acl 591606261 2310524507 -1 false c +4294967117 pg_db_role_setting 591606261 2310524507 -1 false c +4294967118 pg_database 591606261 2310524507 -1 false c +4294967119 pg_cursors 591606261 2310524507 -1 false c +4294967120 pg_conversion 591606261 2310524507 -1 false c +4294967121 pg_constraint 591606261 2310524507 -1 false c +4294967122 pg_config 591606261 2310524507 -1 false c +4294967123 pg_collation 591606261 2310524507 -1 false c +4294967124 pg_class 591606261 2310524507 -1 false c +4294967125 pg_cast 591606261 2310524507 -1 false c +4294967126 pg_available_extensions 591606261 2310524507 -1 false c +4294967127 pg_available_extension_versions 591606261 2310524507 -1 false c +4294967128 pg_auth_members 591606261 2310524507 -1 false c +4294967129 pg_authid 591606261 2310524507 -1 false c +4294967130 pg_attribute 591606261 2310524507 -1 false c +4294967131 pg_attrdef 591606261 2310524507 -1 false c +4294967132 pg_amproc 591606261 2310524507 -1 false c +4294967133 pg_amop 591606261 2310524507 -1 false c +4294967134 pg_am 591606261 2310524507 -1 false c +4294967135 pg_aggregate 591606261 2310524507 -1 false c +4294967137 views 198834802 2310524507 -1 false c +4294967138 view_table_usage 198834802 2310524507 -1 false c +4294967139 view_routine_usage 198834802 2310524507 -1 false c +4294967140 view_column_usage 198834802 2310524507 -1 false c +4294967141 user_privileges 198834802 2310524507 -1 false c +4294967142 user_mappings 198834802 2310524507 -1 false c +4294967143 user_mapping_options 198834802 2310524507 -1 false c +4294967144 user_defined_types 198834802 2310524507 -1 false c +4294967145 user_attributes 198834802 2310524507 -1 false c +4294967146 usage_privileges 198834802 2310524507 -1 false c +4294967147 udt_privileges 198834802 2310524507 -1 false c +4294967148 type_privileges 198834802 2310524507 -1 false c +4294967149 triggers 198834802 2310524507 -1 false c +4294967150 triggered_update_columns 198834802 2310524507 -1 false c +4294967151 transforms 198834802 2310524507 -1 false c +4294967152 tablespaces 198834802 2310524507 -1 false c +4294967153 tablespaces_extensions 198834802 2310524507 -1 false c +4294967154 tables 198834802 2310524507 -1 false c +4294967155 tables_extensions 198834802 2310524507 -1 false c +4294967156 table_privileges 198834802 2310524507 -1 false c +4294967157 table_constraints_extensions 198834802 2310524507 -1 false c +4294967158 table_constraints 198834802 2310524507 -1 false c +4294967159 statistics 198834802 2310524507 -1 false c +4294967160 st_units_of_measure 198834802 2310524507 -1 false c +4294967161 st_spatial_reference_systems 198834802 2310524507 -1 false c +4294967162 st_geometry_columns 198834802 2310524507 -1 false c +4294967163 session_variables 198834802 2310524507 -1 false c +4294967164 sequences 198834802 2310524507 -1 false c +4294967165 schema_privileges 198834802 2310524507 -1 false c +4294967166 schemata 198834802 2310524507 -1 false c +4294967167 schemata_extensions 198834802 2310524507 -1 false c +4294967168 sql_sizing 198834802 2310524507 -1 false c +4294967169 sql_parts 198834802 2310524507 -1 false c +4294967170 sql_implementation_info 198834802 2310524507 -1 false c +4294967171 sql_features 198834802 2310524507 -1 false c +4294967172 routines 198834802 2310524507 -1 false c +4294967173 routine_privileges 198834802 2310524507 -1 false c +4294967174 role_usage_grants 198834802 2310524507 -1 false c +4294967175 role_udt_grants 198834802 2310524507 -1 false c +4294967176 role_table_grants 198834802 2310524507 -1 false c +4294967177 role_routine_grants 198834802 2310524507 -1 false c +4294967178 role_column_grants 198834802 2310524507 -1 false c +4294967179 resource_groups 198834802 2310524507 -1 false c +4294967180 referential_constraints 198834802 2310524507 -1 false c +4294967181 profiling 198834802 2310524507 -1 false c +4294967182 processlist 198834802 2310524507 -1 false c +4294967183 plugins 198834802 2310524507 -1 false c +4294967184 partitions 198834802 2310524507 -1 false c +4294967185 parameters 198834802 2310524507 -1 false c +4294967186 optimizer_trace 198834802 2310524507 -1 false c +4294967187 keywords 198834802 2310524507 -1 false c +4294967188 key_column_usage 198834802 2310524507 -1 false c +4294967189 information_schema_catalog_name 198834802 2310524507 -1 false c +4294967190 foreign_tables 198834802 2310524507 -1 false c +4294967191 foreign_table_options 198834802 2310524507 -1 false c +4294967192 foreign_servers 198834802 2310524507 -1 false c +4294967193 foreign_server_options 198834802 2310524507 -1 false c +4294967194 foreign_data_wrappers 198834802 2310524507 -1 false c +4294967195 foreign_data_wrapper_options 198834802 2310524507 -1 false c +4294967196 files 198834802 2310524507 -1 false c +4294967197 events 198834802 2310524507 -1 false c +4294967198 engines 198834802 2310524507 -1 false c +4294967199 enabled_roles 198834802 2310524507 -1 false c +4294967200 element_types 198834802 2310524507 -1 false c +4294967201 domains 198834802 2310524507 -1 false c +4294967202 domain_udt_usage 198834802 2310524507 -1 false c +4294967203 domain_constraints 198834802 2310524507 -1 false c +4294967204 data_type_privileges 198834802 2310524507 -1 false c +4294967205 constraint_table_usage 198834802 2310524507 -1 false c +4294967206 constraint_column_usage 198834802 2310524507 -1 false c +4294967207 columns 198834802 2310524507 -1 false c +4294967208 columns_extensions 198834802 2310524507 -1 false c +4294967209 column_udt_usage 198834802 2310524507 -1 false c +4294967210 column_statistics 198834802 2310524507 -1 false c +4294967211 column_privileges 198834802 2310524507 -1 false c +4294967212 column_options 198834802 2310524507 -1 false c +4294967213 column_domain_usage 198834802 2310524507 -1 false c +4294967214 column_column_usage 198834802 2310524507 -1 false c +4294967215 collations 198834802 2310524507 -1 false c +4294967216 collation_character_set_applicability 198834802 2310524507 -1 false c +4294967217 check_constraints 198834802 2310524507 -1 false c +4294967218 check_constraint_routine_usage 198834802 2310524507 -1 false c +4294967219 character_sets 198834802 2310524507 -1 false c +4294967220 attributes 198834802 2310524507 -1 false c +4294967221 applicable_roles 198834802 2310524507 -1 false c +4294967222 administrable_role_authorizations 198834802 2310524507 -1 false c +4294967224 super_regions 194902141 2310524507 -1 false c +4294967225 pg_catalog_table_is_implemented 194902141 2310524507 -1 false c +4294967226 tenant_usage_details 194902141 2310524507 -1 false c +4294967227 active_range_feeds 194902141 2310524507 -1 false c +4294967228 default_privileges 194902141 2310524507 -1 false c +4294967229 regions 194902141 2310524507 -1 false c +4294967230 cluster_inflight_traces 194902141 2310524507 -1 false c +4294967231 lost_descriptors_with_data 194902141 2310524507 -1 false c +4294967232 cross_db_references 194902141 2310524507 -1 false c +4294967233 cluster_database_privileges 194902141 2310524507 -1 false c +4294967234 invalid_objects 194902141 2310524507 -1 false c +4294967235 zones 194902141 2310524507 -1 false c +4294967236 transaction_statistics 194902141 2310524507 -1 false c +4294967237 node_transaction_statistics 194902141 2310524507 -1 false c +4294967238 table_row_statistics 194902141 2310524507 -1 false c +4294967239 tables 194902141 2310524507 -1 false c +4294967240 table_indexes 194902141 2310524507 -1 false c +4294967241 table_columns 194902141 2310524507 -1 false c +4294967242 statement_statistics 194902141 2310524507 -1 false c +4294967243 session_variables 194902141 2310524507 -1 false c +4294967244 session_trace 194902141 2310524507 -1 false c +4294967245 schema_changes 194902141 2310524507 -1 false c +4294967246 node_runtime_info 194902141 2310524507 -1 false c +4294967247 ranges 194902141 2310524507 -1 false c +4294967248 ranges_no_leases 194902141 2310524507 -1 false c +4294967249 predefined_comments 194902141 2310524507 -1 false c +4294967250 partitions 194902141 2310524507 -1 false c +4294967251 node_txn_stats 194902141 2310524507 -1 false c +4294967252 node_statement_statistics 194902141 2310524507 -1 false c +4294967253 node_metrics 194902141 2310524507 -1 false c +4294967254 node_sessions 194902141 2310524507 -1 false c +4294967255 node_transactions 194902141 2310524507 -1 false c +4294967256 node_queries 194902141 2310524507 -1 false c +4294967257 node_execution_insights 194902141 2310524507 -1 false c +4294967258 node_distsql_flows 194902141 2310524507 -1 false c +4294967259 node_contention_events 194902141 2310524507 -1 false c +4294967260 leases 194902141 2310524507 -1 false c +4294967261 kv_store_status 194902141 2310524507 -1 false c +4294967262 kv_node_status 194902141 2310524507 -1 false c +4294967263 jobs 194902141 2310524507 -1 false c +4294967264 node_inflight_trace_spans 194902141 2310524507 -1 false c +4294967265 index_usage_statistics 194902141 2310524507 -1 false c +4294967266 index_columns 194902141 2310524507 -1 false c +4294967267 transaction_contention_events 194902141 2310524507 -1 false c +4294967268 gossip_network 194902141 2310524507 -1 false c +4294967269 gossip_liveness 194902141 2310524507 -1 false c +4294967270 gossip_alerts 194902141 2310524507 -1 false c +4294967271 gossip_nodes 194902141 2310524507 -1 false c +4294967272 kv_node_liveness 194902141 2310524507 -1 false c +4294967273 forward_dependencies 194902141 2310524507 -1 false c +4294967274 feature_usage 194902141 2310524507 -1 false c +4294967275 databases 194902141 2310524507 -1 false c +4294967276 create_type_statements 194902141 2310524507 -1 false c +4294967277 create_statements 194902141 2310524507 -1 false c +4294967278 create_schema_statements 194902141 2310524507 -1 false c +4294967279 create_function_statements 194902141 2310524507 -1 false c 4294967280 cluster_transaction_statistics 194902141 2310524507 -1 false c 4294967281 cluster_statement_statistics 194902141 2310524507 -1 false c 4294967282 cluster_settings 194902141 2310524507 -1 false c @@ -2089,279 +2090,280 @@ oid typname typcategory typispreferred 100132 _newtype1 A false true , 0 100131 0 100133 newtype2 E false true , 0 0 100134 100134 _newtype2 A false true , 0 100133 0 -4294967004 spatial_ref_sys C false true , 4294967004 0 0 -4294967005 geometry_columns C false true , 4294967005 0 0 -4294967006 geography_columns C false true , 4294967006 0 0 -4294967008 pg_views C false true , 4294967008 0 0 -4294967009 pg_user C false true , 4294967009 0 0 -4294967010 pg_user_mappings C false true , 4294967010 0 0 -4294967011 pg_user_mapping C false true , 4294967011 0 0 -4294967012 pg_type C false true , 4294967012 0 0 -4294967013 pg_ts_template C false true , 4294967013 0 0 -4294967014 pg_ts_parser C false true , 4294967014 0 0 -4294967015 pg_ts_dict C false true , 4294967015 0 0 -4294967016 pg_ts_config C false true , 4294967016 0 0 -4294967017 pg_ts_config_map C false true , 4294967017 0 0 -4294967018 pg_trigger C false true , 4294967018 0 0 -4294967019 pg_transform C false true , 4294967019 0 0 -4294967020 pg_timezone_names C false true , 4294967020 0 0 -4294967021 pg_timezone_abbrevs C false true , 4294967021 0 0 -4294967022 pg_tablespace C false true , 4294967022 0 0 -4294967023 pg_tables C false true , 4294967023 0 0 -4294967024 pg_subscription C false true , 4294967024 0 0 -4294967025 pg_subscription_rel C false true , 4294967025 0 0 -4294967026 pg_stats C false true , 4294967026 0 0 -4294967027 pg_stats_ext C false true , 4294967027 0 0 -4294967028 pg_statistic C false true , 4294967028 0 0 -4294967029 pg_statistic_ext C false true , 4294967029 0 0 -4294967030 pg_statistic_ext_data C false true , 4294967030 0 0 -4294967031 pg_statio_user_tables C false true , 4294967031 0 0 -4294967032 pg_statio_user_sequences C false true , 4294967032 0 0 -4294967033 pg_statio_user_indexes C false true , 4294967033 0 0 -4294967034 pg_statio_sys_tables C false true , 4294967034 0 0 -4294967035 pg_statio_sys_sequences C false true , 4294967035 0 0 -4294967036 pg_statio_sys_indexes C false true , 4294967036 0 0 -4294967037 pg_statio_all_tables C false true , 4294967037 0 0 -4294967038 pg_statio_all_sequences C false true , 4294967038 0 0 -4294967039 pg_statio_all_indexes C false true , 4294967039 0 0 -4294967040 pg_stat_xact_user_tables C false true , 4294967040 0 0 -4294967041 pg_stat_xact_user_functions C false true , 4294967041 0 0 -4294967042 pg_stat_xact_sys_tables C false true , 4294967042 0 0 -4294967043 pg_stat_xact_all_tables C false true , 4294967043 0 0 -4294967044 pg_stat_wal_receiver C false true , 4294967044 0 0 -4294967045 pg_stat_user_tables C false true , 4294967045 0 0 -4294967046 pg_stat_user_indexes C false true , 4294967046 0 0 -4294967047 pg_stat_user_functions C false true , 4294967047 0 0 -4294967048 pg_stat_sys_tables C false true , 4294967048 0 0 -4294967049 pg_stat_sys_indexes C false true , 4294967049 0 0 -4294967050 pg_stat_subscription C false true , 4294967050 0 0 -4294967051 pg_stat_ssl C false true , 4294967051 0 0 -4294967052 pg_stat_slru C false true , 4294967052 0 0 -4294967053 pg_stat_replication C false true , 4294967053 0 0 -4294967054 pg_stat_progress_vacuum C false true , 4294967054 0 0 -4294967055 pg_stat_progress_create_index C false true , 4294967055 0 0 -4294967056 pg_stat_progress_cluster C false true , 4294967056 0 0 -4294967057 pg_stat_progress_basebackup C false true , 4294967057 0 0 -4294967058 pg_stat_progress_analyze C false true , 4294967058 0 0 -4294967059 pg_stat_gssapi C false true , 4294967059 0 0 -4294967060 pg_stat_database C false true , 4294967060 0 0 -4294967061 pg_stat_database_conflicts C false true , 4294967061 0 0 -4294967062 pg_stat_bgwriter C false true , 4294967062 0 0 -4294967063 pg_stat_archiver C false true , 4294967063 0 0 -4294967064 pg_stat_all_tables C false true , 4294967064 0 0 -4294967065 pg_stat_all_indexes C false true , 4294967065 0 0 -4294967066 pg_stat_activity C false true , 4294967066 0 0 -4294967067 pg_shmem_allocations C false true , 4294967067 0 0 -4294967068 pg_shdepend C false true , 4294967068 0 0 -4294967069 pg_shseclabel C false true , 4294967069 0 0 -4294967070 pg_shdescription C false true , 4294967070 0 0 -4294967071 pg_shadow C false true , 4294967071 0 0 -4294967072 pg_settings C false true , 4294967072 0 0 -4294967073 pg_sequences C false true , 4294967073 0 0 -4294967074 pg_sequence C false true , 4294967074 0 0 -4294967075 pg_seclabel C false true , 4294967075 0 0 -4294967076 pg_seclabels C false true , 4294967076 0 0 -4294967077 pg_rules C false true , 4294967077 0 0 -4294967078 pg_roles C false true , 4294967078 0 0 -4294967079 pg_rewrite C false true , 4294967079 0 0 -4294967080 pg_replication_slots C false true , 4294967080 0 0 -4294967081 pg_replication_origin C false true , 4294967081 0 0 -4294967082 pg_replication_origin_status C false true , 4294967082 0 0 -4294967083 pg_range C false true , 4294967083 0 0 -4294967084 pg_publication_tables C false true , 4294967084 0 0 -4294967085 pg_publication C false true , 4294967085 0 0 -4294967086 pg_publication_rel C false true , 4294967086 0 0 -4294967087 pg_proc C false true , 4294967087 0 0 -4294967088 pg_prepared_xacts C false true , 4294967088 0 0 -4294967089 pg_prepared_statements C false true , 4294967089 0 0 -4294967090 pg_policy C false true , 4294967090 0 0 -4294967091 pg_policies C false true , 4294967091 0 0 -4294967092 pg_partitioned_table C false true , 4294967092 0 0 -4294967093 pg_opfamily C false true , 4294967093 0 0 -4294967094 pg_operator C false true , 4294967094 0 0 -4294967095 pg_opclass C false true , 4294967095 0 0 -4294967096 pg_namespace C false true , 4294967096 0 0 -4294967097 pg_matviews C false true , 4294967097 0 0 -4294967098 pg_locks C false true , 4294967098 0 0 -4294967099 pg_largeobject C false true , 4294967099 0 0 -4294967100 pg_largeobject_metadata C false true , 4294967100 0 0 -4294967101 pg_language C false true , 4294967101 0 0 -4294967102 pg_init_privs C false true , 4294967102 0 0 -4294967103 pg_inherits C false true , 4294967103 0 0 -4294967104 pg_indexes C false true , 4294967104 0 0 -4294967105 pg_index C false true , 4294967105 0 0 -4294967106 pg_hba_file_rules C false true , 4294967106 0 0 -4294967107 pg_group C false true , 4294967107 0 0 -4294967108 pg_foreign_table C false true , 4294967108 0 0 -4294967109 pg_foreign_server C false true , 4294967109 0 0 -4294967110 pg_foreign_data_wrapper C false true , 4294967110 0 0 -4294967111 pg_file_settings C false true , 4294967111 0 0 -4294967112 pg_extension C false true , 4294967112 0 0 -4294967113 pg_event_trigger C false true , 4294967113 0 0 -4294967114 pg_enum C false true , 4294967114 0 0 -4294967115 pg_description C false true , 4294967115 0 0 -4294967116 pg_depend C false true , 4294967116 0 0 -4294967117 pg_default_acl C false true , 4294967117 0 0 -4294967118 pg_db_role_setting C false true , 4294967118 0 0 -4294967119 pg_database C false true , 4294967119 0 0 -4294967120 pg_cursors C false true , 4294967120 0 0 -4294967121 pg_conversion C false true , 4294967121 0 0 -4294967122 pg_constraint C false true , 4294967122 0 0 -4294967123 pg_config C false true , 4294967123 0 0 -4294967124 pg_collation C false true , 4294967124 0 0 -4294967125 pg_class C false true , 4294967125 0 0 -4294967126 pg_cast C false true , 4294967126 0 0 -4294967127 pg_available_extensions C false true , 4294967127 0 0 -4294967128 pg_available_extension_versions C false true , 4294967128 0 0 -4294967129 pg_auth_members C false true , 4294967129 0 0 -4294967130 pg_authid C false true , 4294967130 0 0 -4294967131 pg_attribute C false true , 4294967131 0 0 -4294967132 pg_attrdef C false true , 4294967132 0 0 -4294967133 pg_amproc C false true , 4294967133 0 0 -4294967134 pg_amop C false true , 4294967134 0 0 -4294967135 pg_am C false true , 4294967135 0 0 -4294967136 pg_aggregate C false true , 4294967136 0 0 -4294967138 views C false true , 4294967138 0 0 -4294967139 view_table_usage C false true , 4294967139 0 0 -4294967140 view_routine_usage C false true , 4294967140 0 0 -4294967141 view_column_usage C false true , 4294967141 0 0 -4294967142 user_privileges C false true , 4294967142 0 0 -4294967143 user_mappings C false true , 4294967143 0 0 -4294967144 user_mapping_options C false true , 4294967144 0 0 -4294967145 user_defined_types C false true , 4294967145 0 0 -4294967146 user_attributes C false true , 4294967146 0 0 -4294967147 usage_privileges C false true , 4294967147 0 0 -4294967148 udt_privileges C false true , 4294967148 0 0 -4294967149 type_privileges C false true , 4294967149 0 0 -4294967150 triggers C false true , 4294967150 0 0 -4294967151 triggered_update_columns C false true , 4294967151 0 0 -4294967152 transforms C false true , 4294967152 0 0 -4294967153 tablespaces C false true , 4294967153 0 0 -4294967154 tablespaces_extensions C false true , 4294967154 0 0 -4294967155 tables C false true , 4294967155 0 0 -4294967156 tables_extensions C false true , 4294967156 0 0 -4294967157 table_privileges C false true , 4294967157 0 0 -4294967158 table_constraints_extensions C false true , 4294967158 0 0 -4294967159 table_constraints C false true , 4294967159 0 0 -4294967160 statistics C false true , 4294967160 0 0 -4294967161 st_units_of_measure C false true , 4294967161 0 0 -4294967162 st_spatial_reference_systems C false true , 4294967162 0 0 -4294967163 st_geometry_columns C false true , 4294967163 0 0 -4294967164 session_variables C false true , 4294967164 0 0 -4294967165 sequences C false true , 4294967165 0 0 -4294967166 schema_privileges C false true , 4294967166 0 0 -4294967167 schemata C false true , 4294967167 0 0 -4294967168 schemata_extensions C false true , 4294967168 0 0 -4294967169 sql_sizing C false true , 4294967169 0 0 -4294967170 sql_parts C false true , 4294967170 0 0 -4294967171 sql_implementation_info C false true , 4294967171 0 0 -4294967172 sql_features C false true , 4294967172 0 0 -4294967173 routines C false true , 4294967173 0 0 -4294967174 routine_privileges C false true , 4294967174 0 0 -4294967175 role_usage_grants C false true , 4294967175 0 0 -4294967176 role_udt_grants C false true , 4294967176 0 0 -4294967177 role_table_grants C false true , 4294967177 0 0 -4294967178 role_routine_grants C false true , 4294967178 0 0 -4294967179 role_column_grants C false true , 4294967179 0 0 -4294967180 resource_groups C false true , 4294967180 0 0 -4294967181 referential_constraints C false true , 4294967181 0 0 -4294967182 profiling C false true , 4294967182 0 0 -4294967183 processlist C false true , 4294967183 0 0 -4294967184 plugins C false true , 4294967184 0 0 -4294967185 partitions C false true , 4294967185 0 0 -4294967186 parameters C false true , 4294967186 0 0 -4294967187 optimizer_trace C false true , 4294967187 0 0 -4294967188 keywords C false true , 4294967188 0 0 -4294967189 key_column_usage C false true , 4294967189 0 0 -4294967190 information_schema_catalog_name C false true , 4294967190 0 0 -4294967191 foreign_tables C false true , 4294967191 0 0 -4294967192 foreign_table_options C false true , 4294967192 0 0 -4294967193 foreign_servers C false true , 4294967193 0 0 -4294967194 foreign_server_options C false true , 4294967194 0 0 -4294967195 foreign_data_wrappers C false true , 4294967195 0 0 -4294967196 foreign_data_wrapper_options C false true , 4294967196 0 0 -4294967197 files C false true , 4294967197 0 0 -4294967198 events C false true , 4294967198 0 0 -4294967199 engines C false true , 4294967199 0 0 -4294967200 enabled_roles C false true , 4294967200 0 0 -4294967201 element_types C false true , 4294967201 0 0 -4294967202 domains C false true , 4294967202 0 0 -4294967203 domain_udt_usage C false true , 4294967203 0 0 -4294967204 domain_constraints C false true , 4294967204 0 0 -4294967205 data_type_privileges C false true , 4294967205 0 0 -4294967206 constraint_table_usage C false true , 4294967206 0 0 -4294967207 constraint_column_usage C false true , 4294967207 0 0 -4294967208 columns C false true , 4294967208 0 0 -4294967209 columns_extensions C false true , 4294967209 0 0 -4294967210 column_udt_usage C false true , 4294967210 0 0 -4294967211 column_statistics C false true , 4294967211 0 0 -4294967212 column_privileges C false true , 4294967212 0 0 -4294967213 column_options C false true , 4294967213 0 0 -4294967214 column_domain_usage C false true , 4294967214 0 0 -4294967215 column_column_usage C false true , 4294967215 0 0 -4294967216 collations C false true , 4294967216 0 0 -4294967217 collation_character_set_applicability C false true , 4294967217 0 0 -4294967218 check_constraints C false true , 4294967218 0 0 -4294967219 check_constraint_routine_usage C false true , 4294967219 0 0 -4294967220 character_sets C false true , 4294967220 0 0 -4294967221 attributes C false true , 4294967221 0 0 -4294967222 applicable_roles C false true , 4294967222 0 0 -4294967223 administrable_role_authorizations C false true , 4294967223 0 0 -4294967225 super_regions C false true , 4294967225 0 0 -4294967226 pg_catalog_table_is_implemented C false true , 4294967226 0 0 -4294967227 tenant_usage_details C false true , 4294967227 0 0 -4294967228 active_range_feeds C false true , 4294967228 0 0 -4294967229 default_privileges C false true , 4294967229 0 0 -4294967230 regions C false true , 4294967230 0 0 -4294967231 cluster_inflight_traces C false true , 4294967231 0 0 -4294967232 lost_descriptors_with_data C false true , 4294967232 0 0 -4294967233 cross_db_references C false true , 4294967233 0 0 -4294967234 cluster_database_privileges C false true , 4294967234 0 0 -4294967235 invalid_objects C false true , 4294967235 0 0 -4294967236 zones C false true , 4294967236 0 0 -4294967237 transaction_statistics C false true , 4294967237 0 0 -4294967238 node_transaction_statistics C false true , 4294967238 0 0 -4294967239 table_row_statistics C false true , 4294967239 0 0 -4294967240 tables C false true , 4294967240 0 0 -4294967241 table_indexes C false true , 4294967241 0 0 -4294967242 table_columns C false true , 4294967242 0 0 -4294967243 statement_statistics C false true , 4294967243 0 0 -4294967244 session_variables C false true , 4294967244 0 0 -4294967245 session_trace C false true , 4294967245 0 0 -4294967246 schema_changes C false true , 4294967246 0 0 -4294967247 node_runtime_info C false true , 4294967247 0 0 -4294967248 ranges C false true , 4294967248 0 0 -4294967249 ranges_no_leases C false true , 4294967249 0 0 -4294967250 predefined_comments C false true , 4294967250 0 0 -4294967251 partitions C false true , 4294967251 0 0 -4294967252 node_txn_stats C false true , 4294967252 0 0 -4294967253 node_statement_statistics C false true , 4294967253 0 0 -4294967254 node_metrics C false true , 4294967254 0 0 -4294967255 node_sessions C false true , 4294967255 0 0 -4294967256 node_transactions C false true , 4294967256 0 0 -4294967257 node_queries C false true , 4294967257 0 0 -4294967258 node_execution_insights C false true , 4294967258 0 0 -4294967259 node_distsql_flows C false true , 4294967259 0 0 -4294967260 node_contention_events C false true , 4294967260 0 0 -4294967261 leases C false true , 4294967261 0 0 -4294967262 kv_store_status C false true , 4294967262 0 0 -4294967263 kv_node_status C false true , 4294967263 0 0 -4294967264 jobs C false true , 4294967264 0 0 -4294967265 node_inflight_trace_spans C false true , 4294967265 0 0 -4294967266 index_usage_statistics C false true , 4294967266 0 0 -4294967267 index_columns C false true , 4294967267 0 0 -4294967268 transaction_contention_events C false true , 4294967268 0 0 -4294967269 gossip_network C false true , 4294967269 0 0 -4294967270 gossip_liveness C false true , 4294967270 0 0 -4294967271 gossip_alerts C false true , 4294967271 0 0 -4294967272 gossip_nodes C false true , 4294967272 0 0 -4294967273 kv_node_liveness C false true , 4294967273 0 0 -4294967274 forward_dependencies C false true , 4294967274 0 0 -4294967275 feature_usage C false true , 4294967275 0 0 -4294967276 databases C false true , 4294967276 0 0 -4294967277 create_type_statements C false true , 4294967277 0 0 -4294967278 create_statements C false true , 4294967278 0 0 -4294967279 create_schema_statements C false true , 4294967279 0 0 +4294967003 spatial_ref_sys C false true , 4294967003 0 0 +4294967004 geometry_columns C false true , 4294967004 0 0 +4294967005 geography_columns C false true , 4294967005 0 0 +4294967007 pg_views C false true , 4294967007 0 0 +4294967008 pg_user C false true , 4294967008 0 0 +4294967009 pg_user_mappings C false true , 4294967009 0 0 +4294967010 pg_user_mapping C false true , 4294967010 0 0 +4294967011 pg_type C false true , 4294967011 0 0 +4294967012 pg_ts_template C false true , 4294967012 0 0 +4294967013 pg_ts_parser C false true , 4294967013 0 0 +4294967014 pg_ts_dict C false true , 4294967014 0 0 +4294967015 pg_ts_config C false true , 4294967015 0 0 +4294967016 pg_ts_config_map C false true , 4294967016 0 0 +4294967017 pg_trigger C false true , 4294967017 0 0 +4294967018 pg_transform C false true , 4294967018 0 0 +4294967019 pg_timezone_names C false true , 4294967019 0 0 +4294967020 pg_timezone_abbrevs C false true , 4294967020 0 0 +4294967021 pg_tablespace C false true , 4294967021 0 0 +4294967022 pg_tables C false true , 4294967022 0 0 +4294967023 pg_subscription C false true , 4294967023 0 0 +4294967024 pg_subscription_rel C false true , 4294967024 0 0 +4294967025 pg_stats C false true , 4294967025 0 0 +4294967026 pg_stats_ext C false true , 4294967026 0 0 +4294967027 pg_statistic C false true , 4294967027 0 0 +4294967028 pg_statistic_ext C false true , 4294967028 0 0 +4294967029 pg_statistic_ext_data C false true , 4294967029 0 0 +4294967030 pg_statio_user_tables C false true , 4294967030 0 0 +4294967031 pg_statio_user_sequences C false true , 4294967031 0 0 +4294967032 pg_statio_user_indexes C false true , 4294967032 0 0 +4294967033 pg_statio_sys_tables C false true , 4294967033 0 0 +4294967034 pg_statio_sys_sequences C false true , 4294967034 0 0 +4294967035 pg_statio_sys_indexes C false true , 4294967035 0 0 +4294967036 pg_statio_all_tables C false true , 4294967036 0 0 +4294967037 pg_statio_all_sequences C false true , 4294967037 0 0 +4294967038 pg_statio_all_indexes C false true , 4294967038 0 0 +4294967039 pg_stat_xact_user_tables C false true , 4294967039 0 0 +4294967040 pg_stat_xact_user_functions C false true , 4294967040 0 0 +4294967041 pg_stat_xact_sys_tables C false true , 4294967041 0 0 +4294967042 pg_stat_xact_all_tables C false true , 4294967042 0 0 +4294967043 pg_stat_wal_receiver C false true , 4294967043 0 0 +4294967044 pg_stat_user_tables C false true , 4294967044 0 0 +4294967045 pg_stat_user_indexes C false true , 4294967045 0 0 +4294967046 pg_stat_user_functions C false true , 4294967046 0 0 +4294967047 pg_stat_sys_tables C false true , 4294967047 0 0 +4294967048 pg_stat_sys_indexes C false true , 4294967048 0 0 +4294967049 pg_stat_subscription C false true , 4294967049 0 0 +4294967050 pg_stat_ssl C false true , 4294967050 0 0 +4294967051 pg_stat_slru C false true , 4294967051 0 0 +4294967052 pg_stat_replication C false true , 4294967052 0 0 +4294967053 pg_stat_progress_vacuum C false true , 4294967053 0 0 +4294967054 pg_stat_progress_create_index C false true , 4294967054 0 0 +4294967055 pg_stat_progress_cluster C false true , 4294967055 0 0 +4294967056 pg_stat_progress_basebackup C false true , 4294967056 0 0 +4294967057 pg_stat_progress_analyze C false true , 4294967057 0 0 +4294967058 pg_stat_gssapi C false true , 4294967058 0 0 +4294967059 pg_stat_database C false true , 4294967059 0 0 +4294967060 pg_stat_database_conflicts C false true , 4294967060 0 0 +4294967061 pg_stat_bgwriter C false true , 4294967061 0 0 +4294967062 pg_stat_archiver C false true , 4294967062 0 0 +4294967063 pg_stat_all_tables C false true , 4294967063 0 0 +4294967064 pg_stat_all_indexes C false true , 4294967064 0 0 +4294967065 pg_stat_activity C false true , 4294967065 0 0 +4294967066 pg_shmem_allocations C false true , 4294967066 0 0 +4294967067 pg_shdepend C false true , 4294967067 0 0 +4294967068 pg_shseclabel C false true , 4294967068 0 0 +4294967069 pg_shdescription C false true , 4294967069 0 0 +4294967070 pg_shadow C false true , 4294967070 0 0 +4294967071 pg_settings C false true , 4294967071 0 0 +4294967072 pg_sequences C false true , 4294967072 0 0 +4294967073 pg_sequence C false true , 4294967073 0 0 +4294967074 pg_seclabel C false true , 4294967074 0 0 +4294967075 pg_seclabels C false true , 4294967075 0 0 +4294967076 pg_rules C false true , 4294967076 0 0 +4294967077 pg_roles C false true , 4294967077 0 0 +4294967078 pg_rewrite C false true , 4294967078 0 0 +4294967079 pg_replication_slots C false true , 4294967079 0 0 +4294967080 pg_replication_origin C false true , 4294967080 0 0 +4294967081 pg_replication_origin_status C false true , 4294967081 0 0 +4294967082 pg_range C false true , 4294967082 0 0 +4294967083 pg_publication_tables C false true , 4294967083 0 0 +4294967084 pg_publication C false true , 4294967084 0 0 +4294967085 pg_publication_rel C false true , 4294967085 0 0 +4294967086 pg_proc C false true , 4294967086 0 0 +4294967087 pg_prepared_xacts C false true , 4294967087 0 0 +4294967088 pg_prepared_statements C false true , 4294967088 0 0 +4294967089 pg_policy C false true , 4294967089 0 0 +4294967090 pg_policies C false true , 4294967090 0 0 +4294967091 pg_partitioned_table C false true , 4294967091 0 0 +4294967092 pg_opfamily C false true , 4294967092 0 0 +4294967093 pg_operator C false true , 4294967093 0 0 +4294967094 pg_opclass C false true , 4294967094 0 0 +4294967095 pg_namespace C false true , 4294967095 0 0 +4294967096 pg_matviews C false true , 4294967096 0 0 +4294967097 pg_locks C false true , 4294967097 0 0 +4294967098 pg_largeobject C false true , 4294967098 0 0 +4294967099 pg_largeobject_metadata C false true , 4294967099 0 0 +4294967100 pg_language C false true , 4294967100 0 0 +4294967101 pg_init_privs C false true , 4294967101 0 0 +4294967102 pg_inherits C false true , 4294967102 0 0 +4294967103 pg_indexes C false true , 4294967103 0 0 +4294967104 pg_index C false true , 4294967104 0 0 +4294967105 pg_hba_file_rules C false true , 4294967105 0 0 +4294967106 pg_group C false true , 4294967106 0 0 +4294967107 pg_foreign_table C false true , 4294967107 0 0 +4294967108 pg_foreign_server C false true , 4294967108 0 0 +4294967109 pg_foreign_data_wrapper C false true , 4294967109 0 0 +4294967110 pg_file_settings C false true , 4294967110 0 0 +4294967111 pg_extension C false true , 4294967111 0 0 +4294967112 pg_event_trigger C false true , 4294967112 0 0 +4294967113 pg_enum C false true , 4294967113 0 0 +4294967114 pg_description C false true , 4294967114 0 0 +4294967115 pg_depend C false true , 4294967115 0 0 +4294967116 pg_default_acl C false true , 4294967116 0 0 +4294967117 pg_db_role_setting C false true , 4294967117 0 0 +4294967118 pg_database C false true , 4294967118 0 0 +4294967119 pg_cursors C false true , 4294967119 0 0 +4294967120 pg_conversion C false true , 4294967120 0 0 +4294967121 pg_constraint C false true , 4294967121 0 0 +4294967122 pg_config C false true , 4294967122 0 0 +4294967123 pg_collation C false true , 4294967123 0 0 +4294967124 pg_class C false true , 4294967124 0 0 +4294967125 pg_cast C false true , 4294967125 0 0 +4294967126 pg_available_extensions C false true , 4294967126 0 0 +4294967127 pg_available_extension_versions C false true , 4294967127 0 0 +4294967128 pg_auth_members C false true , 4294967128 0 0 +4294967129 pg_authid C false true , 4294967129 0 0 +4294967130 pg_attribute C false true , 4294967130 0 0 +4294967131 pg_attrdef C false true , 4294967131 0 0 +4294967132 pg_amproc C false true , 4294967132 0 0 +4294967133 pg_amop C false true , 4294967133 0 0 +4294967134 pg_am C false true , 4294967134 0 0 +4294967135 pg_aggregate C false true , 4294967135 0 0 +4294967137 views C false true , 4294967137 0 0 +4294967138 view_table_usage C false true , 4294967138 0 0 +4294967139 view_routine_usage C false true , 4294967139 0 0 +4294967140 view_column_usage C false true , 4294967140 0 0 +4294967141 user_privileges C false true , 4294967141 0 0 +4294967142 user_mappings C false true , 4294967142 0 0 +4294967143 user_mapping_options C false true , 4294967143 0 0 +4294967144 user_defined_types C false true , 4294967144 0 0 +4294967145 user_attributes C false true , 4294967145 0 0 +4294967146 usage_privileges C false true , 4294967146 0 0 +4294967147 udt_privileges C false true , 4294967147 0 0 +4294967148 type_privileges C false true , 4294967148 0 0 +4294967149 triggers C false true , 4294967149 0 0 +4294967150 triggered_update_columns C false true , 4294967150 0 0 +4294967151 transforms C false true , 4294967151 0 0 +4294967152 tablespaces C false true , 4294967152 0 0 +4294967153 tablespaces_extensions C false true , 4294967153 0 0 +4294967154 tables C false true , 4294967154 0 0 +4294967155 tables_extensions C false true , 4294967155 0 0 +4294967156 table_privileges C false true , 4294967156 0 0 +4294967157 table_constraints_extensions C false true , 4294967157 0 0 +4294967158 table_constraints C false true , 4294967158 0 0 +4294967159 statistics C false true , 4294967159 0 0 +4294967160 st_units_of_measure C false true , 4294967160 0 0 +4294967161 st_spatial_reference_systems C false true , 4294967161 0 0 +4294967162 st_geometry_columns C false true , 4294967162 0 0 +4294967163 session_variables C false true , 4294967163 0 0 +4294967164 sequences C false true , 4294967164 0 0 +4294967165 schema_privileges C false true , 4294967165 0 0 +4294967166 schemata C false true , 4294967166 0 0 +4294967167 schemata_extensions C false true , 4294967167 0 0 +4294967168 sql_sizing C false true , 4294967168 0 0 +4294967169 sql_parts C false true , 4294967169 0 0 +4294967170 sql_implementation_info C false true , 4294967170 0 0 +4294967171 sql_features C false true , 4294967171 0 0 +4294967172 routines C false true , 4294967172 0 0 +4294967173 routine_privileges C false true , 4294967173 0 0 +4294967174 role_usage_grants C false true , 4294967174 0 0 +4294967175 role_udt_grants C false true , 4294967175 0 0 +4294967176 role_table_grants C false true , 4294967176 0 0 +4294967177 role_routine_grants C false true , 4294967177 0 0 +4294967178 role_column_grants C false true , 4294967178 0 0 +4294967179 resource_groups C false true , 4294967179 0 0 +4294967180 referential_constraints C false true , 4294967180 0 0 +4294967181 profiling C false true , 4294967181 0 0 +4294967182 processlist C false true , 4294967182 0 0 +4294967183 plugins C false true , 4294967183 0 0 +4294967184 partitions C false true , 4294967184 0 0 +4294967185 parameters C false true , 4294967185 0 0 +4294967186 optimizer_trace C false true , 4294967186 0 0 +4294967187 keywords C false true , 4294967187 0 0 +4294967188 key_column_usage C false true , 4294967188 0 0 +4294967189 information_schema_catalog_name C false true , 4294967189 0 0 +4294967190 foreign_tables C false true , 4294967190 0 0 +4294967191 foreign_table_options C false true , 4294967191 0 0 +4294967192 foreign_servers C false true , 4294967192 0 0 +4294967193 foreign_server_options C false true , 4294967193 0 0 +4294967194 foreign_data_wrappers C false true , 4294967194 0 0 +4294967195 foreign_data_wrapper_options C false true , 4294967195 0 0 +4294967196 files C false true , 4294967196 0 0 +4294967197 events C false true , 4294967197 0 0 +4294967198 engines C false true , 4294967198 0 0 +4294967199 enabled_roles C false true , 4294967199 0 0 +4294967200 element_types C false true , 4294967200 0 0 +4294967201 domains C false true , 4294967201 0 0 +4294967202 domain_udt_usage C false true , 4294967202 0 0 +4294967203 domain_constraints C false true , 4294967203 0 0 +4294967204 data_type_privileges C false true , 4294967204 0 0 +4294967205 constraint_table_usage C false true , 4294967205 0 0 +4294967206 constraint_column_usage C false true , 4294967206 0 0 +4294967207 columns C false true , 4294967207 0 0 +4294967208 columns_extensions C false true , 4294967208 0 0 +4294967209 column_udt_usage C false true , 4294967209 0 0 +4294967210 column_statistics C false true , 4294967210 0 0 +4294967211 column_privileges C false true , 4294967211 0 0 +4294967212 column_options C false true , 4294967212 0 0 +4294967213 column_domain_usage C false true , 4294967213 0 0 +4294967214 column_column_usage C false true , 4294967214 0 0 +4294967215 collations C false true , 4294967215 0 0 +4294967216 collation_character_set_applicability C false true , 4294967216 0 0 +4294967217 check_constraints C false true , 4294967217 0 0 +4294967218 check_constraint_routine_usage C false true , 4294967218 0 0 +4294967219 character_sets C false true , 4294967219 0 0 +4294967220 attributes C false true , 4294967220 0 0 +4294967221 applicable_roles C false true , 4294967221 0 0 +4294967222 administrable_role_authorizations C false true , 4294967222 0 0 +4294967224 super_regions C false true , 4294967224 0 0 +4294967225 pg_catalog_table_is_implemented C false true , 4294967225 0 0 +4294967226 tenant_usage_details C false true , 4294967226 0 0 +4294967227 active_range_feeds C false true , 4294967227 0 0 +4294967228 default_privileges C false true , 4294967228 0 0 +4294967229 regions C false true , 4294967229 0 0 +4294967230 cluster_inflight_traces C false true , 4294967230 0 0 +4294967231 lost_descriptors_with_data C false true , 4294967231 0 0 +4294967232 cross_db_references C false true , 4294967232 0 0 +4294967233 cluster_database_privileges C false true , 4294967233 0 0 +4294967234 invalid_objects C false true , 4294967234 0 0 +4294967235 zones C false true , 4294967235 0 0 +4294967236 transaction_statistics C false true , 4294967236 0 0 +4294967237 node_transaction_statistics C false true , 4294967237 0 0 +4294967238 table_row_statistics C false true , 4294967238 0 0 +4294967239 tables C false true , 4294967239 0 0 +4294967240 table_indexes C false true , 4294967240 0 0 +4294967241 table_columns C false true , 4294967241 0 0 +4294967242 statement_statistics C false true , 4294967242 0 0 +4294967243 session_variables C false true , 4294967243 0 0 +4294967244 session_trace C false true , 4294967244 0 0 +4294967245 schema_changes C false true , 4294967245 0 0 +4294967246 node_runtime_info C false true , 4294967246 0 0 +4294967247 ranges C false true , 4294967247 0 0 +4294967248 ranges_no_leases C false true , 4294967248 0 0 +4294967249 predefined_comments C false true , 4294967249 0 0 +4294967250 partitions C false true , 4294967250 0 0 +4294967251 node_txn_stats C false true , 4294967251 0 0 +4294967252 node_statement_statistics C false true , 4294967252 0 0 +4294967253 node_metrics C false true , 4294967253 0 0 +4294967254 node_sessions C false true , 4294967254 0 0 +4294967255 node_transactions C false true , 4294967255 0 0 +4294967256 node_queries C false true , 4294967256 0 0 +4294967257 node_execution_insights C false true , 4294967257 0 0 +4294967258 node_distsql_flows C false true , 4294967258 0 0 +4294967259 node_contention_events C false true , 4294967259 0 0 +4294967260 leases C false true , 4294967260 0 0 +4294967261 kv_store_status C false true , 4294967261 0 0 +4294967262 kv_node_status C false true , 4294967262 0 0 +4294967263 jobs C false true , 4294967263 0 0 +4294967264 node_inflight_trace_spans C false true , 4294967264 0 0 +4294967265 index_usage_statistics C false true , 4294967265 0 0 +4294967266 index_columns C false true , 4294967266 0 0 +4294967267 transaction_contention_events C false true , 4294967267 0 0 +4294967268 gossip_network C false true , 4294967268 0 0 +4294967269 gossip_liveness C false true , 4294967269 0 0 +4294967270 gossip_alerts C false true , 4294967270 0 0 +4294967271 gossip_nodes C false true , 4294967271 0 0 +4294967272 kv_node_liveness C false true , 4294967272 0 0 +4294967273 forward_dependencies C false true , 4294967273 0 0 +4294967274 feature_usage C false true , 4294967274 0 0 +4294967275 databases C false true , 4294967275 0 0 +4294967276 create_type_statements C false true , 4294967276 0 0 +4294967277 create_statements C false true , 4294967277 0 0 +4294967278 create_schema_statements C false true , 4294967278 0 0 +4294967279 create_function_statements C false true , 4294967279 0 0 4294967280 cluster_transaction_statistics C false true , 4294967280 0 0 4294967281 cluster_statement_statistics C false true , 4294967281 0 0 4294967282 cluster_settings C false true , 4294967282 0 0 @@ -2481,279 +2483,280 @@ oid typname typinput typoutput 100132 _newtype1 array_in array_out array_recv array_send 0 0 0 100133 newtype2 enum_in enum_out enum_recv enum_send 0 0 0 100134 _newtype2 array_in array_out array_recv array_send 0 0 0 -4294967004 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967005 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967006 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967008 pg_views record_in record_out record_recv record_send 0 0 0 -4294967009 pg_user record_in record_out record_recv record_send 0 0 0 -4294967010 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967011 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967012 pg_type record_in record_out record_recv record_send 0 0 0 -4294967013 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967014 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967015 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967016 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967017 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967018 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967019 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967020 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967021 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967022 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967023 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967024 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967025 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967026 pg_stats record_in record_out record_recv record_send 0 0 0 -4294967027 pg_stats_ext record_in record_out record_recv record_send 0 0 0 -4294967028 pg_statistic record_in record_out record_recv record_send 0 0 0 -4294967029 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967030 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 -4294967031 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967032 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967033 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967034 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967035 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967036 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967037 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967038 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967039 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967040 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967041 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967042 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967043 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967044 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967045 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967046 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967047 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967048 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967049 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967050 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967051 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967066 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967067 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967068 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967069 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967070 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967071 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967072 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967073 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967074 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967075 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967076 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967077 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967078 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967079 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967080 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967081 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967082 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967083 pg_range record_in record_out record_recv record_send 0 0 0 -4294967084 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967085 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967086 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967087 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967088 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967089 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967090 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967091 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967092 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967093 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967094 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967095 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967096 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967097 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967098 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967099 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967100 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967101 pg_language record_in record_out record_recv record_send 0 0 0 -4294967102 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967103 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967104 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967105 pg_index record_in record_out record_recv record_send 0 0 0 -4294967106 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967107 pg_group record_in record_out record_recv record_send 0 0 0 -4294967108 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967109 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967110 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967111 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967112 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967113 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967114 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967115 pg_description record_in record_out record_recv record_send 0 0 0 -4294967116 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967117 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967118 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967119 pg_database record_in record_out record_recv record_send 0 0 0 -4294967120 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967121 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967122 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967123 pg_config record_in record_out record_recv record_send 0 0 0 -4294967124 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967125 pg_class record_in record_out record_recv record_send 0 0 0 -4294967126 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967127 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967128 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967129 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967130 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967131 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967132 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967133 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967134 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967135 pg_am record_in record_out record_recv record_send 0 0 0 -4294967136 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967138 views record_in record_out record_recv record_send 0 0 0 -4294967139 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967140 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967141 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967142 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967143 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967144 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967145 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967146 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967147 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967148 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967149 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967150 triggers record_in record_out record_recv record_send 0 0 0 -4294967151 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967152 transforms record_in record_out record_recv record_send 0 0 0 -4294967153 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967154 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967155 tables record_in record_out record_recv record_send 0 0 0 -4294967156 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967157 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967158 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967159 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967160 statistics record_in record_out record_recv record_send 0 0 0 -4294967161 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967162 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967163 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967164 session_variables record_in record_out record_recv record_send 0 0 0 -4294967165 sequences record_in record_out record_recv record_send 0 0 0 -4294967166 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967167 schemata record_in record_out record_recv record_send 0 0 0 -4294967168 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967169 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967170 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967171 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967172 sql_features record_in record_out record_recv record_send 0 0 0 -4294967173 routines record_in record_out record_recv record_send 0 0 0 -4294967174 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967175 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967176 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967177 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967178 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967179 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967180 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967181 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967182 profiling record_in record_out record_recv record_send 0 0 0 -4294967183 processlist record_in record_out record_recv record_send 0 0 0 -4294967184 plugins record_in record_out record_recv record_send 0 0 0 -4294967185 partitions record_in record_out record_recv record_send 0 0 0 -4294967186 parameters record_in record_out record_recv record_send 0 0 0 -4294967187 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967188 keywords record_in record_out record_recv record_send 0 0 0 -4294967189 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967190 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967191 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967192 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967193 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967194 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967195 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967196 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967197 files record_in record_out record_recv record_send 0 0 0 -4294967198 events record_in record_out record_recv record_send 0 0 0 -4294967199 engines record_in record_out record_recv record_send 0 0 0 -4294967200 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967201 element_types record_in record_out record_recv record_send 0 0 0 -4294967202 domains record_in record_out record_recv record_send 0 0 0 -4294967203 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967204 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967205 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967206 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967207 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967208 columns record_in record_out record_recv record_send 0 0 0 -4294967209 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967210 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967211 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967212 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967213 column_options record_in record_out record_recv record_send 0 0 0 -4294967214 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967215 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967216 collations record_in record_out record_recv record_send 0 0 0 -4294967217 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967218 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967219 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967220 character_sets record_in record_out record_recv record_send 0 0 0 -4294967221 attributes record_in record_out record_recv record_send 0 0 0 -4294967222 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967223 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 -4294967225 super_regions record_in record_out record_recv record_send 0 0 0 -4294967226 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 -4294967227 tenant_usage_details record_in record_out record_recv record_send 0 0 0 -4294967228 active_range_feeds record_in record_out record_recv record_send 0 0 0 -4294967229 default_privileges record_in record_out record_recv record_send 0 0 0 -4294967230 regions record_in record_out record_recv record_send 0 0 0 -4294967231 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 -4294967232 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 -4294967233 cross_db_references record_in record_out record_recv record_send 0 0 0 -4294967234 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 -4294967235 invalid_objects record_in record_out record_recv record_send 0 0 0 -4294967236 zones record_in record_out record_recv record_send 0 0 0 -4294967237 transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967238 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967239 table_row_statistics record_in record_out record_recv record_send 0 0 0 -4294967240 tables record_in record_out record_recv record_send 0 0 0 -4294967241 table_indexes record_in record_out record_recv record_send 0 0 0 -4294967242 table_columns record_in record_out record_recv record_send 0 0 0 -4294967243 statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967244 session_variables record_in record_out record_recv record_send 0 0 0 -4294967245 session_trace record_in record_out record_recv record_send 0 0 0 -4294967246 schema_changes record_in record_out record_recv record_send 0 0 0 -4294967247 node_runtime_info record_in record_out record_recv record_send 0 0 0 -4294967248 ranges record_in record_out record_recv record_send 0 0 0 -4294967249 ranges_no_leases record_in record_out record_recv record_send 0 0 0 -4294967250 predefined_comments record_in record_out record_recv record_send 0 0 0 -4294967251 partitions record_in record_out record_recv record_send 0 0 0 -4294967252 node_txn_stats record_in record_out record_recv record_send 0 0 0 -4294967253 node_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967254 node_metrics record_in record_out record_recv record_send 0 0 0 -4294967255 node_sessions record_in record_out record_recv record_send 0 0 0 -4294967256 node_transactions record_in record_out record_recv record_send 0 0 0 -4294967257 node_queries record_in record_out record_recv record_send 0 0 0 -4294967258 node_execution_insights record_in record_out record_recv record_send 0 0 0 -4294967259 node_distsql_flows record_in record_out record_recv record_send 0 0 0 -4294967260 node_contention_events record_in record_out record_recv record_send 0 0 0 -4294967261 leases record_in record_out record_recv record_send 0 0 0 -4294967262 kv_store_status record_in record_out record_recv record_send 0 0 0 -4294967263 kv_node_status record_in record_out record_recv record_send 0 0 0 -4294967264 jobs record_in record_out record_recv record_send 0 0 0 -4294967265 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 -4294967266 index_usage_statistics record_in record_out record_recv record_send 0 0 0 -4294967267 index_columns record_in record_out record_recv record_send 0 0 0 -4294967268 transaction_contention_events record_in record_out record_recv record_send 0 0 0 -4294967269 gossip_network record_in record_out record_recv record_send 0 0 0 -4294967270 gossip_liveness record_in record_out record_recv record_send 0 0 0 -4294967271 gossip_alerts record_in record_out record_recv record_send 0 0 0 -4294967272 gossip_nodes record_in record_out record_recv record_send 0 0 0 -4294967273 kv_node_liveness record_in record_out record_recv record_send 0 0 0 -4294967274 forward_dependencies record_in record_out record_recv record_send 0 0 0 -4294967275 feature_usage record_in record_out record_recv record_send 0 0 0 -4294967276 databases record_in record_out record_recv record_send 0 0 0 -4294967277 create_type_statements record_in record_out record_recv record_send 0 0 0 -4294967278 create_statements record_in record_out record_recv record_send 0 0 0 -4294967279 create_schema_statements record_in record_out record_recv record_send 0 0 0 +4294967003 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967004 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967005 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967007 pg_views record_in record_out record_recv record_send 0 0 0 +4294967008 pg_user record_in record_out record_recv record_send 0 0 0 +4294967009 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967010 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967011 pg_type record_in record_out record_recv record_send 0 0 0 +4294967012 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967013 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967014 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967015 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967016 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967017 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967018 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967019 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967020 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967021 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967022 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967023 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967024 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967025 pg_stats record_in record_out record_recv record_send 0 0 0 +4294967026 pg_stats_ext record_in record_out record_recv record_send 0 0 0 +4294967027 pg_statistic record_in record_out record_recv record_send 0 0 0 +4294967028 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967029 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 +4294967030 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967031 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967032 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967033 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967034 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967035 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967036 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967037 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967038 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967039 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967040 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967041 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967042 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967043 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967044 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967045 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967046 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967047 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967048 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967049 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967050 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967065 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967066 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967067 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967068 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967069 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967070 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967071 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967072 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967073 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967074 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967075 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967076 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967077 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967078 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967079 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967080 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967081 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967082 pg_range record_in record_out record_recv record_send 0 0 0 +4294967083 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967084 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967085 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967086 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967087 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967088 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967089 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967090 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967091 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967092 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967093 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967094 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967095 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967096 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967097 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967098 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967099 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967100 pg_language record_in record_out record_recv record_send 0 0 0 +4294967101 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967102 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967103 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967104 pg_index record_in record_out record_recv record_send 0 0 0 +4294967105 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967106 pg_group record_in record_out record_recv record_send 0 0 0 +4294967107 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967108 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967109 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967110 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967111 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967112 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967113 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967114 pg_description record_in record_out record_recv record_send 0 0 0 +4294967115 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967116 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967117 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967118 pg_database record_in record_out record_recv record_send 0 0 0 +4294967119 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967120 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967121 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967122 pg_config record_in record_out record_recv record_send 0 0 0 +4294967123 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967124 pg_class record_in record_out record_recv record_send 0 0 0 +4294967125 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967126 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967127 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967128 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967129 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967130 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967131 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967132 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967133 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967134 pg_am record_in record_out record_recv record_send 0 0 0 +4294967135 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967137 views record_in record_out record_recv record_send 0 0 0 +4294967138 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967139 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967140 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967141 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967142 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967143 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967144 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967145 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967146 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967147 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967148 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967149 triggers record_in record_out record_recv record_send 0 0 0 +4294967150 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967151 transforms record_in record_out record_recv record_send 0 0 0 +4294967152 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967153 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967154 tables record_in record_out record_recv record_send 0 0 0 +4294967155 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967156 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967157 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967158 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967159 statistics record_in record_out record_recv record_send 0 0 0 +4294967160 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967161 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967162 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967163 session_variables record_in record_out record_recv record_send 0 0 0 +4294967164 sequences record_in record_out record_recv record_send 0 0 0 +4294967165 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967166 schemata record_in record_out record_recv record_send 0 0 0 +4294967167 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967168 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967169 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967170 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967171 sql_features record_in record_out record_recv record_send 0 0 0 +4294967172 routines record_in record_out record_recv record_send 0 0 0 +4294967173 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967174 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967175 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967176 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967177 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967178 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967179 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967180 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967181 profiling record_in record_out record_recv record_send 0 0 0 +4294967182 processlist record_in record_out record_recv record_send 0 0 0 +4294967183 plugins record_in record_out record_recv record_send 0 0 0 +4294967184 partitions record_in record_out record_recv record_send 0 0 0 +4294967185 parameters record_in record_out record_recv record_send 0 0 0 +4294967186 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967187 keywords record_in record_out record_recv record_send 0 0 0 +4294967188 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967189 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967190 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967191 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967192 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967193 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967194 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967195 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967196 files record_in record_out record_recv record_send 0 0 0 +4294967197 events record_in record_out record_recv record_send 0 0 0 +4294967198 engines record_in record_out record_recv record_send 0 0 0 +4294967199 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967200 element_types record_in record_out record_recv record_send 0 0 0 +4294967201 domains record_in record_out record_recv record_send 0 0 0 +4294967202 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967203 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967204 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967205 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967206 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967207 columns record_in record_out record_recv record_send 0 0 0 +4294967208 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967209 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967210 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967211 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967212 column_options record_in record_out record_recv record_send 0 0 0 +4294967213 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967214 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967215 collations record_in record_out record_recv record_send 0 0 0 +4294967216 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967217 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967218 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967219 character_sets record_in record_out record_recv record_send 0 0 0 +4294967220 attributes record_in record_out record_recv record_send 0 0 0 +4294967221 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967222 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967224 super_regions record_in record_out record_recv record_send 0 0 0 +4294967225 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 +4294967226 tenant_usage_details record_in record_out record_recv record_send 0 0 0 +4294967227 active_range_feeds record_in record_out record_recv record_send 0 0 0 +4294967228 default_privileges record_in record_out record_recv record_send 0 0 0 +4294967229 regions record_in record_out record_recv record_send 0 0 0 +4294967230 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 +4294967231 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 +4294967232 cross_db_references record_in record_out record_recv record_send 0 0 0 +4294967233 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 +4294967234 invalid_objects record_in record_out record_recv record_send 0 0 0 +4294967235 zones record_in record_out record_recv record_send 0 0 0 +4294967236 transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967237 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967238 table_row_statistics record_in record_out record_recv record_send 0 0 0 +4294967239 tables record_in record_out record_recv record_send 0 0 0 +4294967240 table_indexes record_in record_out record_recv record_send 0 0 0 +4294967241 table_columns record_in record_out record_recv record_send 0 0 0 +4294967242 statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967243 session_variables record_in record_out record_recv record_send 0 0 0 +4294967244 session_trace record_in record_out record_recv record_send 0 0 0 +4294967245 schema_changes record_in record_out record_recv record_send 0 0 0 +4294967246 node_runtime_info record_in record_out record_recv record_send 0 0 0 +4294967247 ranges record_in record_out record_recv record_send 0 0 0 +4294967248 ranges_no_leases record_in record_out record_recv record_send 0 0 0 +4294967249 predefined_comments record_in record_out record_recv record_send 0 0 0 +4294967250 partitions record_in record_out record_recv record_send 0 0 0 +4294967251 node_txn_stats record_in record_out record_recv record_send 0 0 0 +4294967252 node_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967253 node_metrics record_in record_out record_recv record_send 0 0 0 +4294967254 node_sessions record_in record_out record_recv record_send 0 0 0 +4294967255 node_transactions record_in record_out record_recv record_send 0 0 0 +4294967256 node_queries record_in record_out record_recv record_send 0 0 0 +4294967257 node_execution_insights record_in record_out record_recv record_send 0 0 0 +4294967258 node_distsql_flows record_in record_out record_recv record_send 0 0 0 +4294967259 node_contention_events record_in record_out record_recv record_send 0 0 0 +4294967260 leases record_in record_out record_recv record_send 0 0 0 +4294967261 kv_store_status record_in record_out record_recv record_send 0 0 0 +4294967262 kv_node_status record_in record_out record_recv record_send 0 0 0 +4294967263 jobs record_in record_out record_recv record_send 0 0 0 +4294967264 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967265 index_usage_statistics record_in record_out record_recv record_send 0 0 0 +4294967266 index_columns record_in record_out record_recv record_send 0 0 0 +4294967267 transaction_contention_events record_in record_out record_recv record_send 0 0 0 +4294967268 gossip_network record_in record_out record_recv record_send 0 0 0 +4294967269 gossip_liveness record_in record_out record_recv record_send 0 0 0 +4294967270 gossip_alerts record_in record_out record_recv record_send 0 0 0 +4294967271 gossip_nodes record_in record_out record_recv record_send 0 0 0 +4294967272 kv_node_liveness record_in record_out record_recv record_send 0 0 0 +4294967273 forward_dependencies record_in record_out record_recv record_send 0 0 0 +4294967274 feature_usage record_in record_out record_recv record_send 0 0 0 +4294967275 databases record_in record_out record_recv record_send 0 0 0 +4294967276 create_type_statements record_in record_out record_recv record_send 0 0 0 +4294967277 create_statements record_in record_out record_recv record_send 0 0 0 +4294967278 create_schema_statements record_in record_out record_recv record_send 0 0 0 +4294967279 create_function_statements record_in record_out record_recv record_send 0 0 0 4294967280 cluster_transaction_statistics record_in record_out record_recv record_send 0 0 0 4294967281 cluster_statement_statistics record_in record_out record_recv record_send 0 0 0 4294967282 cluster_settings record_in record_out record_recv record_send 0 0 0 @@ -2873,279 +2876,280 @@ oid typname typalign typstorage typnotn 100132 _newtype1 NULL NULL false 0 -1 100133 newtype2 NULL NULL false 0 -1 100134 _newtype2 NULL NULL false 0 -1 -4294967004 spatial_ref_sys NULL NULL false 0 -1 -4294967005 geometry_columns NULL NULL false 0 -1 -4294967006 geography_columns NULL NULL false 0 -1 -4294967008 pg_views NULL NULL false 0 -1 -4294967009 pg_user NULL NULL false 0 -1 -4294967010 pg_user_mappings NULL NULL false 0 -1 -4294967011 pg_user_mapping NULL NULL false 0 -1 -4294967012 pg_type NULL NULL false 0 -1 -4294967013 pg_ts_template NULL NULL false 0 -1 -4294967014 pg_ts_parser NULL NULL false 0 -1 -4294967015 pg_ts_dict NULL NULL false 0 -1 -4294967016 pg_ts_config NULL NULL false 0 -1 -4294967017 pg_ts_config_map NULL NULL false 0 -1 -4294967018 pg_trigger NULL NULL false 0 -1 -4294967019 pg_transform NULL NULL false 0 -1 -4294967020 pg_timezone_names NULL NULL false 0 -1 -4294967021 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967022 pg_tablespace NULL NULL false 0 -1 -4294967023 pg_tables NULL NULL false 0 -1 -4294967024 pg_subscription NULL NULL false 0 -1 -4294967025 pg_subscription_rel NULL NULL false 0 -1 -4294967026 pg_stats NULL NULL false 0 -1 -4294967027 pg_stats_ext NULL NULL false 0 -1 -4294967028 pg_statistic NULL NULL false 0 -1 -4294967029 pg_statistic_ext NULL NULL false 0 -1 -4294967030 pg_statistic_ext_data NULL NULL false 0 -1 -4294967031 pg_statio_user_tables NULL NULL false 0 -1 -4294967032 pg_statio_user_sequences NULL NULL false 0 -1 -4294967033 pg_statio_user_indexes NULL NULL false 0 -1 -4294967034 pg_statio_sys_tables NULL NULL false 0 -1 -4294967035 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967036 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967037 pg_statio_all_tables NULL NULL false 0 -1 -4294967038 pg_statio_all_sequences NULL NULL false 0 -1 -4294967039 pg_statio_all_indexes NULL NULL false 0 -1 -4294967040 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967041 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967042 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967043 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967044 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967045 pg_stat_user_tables NULL NULL false 0 -1 -4294967046 pg_stat_user_indexes NULL NULL false 0 -1 -4294967047 pg_stat_user_functions NULL NULL false 0 -1 -4294967048 pg_stat_sys_tables NULL NULL false 0 -1 -4294967049 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967050 pg_stat_subscription NULL NULL false 0 -1 -4294967051 pg_stat_ssl NULL NULL false 0 -1 -4294967052 pg_stat_slru NULL NULL false 0 -1 -4294967053 pg_stat_replication NULL NULL false 0 -1 -4294967054 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967055 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967056 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967057 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967058 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967059 pg_stat_gssapi NULL NULL false 0 -1 -4294967060 pg_stat_database NULL NULL false 0 -1 -4294967061 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967062 pg_stat_bgwriter NULL NULL false 0 -1 -4294967063 pg_stat_archiver NULL NULL false 0 -1 -4294967064 pg_stat_all_tables NULL NULL false 0 -1 -4294967065 pg_stat_all_indexes NULL NULL false 0 -1 -4294967066 pg_stat_activity NULL NULL false 0 -1 -4294967067 pg_shmem_allocations NULL NULL false 0 -1 -4294967068 pg_shdepend NULL NULL false 0 -1 -4294967069 pg_shseclabel NULL NULL false 0 -1 -4294967070 pg_shdescription NULL NULL false 0 -1 -4294967071 pg_shadow NULL NULL false 0 -1 -4294967072 pg_settings NULL NULL false 0 -1 -4294967073 pg_sequences NULL NULL false 0 -1 -4294967074 pg_sequence NULL NULL false 0 -1 -4294967075 pg_seclabel NULL NULL false 0 -1 -4294967076 pg_seclabels NULL NULL false 0 -1 -4294967077 pg_rules NULL NULL false 0 -1 -4294967078 pg_roles NULL NULL false 0 -1 -4294967079 pg_rewrite NULL NULL false 0 -1 -4294967080 pg_replication_slots NULL NULL false 0 -1 -4294967081 pg_replication_origin NULL NULL false 0 -1 -4294967082 pg_replication_origin_status NULL NULL false 0 -1 -4294967083 pg_range NULL NULL false 0 -1 -4294967084 pg_publication_tables NULL NULL false 0 -1 -4294967085 pg_publication NULL NULL false 0 -1 -4294967086 pg_publication_rel NULL NULL false 0 -1 -4294967087 pg_proc NULL NULL false 0 -1 -4294967088 pg_prepared_xacts NULL NULL false 0 -1 -4294967089 pg_prepared_statements NULL NULL false 0 -1 -4294967090 pg_policy NULL NULL false 0 -1 -4294967091 pg_policies NULL NULL false 0 -1 -4294967092 pg_partitioned_table NULL NULL false 0 -1 -4294967093 pg_opfamily NULL NULL false 0 -1 -4294967094 pg_operator NULL NULL false 0 -1 -4294967095 pg_opclass NULL NULL false 0 -1 -4294967096 pg_namespace NULL NULL false 0 -1 -4294967097 pg_matviews NULL NULL false 0 -1 -4294967098 pg_locks NULL NULL false 0 -1 -4294967099 pg_largeobject NULL NULL false 0 -1 -4294967100 pg_largeobject_metadata NULL NULL false 0 -1 -4294967101 pg_language NULL NULL false 0 -1 -4294967102 pg_init_privs NULL NULL false 0 -1 -4294967103 pg_inherits NULL NULL false 0 -1 -4294967104 pg_indexes NULL NULL false 0 -1 -4294967105 pg_index NULL NULL false 0 -1 -4294967106 pg_hba_file_rules NULL NULL false 0 -1 -4294967107 pg_group NULL NULL false 0 -1 -4294967108 pg_foreign_table NULL NULL false 0 -1 -4294967109 pg_foreign_server NULL NULL false 0 -1 -4294967110 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967111 pg_file_settings NULL NULL false 0 -1 -4294967112 pg_extension NULL NULL false 0 -1 -4294967113 pg_event_trigger NULL NULL false 0 -1 -4294967114 pg_enum NULL NULL false 0 -1 -4294967115 pg_description NULL NULL false 0 -1 -4294967116 pg_depend NULL NULL false 0 -1 -4294967117 pg_default_acl NULL NULL false 0 -1 -4294967118 pg_db_role_setting NULL NULL false 0 -1 -4294967119 pg_database NULL NULL false 0 -1 -4294967120 pg_cursors NULL NULL false 0 -1 -4294967121 pg_conversion NULL NULL false 0 -1 -4294967122 pg_constraint NULL NULL false 0 -1 -4294967123 pg_config NULL NULL false 0 -1 -4294967124 pg_collation NULL NULL false 0 -1 -4294967125 pg_class NULL NULL false 0 -1 -4294967126 pg_cast NULL NULL false 0 -1 -4294967127 pg_available_extensions NULL NULL false 0 -1 -4294967128 pg_available_extension_versions NULL NULL false 0 -1 -4294967129 pg_auth_members NULL NULL false 0 -1 -4294967130 pg_authid NULL NULL false 0 -1 -4294967131 pg_attribute NULL NULL false 0 -1 -4294967132 pg_attrdef NULL NULL false 0 -1 -4294967133 pg_amproc NULL NULL false 0 -1 -4294967134 pg_amop NULL NULL false 0 -1 -4294967135 pg_am NULL NULL false 0 -1 -4294967136 pg_aggregate NULL NULL false 0 -1 -4294967138 views NULL NULL false 0 -1 -4294967139 view_table_usage NULL NULL false 0 -1 -4294967140 view_routine_usage NULL NULL false 0 -1 -4294967141 view_column_usage NULL NULL false 0 -1 -4294967142 user_privileges NULL NULL false 0 -1 -4294967143 user_mappings NULL NULL false 0 -1 -4294967144 user_mapping_options NULL NULL false 0 -1 -4294967145 user_defined_types NULL NULL false 0 -1 -4294967146 user_attributes NULL NULL false 0 -1 -4294967147 usage_privileges NULL NULL false 0 -1 -4294967148 udt_privileges NULL NULL false 0 -1 -4294967149 type_privileges NULL NULL false 0 -1 -4294967150 triggers NULL NULL false 0 -1 -4294967151 triggered_update_columns NULL NULL false 0 -1 -4294967152 transforms NULL NULL false 0 -1 -4294967153 tablespaces NULL NULL false 0 -1 -4294967154 tablespaces_extensions NULL NULL false 0 -1 -4294967155 tables NULL NULL false 0 -1 -4294967156 tables_extensions NULL NULL false 0 -1 -4294967157 table_privileges NULL NULL false 0 -1 -4294967158 table_constraints_extensions NULL NULL false 0 -1 -4294967159 table_constraints NULL NULL false 0 -1 -4294967160 statistics NULL NULL false 0 -1 -4294967161 st_units_of_measure NULL NULL false 0 -1 -4294967162 st_spatial_reference_systems NULL NULL false 0 -1 -4294967163 st_geometry_columns NULL NULL false 0 -1 -4294967164 session_variables NULL NULL false 0 -1 -4294967165 sequences NULL NULL false 0 -1 -4294967166 schema_privileges NULL NULL false 0 -1 -4294967167 schemata NULL NULL false 0 -1 -4294967168 schemata_extensions NULL NULL false 0 -1 -4294967169 sql_sizing NULL NULL false 0 -1 -4294967170 sql_parts NULL NULL false 0 -1 -4294967171 sql_implementation_info NULL NULL false 0 -1 -4294967172 sql_features NULL NULL false 0 -1 -4294967173 routines NULL NULL false 0 -1 -4294967174 routine_privileges NULL NULL false 0 -1 -4294967175 role_usage_grants NULL NULL false 0 -1 -4294967176 role_udt_grants NULL NULL false 0 -1 -4294967177 role_table_grants NULL NULL false 0 -1 -4294967178 role_routine_grants NULL NULL false 0 -1 -4294967179 role_column_grants NULL NULL false 0 -1 -4294967180 resource_groups NULL NULL false 0 -1 -4294967181 referential_constraints NULL NULL false 0 -1 -4294967182 profiling NULL NULL false 0 -1 -4294967183 processlist NULL NULL false 0 -1 -4294967184 plugins NULL NULL false 0 -1 -4294967185 partitions NULL NULL false 0 -1 -4294967186 parameters NULL NULL false 0 -1 -4294967187 optimizer_trace NULL NULL false 0 -1 -4294967188 keywords NULL NULL false 0 -1 -4294967189 key_column_usage NULL NULL false 0 -1 -4294967190 information_schema_catalog_name NULL NULL false 0 -1 -4294967191 foreign_tables NULL NULL false 0 -1 -4294967192 foreign_table_options NULL NULL false 0 -1 -4294967193 foreign_servers NULL NULL false 0 -1 -4294967194 foreign_server_options NULL NULL false 0 -1 -4294967195 foreign_data_wrappers NULL NULL false 0 -1 -4294967196 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967197 files NULL NULL false 0 -1 -4294967198 events NULL NULL false 0 -1 -4294967199 engines NULL NULL false 0 -1 -4294967200 enabled_roles NULL NULL false 0 -1 -4294967201 element_types NULL NULL false 0 -1 -4294967202 domains NULL NULL false 0 -1 -4294967203 domain_udt_usage NULL NULL false 0 -1 -4294967204 domain_constraints NULL NULL false 0 -1 -4294967205 data_type_privileges NULL NULL false 0 -1 -4294967206 constraint_table_usage NULL NULL false 0 -1 -4294967207 constraint_column_usage NULL NULL false 0 -1 -4294967208 columns NULL NULL false 0 -1 -4294967209 columns_extensions NULL NULL false 0 -1 -4294967210 column_udt_usage NULL NULL false 0 -1 -4294967211 column_statistics NULL NULL false 0 -1 -4294967212 column_privileges NULL NULL false 0 -1 -4294967213 column_options NULL NULL false 0 -1 -4294967214 column_domain_usage NULL NULL false 0 -1 -4294967215 column_column_usage NULL NULL false 0 -1 -4294967216 collations NULL NULL false 0 -1 -4294967217 collation_character_set_applicability NULL NULL false 0 -1 -4294967218 check_constraints NULL NULL false 0 -1 -4294967219 check_constraint_routine_usage NULL NULL false 0 -1 -4294967220 character_sets NULL NULL false 0 -1 -4294967221 attributes NULL NULL false 0 -1 -4294967222 applicable_roles NULL NULL false 0 -1 -4294967223 administrable_role_authorizations NULL NULL false 0 -1 -4294967225 super_regions NULL NULL false 0 -1 -4294967226 pg_catalog_table_is_implemented NULL NULL false 0 -1 -4294967227 tenant_usage_details NULL NULL false 0 -1 -4294967228 active_range_feeds NULL NULL false 0 -1 -4294967229 default_privileges NULL NULL false 0 -1 -4294967230 regions NULL NULL false 0 -1 -4294967231 cluster_inflight_traces NULL NULL false 0 -1 -4294967232 lost_descriptors_with_data NULL NULL false 0 -1 -4294967233 cross_db_references NULL NULL false 0 -1 -4294967234 cluster_database_privileges NULL NULL false 0 -1 -4294967235 invalid_objects NULL NULL false 0 -1 -4294967236 zones NULL NULL false 0 -1 -4294967237 transaction_statistics NULL NULL false 0 -1 -4294967238 node_transaction_statistics NULL NULL false 0 -1 -4294967239 table_row_statistics NULL NULL false 0 -1 -4294967240 tables NULL NULL false 0 -1 -4294967241 table_indexes NULL NULL false 0 -1 -4294967242 table_columns NULL NULL false 0 -1 -4294967243 statement_statistics NULL NULL false 0 -1 -4294967244 session_variables NULL NULL false 0 -1 -4294967245 session_trace NULL NULL false 0 -1 -4294967246 schema_changes NULL NULL false 0 -1 -4294967247 node_runtime_info NULL NULL false 0 -1 -4294967248 ranges NULL NULL false 0 -1 -4294967249 ranges_no_leases NULL NULL false 0 -1 -4294967250 predefined_comments NULL NULL false 0 -1 -4294967251 partitions NULL NULL false 0 -1 -4294967252 node_txn_stats NULL NULL false 0 -1 -4294967253 node_statement_statistics NULL NULL false 0 -1 -4294967254 node_metrics NULL NULL false 0 -1 -4294967255 node_sessions NULL NULL false 0 -1 -4294967256 node_transactions NULL NULL false 0 -1 -4294967257 node_queries NULL NULL false 0 -1 -4294967258 node_execution_insights NULL NULL false 0 -1 -4294967259 node_distsql_flows NULL NULL false 0 -1 -4294967260 node_contention_events NULL NULL false 0 -1 -4294967261 leases NULL NULL false 0 -1 -4294967262 kv_store_status NULL NULL false 0 -1 -4294967263 kv_node_status NULL NULL false 0 -1 -4294967264 jobs NULL NULL false 0 -1 -4294967265 node_inflight_trace_spans NULL NULL false 0 -1 -4294967266 index_usage_statistics NULL NULL false 0 -1 -4294967267 index_columns NULL NULL false 0 -1 -4294967268 transaction_contention_events NULL NULL false 0 -1 -4294967269 gossip_network NULL NULL false 0 -1 -4294967270 gossip_liveness NULL NULL false 0 -1 -4294967271 gossip_alerts NULL NULL false 0 -1 -4294967272 gossip_nodes NULL NULL false 0 -1 -4294967273 kv_node_liveness NULL NULL false 0 -1 -4294967274 forward_dependencies NULL NULL false 0 -1 -4294967275 feature_usage NULL NULL false 0 -1 -4294967276 databases NULL NULL false 0 -1 -4294967277 create_type_statements NULL NULL false 0 -1 -4294967278 create_statements NULL NULL false 0 -1 -4294967279 create_schema_statements NULL NULL false 0 -1 +4294967003 spatial_ref_sys NULL NULL false 0 -1 +4294967004 geometry_columns NULL NULL false 0 -1 +4294967005 geography_columns NULL NULL false 0 -1 +4294967007 pg_views NULL NULL false 0 -1 +4294967008 pg_user NULL NULL false 0 -1 +4294967009 pg_user_mappings NULL NULL false 0 -1 +4294967010 pg_user_mapping NULL NULL false 0 -1 +4294967011 pg_type NULL NULL false 0 -1 +4294967012 pg_ts_template NULL NULL false 0 -1 +4294967013 pg_ts_parser NULL NULL false 0 -1 +4294967014 pg_ts_dict NULL NULL false 0 -1 +4294967015 pg_ts_config NULL NULL false 0 -1 +4294967016 pg_ts_config_map NULL NULL false 0 -1 +4294967017 pg_trigger NULL NULL false 0 -1 +4294967018 pg_transform NULL NULL false 0 -1 +4294967019 pg_timezone_names NULL NULL false 0 -1 +4294967020 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967021 pg_tablespace NULL NULL false 0 -1 +4294967022 pg_tables NULL NULL false 0 -1 +4294967023 pg_subscription NULL NULL false 0 -1 +4294967024 pg_subscription_rel NULL NULL false 0 -1 +4294967025 pg_stats NULL NULL false 0 -1 +4294967026 pg_stats_ext NULL NULL false 0 -1 +4294967027 pg_statistic NULL NULL false 0 -1 +4294967028 pg_statistic_ext NULL NULL false 0 -1 +4294967029 pg_statistic_ext_data NULL NULL false 0 -1 +4294967030 pg_statio_user_tables NULL NULL false 0 -1 +4294967031 pg_statio_user_sequences NULL NULL false 0 -1 +4294967032 pg_statio_user_indexes NULL NULL false 0 -1 +4294967033 pg_statio_sys_tables NULL NULL false 0 -1 +4294967034 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967035 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967036 pg_statio_all_tables NULL NULL false 0 -1 +4294967037 pg_statio_all_sequences NULL NULL false 0 -1 +4294967038 pg_statio_all_indexes NULL NULL false 0 -1 +4294967039 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967040 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967041 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967042 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967043 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967044 pg_stat_user_tables NULL NULL false 0 -1 +4294967045 pg_stat_user_indexes NULL NULL false 0 -1 +4294967046 pg_stat_user_functions NULL NULL false 0 -1 +4294967047 pg_stat_sys_tables NULL NULL false 0 -1 +4294967048 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967049 pg_stat_subscription NULL NULL false 0 -1 +4294967050 pg_stat_ssl NULL NULL false 0 -1 +4294967051 pg_stat_slru NULL NULL false 0 -1 +4294967052 pg_stat_replication NULL NULL false 0 -1 +4294967053 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967054 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967055 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967056 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967057 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967058 pg_stat_gssapi NULL NULL false 0 -1 +4294967059 pg_stat_database NULL NULL false 0 -1 +4294967060 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967061 pg_stat_bgwriter NULL NULL false 0 -1 +4294967062 pg_stat_archiver NULL NULL false 0 -1 +4294967063 pg_stat_all_tables NULL NULL false 0 -1 +4294967064 pg_stat_all_indexes NULL NULL false 0 -1 +4294967065 pg_stat_activity NULL NULL false 0 -1 +4294967066 pg_shmem_allocations NULL NULL false 0 -1 +4294967067 pg_shdepend NULL NULL false 0 -1 +4294967068 pg_shseclabel NULL NULL false 0 -1 +4294967069 pg_shdescription NULL NULL false 0 -1 +4294967070 pg_shadow NULL NULL false 0 -1 +4294967071 pg_settings NULL NULL false 0 -1 +4294967072 pg_sequences NULL NULL false 0 -1 +4294967073 pg_sequence NULL NULL false 0 -1 +4294967074 pg_seclabel NULL NULL false 0 -1 +4294967075 pg_seclabels NULL NULL false 0 -1 +4294967076 pg_rules NULL NULL false 0 -1 +4294967077 pg_roles NULL NULL false 0 -1 +4294967078 pg_rewrite NULL NULL false 0 -1 +4294967079 pg_replication_slots NULL NULL false 0 -1 +4294967080 pg_replication_origin NULL NULL false 0 -1 +4294967081 pg_replication_origin_status NULL NULL false 0 -1 +4294967082 pg_range NULL NULL false 0 -1 +4294967083 pg_publication_tables NULL NULL false 0 -1 +4294967084 pg_publication NULL NULL false 0 -1 +4294967085 pg_publication_rel NULL NULL false 0 -1 +4294967086 pg_proc NULL NULL false 0 -1 +4294967087 pg_prepared_xacts NULL NULL false 0 -1 +4294967088 pg_prepared_statements NULL NULL false 0 -1 +4294967089 pg_policy NULL NULL false 0 -1 +4294967090 pg_policies NULL NULL false 0 -1 +4294967091 pg_partitioned_table NULL NULL false 0 -1 +4294967092 pg_opfamily NULL NULL false 0 -1 +4294967093 pg_operator NULL NULL false 0 -1 +4294967094 pg_opclass NULL NULL false 0 -1 +4294967095 pg_namespace NULL NULL false 0 -1 +4294967096 pg_matviews NULL NULL false 0 -1 +4294967097 pg_locks NULL NULL false 0 -1 +4294967098 pg_largeobject NULL NULL false 0 -1 +4294967099 pg_largeobject_metadata NULL NULL false 0 -1 +4294967100 pg_language NULL NULL false 0 -1 +4294967101 pg_init_privs NULL NULL false 0 -1 +4294967102 pg_inherits NULL NULL false 0 -1 +4294967103 pg_indexes NULL NULL false 0 -1 +4294967104 pg_index NULL NULL false 0 -1 +4294967105 pg_hba_file_rules NULL NULL false 0 -1 +4294967106 pg_group NULL NULL false 0 -1 +4294967107 pg_foreign_table NULL NULL false 0 -1 +4294967108 pg_foreign_server NULL NULL false 0 -1 +4294967109 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967110 pg_file_settings NULL NULL false 0 -1 +4294967111 pg_extension NULL NULL false 0 -1 +4294967112 pg_event_trigger NULL NULL false 0 -1 +4294967113 pg_enum NULL NULL false 0 -1 +4294967114 pg_description NULL NULL false 0 -1 +4294967115 pg_depend NULL NULL false 0 -1 +4294967116 pg_default_acl NULL NULL false 0 -1 +4294967117 pg_db_role_setting NULL NULL false 0 -1 +4294967118 pg_database NULL NULL false 0 -1 +4294967119 pg_cursors NULL NULL false 0 -1 +4294967120 pg_conversion NULL NULL false 0 -1 +4294967121 pg_constraint NULL NULL false 0 -1 +4294967122 pg_config NULL NULL false 0 -1 +4294967123 pg_collation NULL NULL false 0 -1 +4294967124 pg_class NULL NULL false 0 -1 +4294967125 pg_cast NULL NULL false 0 -1 +4294967126 pg_available_extensions NULL NULL false 0 -1 +4294967127 pg_available_extension_versions NULL NULL false 0 -1 +4294967128 pg_auth_members NULL NULL false 0 -1 +4294967129 pg_authid NULL NULL false 0 -1 +4294967130 pg_attribute NULL NULL false 0 -1 +4294967131 pg_attrdef NULL NULL false 0 -1 +4294967132 pg_amproc NULL NULL false 0 -1 +4294967133 pg_amop NULL NULL false 0 -1 +4294967134 pg_am NULL NULL false 0 -1 +4294967135 pg_aggregate NULL NULL false 0 -1 +4294967137 views NULL NULL false 0 -1 +4294967138 view_table_usage NULL NULL false 0 -1 +4294967139 view_routine_usage NULL NULL false 0 -1 +4294967140 view_column_usage NULL NULL false 0 -1 +4294967141 user_privileges NULL NULL false 0 -1 +4294967142 user_mappings NULL NULL false 0 -1 +4294967143 user_mapping_options NULL NULL false 0 -1 +4294967144 user_defined_types NULL NULL false 0 -1 +4294967145 user_attributes NULL NULL false 0 -1 +4294967146 usage_privileges NULL NULL false 0 -1 +4294967147 udt_privileges NULL NULL false 0 -1 +4294967148 type_privileges NULL NULL false 0 -1 +4294967149 triggers NULL NULL false 0 -1 +4294967150 triggered_update_columns NULL NULL false 0 -1 +4294967151 transforms NULL NULL false 0 -1 +4294967152 tablespaces NULL NULL false 0 -1 +4294967153 tablespaces_extensions NULL NULL false 0 -1 +4294967154 tables NULL NULL false 0 -1 +4294967155 tables_extensions NULL NULL false 0 -1 +4294967156 table_privileges NULL NULL false 0 -1 +4294967157 table_constraints_extensions NULL NULL false 0 -1 +4294967158 table_constraints NULL NULL false 0 -1 +4294967159 statistics NULL NULL false 0 -1 +4294967160 st_units_of_measure NULL NULL false 0 -1 +4294967161 st_spatial_reference_systems NULL NULL false 0 -1 +4294967162 st_geometry_columns NULL NULL false 0 -1 +4294967163 session_variables NULL NULL false 0 -1 +4294967164 sequences NULL NULL false 0 -1 +4294967165 schema_privileges NULL NULL false 0 -1 +4294967166 schemata NULL NULL false 0 -1 +4294967167 schemata_extensions NULL NULL false 0 -1 +4294967168 sql_sizing NULL NULL false 0 -1 +4294967169 sql_parts NULL NULL false 0 -1 +4294967170 sql_implementation_info NULL NULL false 0 -1 +4294967171 sql_features NULL NULL false 0 -1 +4294967172 routines NULL NULL false 0 -1 +4294967173 routine_privileges NULL NULL false 0 -1 +4294967174 role_usage_grants NULL NULL false 0 -1 +4294967175 role_udt_grants NULL NULL false 0 -1 +4294967176 role_table_grants NULL NULL false 0 -1 +4294967177 role_routine_grants NULL NULL false 0 -1 +4294967178 role_column_grants NULL NULL false 0 -1 +4294967179 resource_groups NULL NULL false 0 -1 +4294967180 referential_constraints NULL NULL false 0 -1 +4294967181 profiling NULL NULL false 0 -1 +4294967182 processlist NULL NULL false 0 -1 +4294967183 plugins NULL NULL false 0 -1 +4294967184 partitions NULL NULL false 0 -1 +4294967185 parameters NULL NULL false 0 -1 +4294967186 optimizer_trace NULL NULL false 0 -1 +4294967187 keywords NULL NULL false 0 -1 +4294967188 key_column_usage NULL NULL false 0 -1 +4294967189 information_schema_catalog_name NULL NULL false 0 -1 +4294967190 foreign_tables NULL NULL false 0 -1 +4294967191 foreign_table_options NULL NULL false 0 -1 +4294967192 foreign_servers NULL NULL false 0 -1 +4294967193 foreign_server_options NULL NULL false 0 -1 +4294967194 foreign_data_wrappers NULL NULL false 0 -1 +4294967195 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967196 files NULL NULL false 0 -1 +4294967197 events NULL NULL false 0 -1 +4294967198 engines NULL NULL false 0 -1 +4294967199 enabled_roles NULL NULL false 0 -1 +4294967200 element_types NULL NULL false 0 -1 +4294967201 domains NULL NULL false 0 -1 +4294967202 domain_udt_usage NULL NULL false 0 -1 +4294967203 domain_constraints NULL NULL false 0 -1 +4294967204 data_type_privileges NULL NULL false 0 -1 +4294967205 constraint_table_usage NULL NULL false 0 -1 +4294967206 constraint_column_usage NULL NULL false 0 -1 +4294967207 columns NULL NULL false 0 -1 +4294967208 columns_extensions NULL NULL false 0 -1 +4294967209 column_udt_usage NULL NULL false 0 -1 +4294967210 column_statistics NULL NULL false 0 -1 +4294967211 column_privileges NULL NULL false 0 -1 +4294967212 column_options NULL NULL false 0 -1 +4294967213 column_domain_usage NULL NULL false 0 -1 +4294967214 column_column_usage NULL NULL false 0 -1 +4294967215 collations NULL NULL false 0 -1 +4294967216 collation_character_set_applicability NULL NULL false 0 -1 +4294967217 check_constraints NULL NULL false 0 -1 +4294967218 check_constraint_routine_usage NULL NULL false 0 -1 +4294967219 character_sets NULL NULL false 0 -1 +4294967220 attributes NULL NULL false 0 -1 +4294967221 applicable_roles NULL NULL false 0 -1 +4294967222 administrable_role_authorizations NULL NULL false 0 -1 +4294967224 super_regions NULL NULL false 0 -1 +4294967225 pg_catalog_table_is_implemented NULL NULL false 0 -1 +4294967226 tenant_usage_details NULL NULL false 0 -1 +4294967227 active_range_feeds NULL NULL false 0 -1 +4294967228 default_privileges NULL NULL false 0 -1 +4294967229 regions NULL NULL false 0 -1 +4294967230 cluster_inflight_traces NULL NULL false 0 -1 +4294967231 lost_descriptors_with_data NULL NULL false 0 -1 +4294967232 cross_db_references NULL NULL false 0 -1 +4294967233 cluster_database_privileges NULL NULL false 0 -1 +4294967234 invalid_objects NULL NULL false 0 -1 +4294967235 zones NULL NULL false 0 -1 +4294967236 transaction_statistics NULL NULL false 0 -1 +4294967237 node_transaction_statistics NULL NULL false 0 -1 +4294967238 table_row_statistics NULL NULL false 0 -1 +4294967239 tables NULL NULL false 0 -1 +4294967240 table_indexes NULL NULL false 0 -1 +4294967241 table_columns NULL NULL false 0 -1 +4294967242 statement_statistics NULL NULL false 0 -1 +4294967243 session_variables NULL NULL false 0 -1 +4294967244 session_trace NULL NULL false 0 -1 +4294967245 schema_changes NULL NULL false 0 -1 +4294967246 node_runtime_info NULL NULL false 0 -1 +4294967247 ranges NULL NULL false 0 -1 +4294967248 ranges_no_leases NULL NULL false 0 -1 +4294967249 predefined_comments NULL NULL false 0 -1 +4294967250 partitions NULL NULL false 0 -1 +4294967251 node_txn_stats NULL NULL false 0 -1 +4294967252 node_statement_statistics NULL NULL false 0 -1 +4294967253 node_metrics NULL NULL false 0 -1 +4294967254 node_sessions NULL NULL false 0 -1 +4294967255 node_transactions NULL NULL false 0 -1 +4294967256 node_queries NULL NULL false 0 -1 +4294967257 node_execution_insights NULL NULL false 0 -1 +4294967258 node_distsql_flows NULL NULL false 0 -1 +4294967259 node_contention_events NULL NULL false 0 -1 +4294967260 leases NULL NULL false 0 -1 +4294967261 kv_store_status NULL NULL false 0 -1 +4294967262 kv_node_status NULL NULL false 0 -1 +4294967263 jobs NULL NULL false 0 -1 +4294967264 node_inflight_trace_spans NULL NULL false 0 -1 +4294967265 index_usage_statistics NULL NULL false 0 -1 +4294967266 index_columns NULL NULL false 0 -1 +4294967267 transaction_contention_events NULL NULL false 0 -1 +4294967268 gossip_network NULL NULL false 0 -1 +4294967269 gossip_liveness NULL NULL false 0 -1 +4294967270 gossip_alerts NULL NULL false 0 -1 +4294967271 gossip_nodes NULL NULL false 0 -1 +4294967272 kv_node_liveness NULL NULL false 0 -1 +4294967273 forward_dependencies NULL NULL false 0 -1 +4294967274 feature_usage NULL NULL false 0 -1 +4294967275 databases NULL NULL false 0 -1 +4294967276 create_type_statements NULL NULL false 0 -1 +4294967277 create_statements NULL NULL false 0 -1 +4294967278 create_schema_statements NULL NULL false 0 -1 +4294967279 create_function_statements NULL NULL false 0 -1 4294967280 cluster_transaction_statistics NULL NULL false 0 -1 4294967281 cluster_statement_statistics NULL NULL false 0 -1 4294967282 cluster_settings NULL NULL false 0 -1 @@ -3265,279 +3269,280 @@ oid typname typndims typcollation typde 100132 _newtype1 0 0 NULL NULL NULL 100133 newtype2 0 0 NULL NULL NULL 100134 _newtype2 0 0 NULL NULL NULL -4294967004 spatial_ref_sys 0 0 NULL NULL NULL -4294967005 geometry_columns 0 0 NULL NULL NULL -4294967006 geography_columns 0 0 NULL NULL NULL -4294967008 pg_views 0 0 NULL NULL NULL -4294967009 pg_user 0 0 NULL NULL NULL -4294967010 pg_user_mappings 0 0 NULL NULL NULL -4294967011 pg_user_mapping 0 0 NULL NULL NULL -4294967012 pg_type 0 0 NULL NULL NULL -4294967013 pg_ts_template 0 0 NULL NULL NULL -4294967014 pg_ts_parser 0 0 NULL NULL NULL -4294967015 pg_ts_dict 0 0 NULL NULL NULL -4294967016 pg_ts_config 0 0 NULL NULL NULL -4294967017 pg_ts_config_map 0 0 NULL NULL NULL -4294967018 pg_trigger 0 0 NULL NULL NULL -4294967019 pg_transform 0 0 NULL NULL NULL -4294967020 pg_timezone_names 0 0 NULL NULL NULL -4294967021 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967022 pg_tablespace 0 0 NULL NULL NULL -4294967023 pg_tables 0 0 NULL NULL NULL -4294967024 pg_subscription 0 0 NULL NULL NULL -4294967025 pg_subscription_rel 0 0 NULL NULL NULL -4294967026 pg_stats 0 0 NULL NULL NULL -4294967027 pg_stats_ext 0 0 NULL NULL NULL -4294967028 pg_statistic 0 0 NULL NULL NULL -4294967029 pg_statistic_ext 0 0 NULL NULL NULL -4294967030 pg_statistic_ext_data 0 0 NULL NULL NULL -4294967031 pg_statio_user_tables 0 0 NULL NULL NULL -4294967032 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967033 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967034 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967035 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967036 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967037 pg_statio_all_tables 0 0 NULL NULL NULL -4294967038 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967039 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967040 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967041 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967042 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967043 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967044 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967045 pg_stat_user_tables 0 0 NULL NULL NULL -4294967046 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967047 pg_stat_user_functions 0 0 NULL NULL NULL -4294967048 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967049 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967050 pg_stat_subscription 0 0 NULL NULL NULL -4294967051 pg_stat_ssl 0 0 NULL NULL NULL -4294967052 pg_stat_slru 0 0 NULL NULL NULL -4294967053 pg_stat_replication 0 0 NULL NULL NULL -4294967054 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967055 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967056 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967057 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967058 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967059 pg_stat_gssapi 0 0 NULL NULL NULL -4294967060 pg_stat_database 0 0 NULL NULL NULL -4294967061 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967062 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967063 pg_stat_archiver 0 0 NULL NULL NULL -4294967064 pg_stat_all_tables 0 0 NULL NULL NULL -4294967065 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967066 pg_stat_activity 0 0 NULL NULL NULL -4294967067 pg_shmem_allocations 0 0 NULL NULL NULL -4294967068 pg_shdepend 0 0 NULL NULL NULL -4294967069 pg_shseclabel 0 0 NULL NULL NULL -4294967070 pg_shdescription 0 0 NULL NULL NULL -4294967071 pg_shadow 0 0 NULL NULL NULL -4294967072 pg_settings 0 0 NULL NULL NULL -4294967073 pg_sequences 0 0 NULL NULL NULL -4294967074 pg_sequence 0 0 NULL NULL NULL -4294967075 pg_seclabel 0 0 NULL NULL NULL -4294967076 pg_seclabels 0 0 NULL NULL NULL -4294967077 pg_rules 0 0 NULL NULL NULL -4294967078 pg_roles 0 0 NULL NULL NULL -4294967079 pg_rewrite 0 0 NULL NULL NULL -4294967080 pg_replication_slots 0 0 NULL NULL NULL -4294967081 pg_replication_origin 0 0 NULL NULL NULL -4294967082 pg_replication_origin_status 0 0 NULL NULL NULL -4294967083 pg_range 0 0 NULL NULL NULL -4294967084 pg_publication_tables 0 0 NULL NULL NULL -4294967085 pg_publication 0 0 NULL NULL NULL -4294967086 pg_publication_rel 0 0 NULL NULL NULL -4294967087 pg_proc 0 0 NULL NULL NULL -4294967088 pg_prepared_xacts 0 0 NULL NULL NULL -4294967089 pg_prepared_statements 0 0 NULL NULL NULL -4294967090 pg_policy 0 0 NULL NULL NULL -4294967091 pg_policies 0 0 NULL NULL NULL -4294967092 pg_partitioned_table 0 0 NULL NULL NULL -4294967093 pg_opfamily 0 0 NULL NULL NULL -4294967094 pg_operator 0 0 NULL NULL NULL -4294967095 pg_opclass 0 0 NULL NULL NULL -4294967096 pg_namespace 0 0 NULL NULL NULL -4294967097 pg_matviews 0 0 NULL NULL NULL -4294967098 pg_locks 0 0 NULL NULL NULL -4294967099 pg_largeobject 0 0 NULL NULL NULL -4294967100 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967101 pg_language 0 0 NULL NULL NULL -4294967102 pg_init_privs 0 0 NULL NULL NULL -4294967103 pg_inherits 0 0 NULL NULL NULL -4294967104 pg_indexes 0 0 NULL NULL NULL -4294967105 pg_index 0 0 NULL NULL NULL -4294967106 pg_hba_file_rules 0 0 NULL NULL NULL -4294967107 pg_group 0 0 NULL NULL NULL -4294967108 pg_foreign_table 0 0 NULL NULL NULL -4294967109 pg_foreign_server 0 0 NULL NULL NULL -4294967110 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967111 pg_file_settings 0 0 NULL NULL NULL -4294967112 pg_extension 0 0 NULL NULL NULL -4294967113 pg_event_trigger 0 0 NULL NULL NULL -4294967114 pg_enum 0 0 NULL NULL NULL -4294967115 pg_description 0 0 NULL NULL NULL -4294967116 pg_depend 0 0 NULL NULL NULL -4294967117 pg_default_acl 0 0 NULL NULL NULL -4294967118 pg_db_role_setting 0 0 NULL NULL NULL -4294967119 pg_database 0 0 NULL NULL NULL -4294967120 pg_cursors 0 0 NULL NULL NULL -4294967121 pg_conversion 0 0 NULL NULL NULL -4294967122 pg_constraint 0 0 NULL NULL NULL -4294967123 pg_config 0 0 NULL NULL NULL -4294967124 pg_collation 0 0 NULL NULL NULL -4294967125 pg_class 0 0 NULL NULL NULL -4294967126 pg_cast 0 0 NULL NULL NULL -4294967127 pg_available_extensions 0 0 NULL NULL NULL -4294967128 pg_available_extension_versions 0 0 NULL NULL NULL -4294967129 pg_auth_members 0 0 NULL NULL NULL -4294967130 pg_authid 0 0 NULL NULL NULL -4294967131 pg_attribute 0 0 NULL NULL NULL -4294967132 pg_attrdef 0 0 NULL NULL NULL -4294967133 pg_amproc 0 0 NULL NULL NULL -4294967134 pg_amop 0 0 NULL NULL NULL -4294967135 pg_am 0 0 NULL NULL NULL -4294967136 pg_aggregate 0 0 NULL NULL NULL -4294967138 views 0 0 NULL NULL NULL -4294967139 view_table_usage 0 0 NULL NULL NULL -4294967140 view_routine_usage 0 0 NULL NULL NULL -4294967141 view_column_usage 0 0 NULL NULL NULL -4294967142 user_privileges 0 0 NULL NULL NULL -4294967143 user_mappings 0 0 NULL NULL NULL -4294967144 user_mapping_options 0 0 NULL NULL NULL -4294967145 user_defined_types 0 0 NULL NULL NULL -4294967146 user_attributes 0 0 NULL NULL NULL -4294967147 usage_privileges 0 0 NULL NULL NULL -4294967148 udt_privileges 0 0 NULL NULL NULL -4294967149 type_privileges 0 0 NULL NULL NULL -4294967150 triggers 0 0 NULL NULL NULL -4294967151 triggered_update_columns 0 0 NULL NULL NULL -4294967152 transforms 0 0 NULL NULL NULL -4294967153 tablespaces 0 0 NULL NULL NULL -4294967154 tablespaces_extensions 0 0 NULL NULL NULL -4294967155 tables 0 0 NULL NULL NULL -4294967156 tables_extensions 0 0 NULL NULL NULL -4294967157 table_privileges 0 0 NULL NULL NULL -4294967158 table_constraints_extensions 0 0 NULL NULL NULL -4294967159 table_constraints 0 0 NULL NULL NULL -4294967160 statistics 0 0 NULL NULL NULL -4294967161 st_units_of_measure 0 0 NULL NULL NULL -4294967162 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967163 st_geometry_columns 0 0 NULL NULL NULL -4294967164 session_variables 0 0 NULL NULL NULL -4294967165 sequences 0 0 NULL NULL NULL -4294967166 schema_privileges 0 0 NULL NULL NULL -4294967167 schemata 0 0 NULL NULL NULL -4294967168 schemata_extensions 0 0 NULL NULL NULL -4294967169 sql_sizing 0 0 NULL NULL NULL -4294967170 sql_parts 0 0 NULL NULL NULL -4294967171 sql_implementation_info 0 0 NULL NULL NULL -4294967172 sql_features 0 0 NULL NULL NULL -4294967173 routines 0 0 NULL NULL NULL -4294967174 routine_privileges 0 0 NULL NULL NULL -4294967175 role_usage_grants 0 0 NULL NULL NULL -4294967176 role_udt_grants 0 0 NULL NULL NULL -4294967177 role_table_grants 0 0 NULL NULL NULL -4294967178 role_routine_grants 0 0 NULL NULL NULL -4294967179 role_column_grants 0 0 NULL NULL NULL -4294967180 resource_groups 0 0 NULL NULL NULL -4294967181 referential_constraints 0 0 NULL NULL NULL -4294967182 profiling 0 0 NULL NULL NULL -4294967183 processlist 0 0 NULL NULL NULL -4294967184 plugins 0 0 NULL NULL NULL -4294967185 partitions 0 0 NULL NULL NULL -4294967186 parameters 0 0 NULL NULL NULL -4294967187 optimizer_trace 0 0 NULL NULL NULL -4294967188 keywords 0 0 NULL NULL NULL -4294967189 key_column_usage 0 0 NULL NULL NULL -4294967190 information_schema_catalog_name 0 0 NULL NULL NULL -4294967191 foreign_tables 0 0 NULL NULL NULL -4294967192 foreign_table_options 0 0 NULL NULL NULL -4294967193 foreign_servers 0 0 NULL NULL NULL -4294967194 foreign_server_options 0 0 NULL NULL NULL -4294967195 foreign_data_wrappers 0 0 NULL NULL NULL -4294967196 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967197 files 0 0 NULL NULL NULL -4294967198 events 0 0 NULL NULL NULL -4294967199 engines 0 0 NULL NULL NULL -4294967200 enabled_roles 0 0 NULL NULL NULL -4294967201 element_types 0 0 NULL NULL NULL -4294967202 domains 0 0 NULL NULL NULL -4294967203 domain_udt_usage 0 0 NULL NULL NULL -4294967204 domain_constraints 0 0 NULL NULL NULL -4294967205 data_type_privileges 0 0 NULL NULL NULL -4294967206 constraint_table_usage 0 0 NULL NULL NULL -4294967207 constraint_column_usage 0 0 NULL NULL NULL -4294967208 columns 0 0 NULL NULL NULL -4294967209 columns_extensions 0 0 NULL NULL NULL -4294967210 column_udt_usage 0 0 NULL NULL NULL -4294967211 column_statistics 0 0 NULL NULL NULL -4294967212 column_privileges 0 0 NULL NULL NULL -4294967213 column_options 0 0 NULL NULL NULL -4294967214 column_domain_usage 0 0 NULL NULL NULL -4294967215 column_column_usage 0 0 NULL NULL NULL -4294967216 collations 0 0 NULL NULL NULL -4294967217 collation_character_set_applicability 0 0 NULL NULL NULL -4294967218 check_constraints 0 0 NULL NULL NULL -4294967219 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967220 character_sets 0 0 NULL NULL NULL -4294967221 attributes 0 0 NULL NULL NULL -4294967222 applicable_roles 0 0 NULL NULL NULL -4294967223 administrable_role_authorizations 0 0 NULL NULL NULL -4294967225 super_regions 0 0 NULL NULL NULL -4294967226 pg_catalog_table_is_implemented 0 0 NULL NULL NULL -4294967227 tenant_usage_details 0 0 NULL NULL NULL -4294967228 active_range_feeds 0 0 NULL NULL NULL -4294967229 default_privileges 0 0 NULL NULL NULL -4294967230 regions 0 0 NULL NULL NULL -4294967231 cluster_inflight_traces 0 0 NULL NULL NULL -4294967232 lost_descriptors_with_data 0 0 NULL NULL NULL -4294967233 cross_db_references 0 0 NULL NULL NULL -4294967234 cluster_database_privileges 0 0 NULL NULL NULL -4294967235 invalid_objects 0 0 NULL NULL NULL -4294967236 zones 0 0 NULL NULL NULL -4294967237 transaction_statistics 0 0 NULL NULL NULL -4294967238 node_transaction_statistics 0 0 NULL NULL NULL -4294967239 table_row_statistics 0 0 NULL NULL NULL -4294967240 tables 0 0 NULL NULL NULL -4294967241 table_indexes 0 0 NULL NULL NULL -4294967242 table_columns 0 0 NULL NULL NULL -4294967243 statement_statistics 0 0 NULL NULL NULL -4294967244 session_variables 0 0 NULL NULL NULL -4294967245 session_trace 0 0 NULL NULL NULL -4294967246 schema_changes 0 0 NULL NULL NULL -4294967247 node_runtime_info 0 0 NULL NULL NULL -4294967248 ranges 0 0 NULL NULL NULL -4294967249 ranges_no_leases 0 0 NULL NULL NULL -4294967250 predefined_comments 0 0 NULL NULL NULL -4294967251 partitions 0 0 NULL NULL NULL -4294967252 node_txn_stats 0 0 NULL NULL NULL -4294967253 node_statement_statistics 0 0 NULL NULL NULL -4294967254 node_metrics 0 0 NULL NULL NULL -4294967255 node_sessions 0 0 NULL NULL NULL -4294967256 node_transactions 0 0 NULL NULL NULL -4294967257 node_queries 0 0 NULL NULL NULL -4294967258 node_execution_insights 0 0 NULL NULL NULL -4294967259 node_distsql_flows 0 0 NULL NULL NULL -4294967260 node_contention_events 0 0 NULL NULL NULL -4294967261 leases 0 0 NULL NULL NULL -4294967262 kv_store_status 0 0 NULL NULL NULL -4294967263 kv_node_status 0 0 NULL NULL NULL -4294967264 jobs 0 0 NULL NULL NULL -4294967265 node_inflight_trace_spans 0 0 NULL NULL NULL -4294967266 index_usage_statistics 0 0 NULL NULL NULL -4294967267 index_columns 0 0 NULL NULL NULL -4294967268 transaction_contention_events 0 0 NULL NULL NULL -4294967269 gossip_network 0 0 NULL NULL NULL -4294967270 gossip_liveness 0 0 NULL NULL NULL -4294967271 gossip_alerts 0 0 NULL NULL NULL -4294967272 gossip_nodes 0 0 NULL NULL NULL -4294967273 kv_node_liveness 0 0 NULL NULL NULL -4294967274 forward_dependencies 0 0 NULL NULL NULL -4294967275 feature_usage 0 0 NULL NULL NULL -4294967276 databases 0 0 NULL NULL NULL -4294967277 create_type_statements 0 0 NULL NULL NULL -4294967278 create_statements 0 0 NULL NULL NULL -4294967279 create_schema_statements 0 0 NULL NULL NULL +4294967003 spatial_ref_sys 0 0 NULL NULL NULL +4294967004 geometry_columns 0 0 NULL NULL NULL +4294967005 geography_columns 0 0 NULL NULL NULL +4294967007 pg_views 0 0 NULL NULL NULL +4294967008 pg_user 0 0 NULL NULL NULL +4294967009 pg_user_mappings 0 0 NULL NULL NULL +4294967010 pg_user_mapping 0 0 NULL NULL NULL +4294967011 pg_type 0 0 NULL NULL NULL +4294967012 pg_ts_template 0 0 NULL NULL NULL +4294967013 pg_ts_parser 0 0 NULL NULL NULL +4294967014 pg_ts_dict 0 0 NULL NULL NULL +4294967015 pg_ts_config 0 0 NULL NULL NULL +4294967016 pg_ts_config_map 0 0 NULL NULL NULL +4294967017 pg_trigger 0 0 NULL NULL NULL +4294967018 pg_transform 0 0 NULL NULL NULL +4294967019 pg_timezone_names 0 0 NULL NULL NULL +4294967020 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967021 pg_tablespace 0 0 NULL NULL NULL +4294967022 pg_tables 0 0 NULL NULL NULL +4294967023 pg_subscription 0 0 NULL NULL NULL +4294967024 pg_subscription_rel 0 0 NULL NULL NULL +4294967025 pg_stats 0 0 NULL NULL NULL +4294967026 pg_stats_ext 0 0 NULL NULL NULL +4294967027 pg_statistic 0 0 NULL NULL NULL +4294967028 pg_statistic_ext 0 0 NULL NULL NULL +4294967029 pg_statistic_ext_data 0 0 NULL NULL NULL +4294967030 pg_statio_user_tables 0 0 NULL NULL NULL +4294967031 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967032 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967033 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967034 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967035 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967036 pg_statio_all_tables 0 0 NULL NULL NULL +4294967037 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967038 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967039 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967040 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967041 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967042 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967043 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967044 pg_stat_user_tables 0 0 NULL NULL NULL +4294967045 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967046 pg_stat_user_functions 0 0 NULL NULL NULL +4294967047 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967048 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967049 pg_stat_subscription 0 0 NULL NULL NULL +4294967050 pg_stat_ssl 0 0 NULL NULL NULL +4294967051 pg_stat_slru 0 0 NULL NULL NULL +4294967052 pg_stat_replication 0 0 NULL NULL NULL +4294967053 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967054 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967055 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967056 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967057 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967058 pg_stat_gssapi 0 0 NULL NULL NULL +4294967059 pg_stat_database 0 0 NULL NULL NULL +4294967060 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967061 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967062 pg_stat_archiver 0 0 NULL NULL NULL +4294967063 pg_stat_all_tables 0 0 NULL NULL NULL +4294967064 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967065 pg_stat_activity 0 0 NULL NULL NULL +4294967066 pg_shmem_allocations 0 0 NULL NULL NULL +4294967067 pg_shdepend 0 0 NULL NULL NULL +4294967068 pg_shseclabel 0 0 NULL NULL NULL +4294967069 pg_shdescription 0 0 NULL NULL NULL +4294967070 pg_shadow 0 0 NULL NULL NULL +4294967071 pg_settings 0 0 NULL NULL NULL +4294967072 pg_sequences 0 0 NULL NULL NULL +4294967073 pg_sequence 0 0 NULL NULL NULL +4294967074 pg_seclabel 0 0 NULL NULL NULL +4294967075 pg_seclabels 0 0 NULL NULL NULL +4294967076 pg_rules 0 0 NULL NULL NULL +4294967077 pg_roles 0 0 NULL NULL NULL +4294967078 pg_rewrite 0 0 NULL NULL NULL +4294967079 pg_replication_slots 0 0 NULL NULL NULL +4294967080 pg_replication_origin 0 0 NULL NULL NULL +4294967081 pg_replication_origin_status 0 0 NULL NULL NULL +4294967082 pg_range 0 0 NULL NULL NULL +4294967083 pg_publication_tables 0 0 NULL NULL NULL +4294967084 pg_publication 0 0 NULL NULL NULL +4294967085 pg_publication_rel 0 0 NULL NULL NULL +4294967086 pg_proc 0 0 NULL NULL NULL +4294967087 pg_prepared_xacts 0 0 NULL NULL NULL +4294967088 pg_prepared_statements 0 0 NULL NULL NULL +4294967089 pg_policy 0 0 NULL NULL NULL +4294967090 pg_policies 0 0 NULL NULL NULL +4294967091 pg_partitioned_table 0 0 NULL NULL NULL +4294967092 pg_opfamily 0 0 NULL NULL NULL +4294967093 pg_operator 0 0 NULL NULL NULL +4294967094 pg_opclass 0 0 NULL NULL NULL +4294967095 pg_namespace 0 0 NULL NULL NULL +4294967096 pg_matviews 0 0 NULL NULL NULL +4294967097 pg_locks 0 0 NULL NULL NULL +4294967098 pg_largeobject 0 0 NULL NULL NULL +4294967099 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967100 pg_language 0 0 NULL NULL NULL +4294967101 pg_init_privs 0 0 NULL NULL NULL +4294967102 pg_inherits 0 0 NULL NULL NULL +4294967103 pg_indexes 0 0 NULL NULL NULL +4294967104 pg_index 0 0 NULL NULL NULL +4294967105 pg_hba_file_rules 0 0 NULL NULL NULL +4294967106 pg_group 0 0 NULL NULL NULL +4294967107 pg_foreign_table 0 0 NULL NULL NULL +4294967108 pg_foreign_server 0 0 NULL NULL NULL +4294967109 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967110 pg_file_settings 0 0 NULL NULL NULL +4294967111 pg_extension 0 0 NULL NULL NULL +4294967112 pg_event_trigger 0 0 NULL NULL NULL +4294967113 pg_enum 0 0 NULL NULL NULL +4294967114 pg_description 0 0 NULL NULL NULL +4294967115 pg_depend 0 0 NULL NULL NULL +4294967116 pg_default_acl 0 0 NULL NULL NULL +4294967117 pg_db_role_setting 0 0 NULL NULL NULL +4294967118 pg_database 0 0 NULL NULL NULL +4294967119 pg_cursors 0 0 NULL NULL NULL +4294967120 pg_conversion 0 0 NULL NULL NULL +4294967121 pg_constraint 0 0 NULL NULL NULL +4294967122 pg_config 0 0 NULL NULL NULL +4294967123 pg_collation 0 0 NULL NULL NULL +4294967124 pg_class 0 0 NULL NULL NULL +4294967125 pg_cast 0 0 NULL NULL NULL +4294967126 pg_available_extensions 0 0 NULL NULL NULL +4294967127 pg_available_extension_versions 0 0 NULL NULL NULL +4294967128 pg_auth_members 0 0 NULL NULL NULL +4294967129 pg_authid 0 0 NULL NULL NULL +4294967130 pg_attribute 0 0 NULL NULL NULL +4294967131 pg_attrdef 0 0 NULL NULL NULL +4294967132 pg_amproc 0 0 NULL NULL NULL +4294967133 pg_amop 0 0 NULL NULL NULL +4294967134 pg_am 0 0 NULL NULL NULL +4294967135 pg_aggregate 0 0 NULL NULL NULL +4294967137 views 0 0 NULL NULL NULL +4294967138 view_table_usage 0 0 NULL NULL NULL +4294967139 view_routine_usage 0 0 NULL NULL NULL +4294967140 view_column_usage 0 0 NULL NULL NULL +4294967141 user_privileges 0 0 NULL NULL NULL +4294967142 user_mappings 0 0 NULL NULL NULL +4294967143 user_mapping_options 0 0 NULL NULL NULL +4294967144 user_defined_types 0 0 NULL NULL NULL +4294967145 user_attributes 0 0 NULL NULL NULL +4294967146 usage_privileges 0 0 NULL NULL NULL +4294967147 udt_privileges 0 0 NULL NULL NULL +4294967148 type_privileges 0 0 NULL NULL NULL +4294967149 triggers 0 0 NULL NULL NULL +4294967150 triggered_update_columns 0 0 NULL NULL NULL +4294967151 transforms 0 0 NULL NULL NULL +4294967152 tablespaces 0 0 NULL NULL NULL +4294967153 tablespaces_extensions 0 0 NULL NULL NULL +4294967154 tables 0 0 NULL NULL NULL +4294967155 tables_extensions 0 0 NULL NULL NULL +4294967156 table_privileges 0 0 NULL NULL NULL +4294967157 table_constraints_extensions 0 0 NULL NULL NULL +4294967158 table_constraints 0 0 NULL NULL NULL +4294967159 statistics 0 0 NULL NULL NULL +4294967160 st_units_of_measure 0 0 NULL NULL NULL +4294967161 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967162 st_geometry_columns 0 0 NULL NULL NULL +4294967163 session_variables 0 0 NULL NULL NULL +4294967164 sequences 0 0 NULL NULL NULL +4294967165 schema_privileges 0 0 NULL NULL NULL +4294967166 schemata 0 0 NULL NULL NULL +4294967167 schemata_extensions 0 0 NULL NULL NULL +4294967168 sql_sizing 0 0 NULL NULL NULL +4294967169 sql_parts 0 0 NULL NULL NULL +4294967170 sql_implementation_info 0 0 NULL NULL NULL +4294967171 sql_features 0 0 NULL NULL NULL +4294967172 routines 0 0 NULL NULL NULL +4294967173 routine_privileges 0 0 NULL NULL NULL +4294967174 role_usage_grants 0 0 NULL NULL NULL +4294967175 role_udt_grants 0 0 NULL NULL NULL +4294967176 role_table_grants 0 0 NULL NULL NULL +4294967177 role_routine_grants 0 0 NULL NULL NULL +4294967178 role_column_grants 0 0 NULL NULL NULL +4294967179 resource_groups 0 0 NULL NULL NULL +4294967180 referential_constraints 0 0 NULL NULL NULL +4294967181 profiling 0 0 NULL NULL NULL +4294967182 processlist 0 0 NULL NULL NULL +4294967183 plugins 0 0 NULL NULL NULL +4294967184 partitions 0 0 NULL NULL NULL +4294967185 parameters 0 0 NULL NULL NULL +4294967186 optimizer_trace 0 0 NULL NULL NULL +4294967187 keywords 0 0 NULL NULL NULL +4294967188 key_column_usage 0 0 NULL NULL NULL +4294967189 information_schema_catalog_name 0 0 NULL NULL NULL +4294967190 foreign_tables 0 0 NULL NULL NULL +4294967191 foreign_table_options 0 0 NULL NULL NULL +4294967192 foreign_servers 0 0 NULL NULL NULL +4294967193 foreign_server_options 0 0 NULL NULL NULL +4294967194 foreign_data_wrappers 0 0 NULL NULL NULL +4294967195 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967196 files 0 0 NULL NULL NULL +4294967197 events 0 0 NULL NULL NULL +4294967198 engines 0 0 NULL NULL NULL +4294967199 enabled_roles 0 0 NULL NULL NULL +4294967200 element_types 0 0 NULL NULL NULL +4294967201 domains 0 0 NULL NULL NULL +4294967202 domain_udt_usage 0 0 NULL NULL NULL +4294967203 domain_constraints 0 0 NULL NULL NULL +4294967204 data_type_privileges 0 0 NULL NULL NULL +4294967205 constraint_table_usage 0 0 NULL NULL NULL +4294967206 constraint_column_usage 0 0 NULL NULL NULL +4294967207 columns 0 0 NULL NULL NULL +4294967208 columns_extensions 0 0 NULL NULL NULL +4294967209 column_udt_usage 0 0 NULL NULL NULL +4294967210 column_statistics 0 0 NULL NULL NULL +4294967211 column_privileges 0 0 NULL NULL NULL +4294967212 column_options 0 0 NULL NULL NULL +4294967213 column_domain_usage 0 0 NULL NULL NULL +4294967214 column_column_usage 0 0 NULL NULL NULL +4294967215 collations 0 0 NULL NULL NULL +4294967216 collation_character_set_applicability 0 0 NULL NULL NULL +4294967217 check_constraints 0 0 NULL NULL NULL +4294967218 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967219 character_sets 0 0 NULL NULL NULL +4294967220 attributes 0 0 NULL NULL NULL +4294967221 applicable_roles 0 0 NULL NULL NULL +4294967222 administrable_role_authorizations 0 0 NULL NULL NULL +4294967224 super_regions 0 0 NULL NULL NULL +4294967225 pg_catalog_table_is_implemented 0 0 NULL NULL NULL +4294967226 tenant_usage_details 0 0 NULL NULL NULL +4294967227 active_range_feeds 0 0 NULL NULL NULL +4294967228 default_privileges 0 0 NULL NULL NULL +4294967229 regions 0 0 NULL NULL NULL +4294967230 cluster_inflight_traces 0 0 NULL NULL NULL +4294967231 lost_descriptors_with_data 0 0 NULL NULL NULL +4294967232 cross_db_references 0 0 NULL NULL NULL +4294967233 cluster_database_privileges 0 0 NULL NULL NULL +4294967234 invalid_objects 0 0 NULL NULL NULL +4294967235 zones 0 0 NULL NULL NULL +4294967236 transaction_statistics 0 0 NULL NULL NULL +4294967237 node_transaction_statistics 0 0 NULL NULL NULL +4294967238 table_row_statistics 0 0 NULL NULL NULL +4294967239 tables 0 0 NULL NULL NULL +4294967240 table_indexes 0 0 NULL NULL NULL +4294967241 table_columns 0 0 NULL NULL NULL +4294967242 statement_statistics 0 0 NULL NULL NULL +4294967243 session_variables 0 0 NULL NULL NULL +4294967244 session_trace 0 0 NULL NULL NULL +4294967245 schema_changes 0 0 NULL NULL NULL +4294967246 node_runtime_info 0 0 NULL NULL NULL +4294967247 ranges 0 0 NULL NULL NULL +4294967248 ranges_no_leases 0 0 NULL NULL NULL +4294967249 predefined_comments 0 0 NULL NULL NULL +4294967250 partitions 0 0 NULL NULL NULL +4294967251 node_txn_stats 0 0 NULL NULL NULL +4294967252 node_statement_statistics 0 0 NULL NULL NULL +4294967253 node_metrics 0 0 NULL NULL NULL +4294967254 node_sessions 0 0 NULL NULL NULL +4294967255 node_transactions 0 0 NULL NULL NULL +4294967256 node_queries 0 0 NULL NULL NULL +4294967257 node_execution_insights 0 0 NULL NULL NULL +4294967258 node_distsql_flows 0 0 NULL NULL NULL +4294967259 node_contention_events 0 0 NULL NULL NULL +4294967260 leases 0 0 NULL NULL NULL +4294967261 kv_store_status 0 0 NULL NULL NULL +4294967262 kv_node_status 0 0 NULL NULL NULL +4294967263 jobs 0 0 NULL NULL NULL +4294967264 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967265 index_usage_statistics 0 0 NULL NULL NULL +4294967266 index_columns 0 0 NULL NULL NULL +4294967267 transaction_contention_events 0 0 NULL NULL NULL +4294967268 gossip_network 0 0 NULL NULL NULL +4294967269 gossip_liveness 0 0 NULL NULL NULL +4294967270 gossip_alerts 0 0 NULL NULL NULL +4294967271 gossip_nodes 0 0 NULL NULL NULL +4294967272 kv_node_liveness 0 0 NULL NULL NULL +4294967273 forward_dependencies 0 0 NULL NULL NULL +4294967274 feature_usage 0 0 NULL NULL NULL +4294967275 databases 0 0 NULL NULL NULL +4294967276 create_type_statements 0 0 NULL NULL NULL +4294967277 create_statements 0 0 NULL NULL NULL +4294967278 create_schema_statements 0 0 NULL NULL NULL +4294967279 create_function_statements 0 0 NULL NULL NULL 4294967280 cluster_transaction_statistics 0 0 NULL NULL NULL 4294967281 cluster_statement_statistics 0 0 NULL NULL NULL 4294967282 cluster_settings 0 0 NULL NULL NULL @@ -3788,286 +3793,287 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967228 4294967125 0 node-level table listing all currently running range feeds -4294967294 4294967125 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967125 0 built-in functions (RAM/static) -4294967288 4294967125 0 contention information (cluster RPC; expensive!) -4294967234 4294967125 0 virtual table with database privileges -4294967287 4294967125 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967231 4294967125 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967125 0 cluster-wide locks held in lock tables. Querying this table is an -4294967285 4294967125 0 running queries visible by current user (cluster RPC; expensive!) -4294967283 4294967125 0 running sessions visible to current user (cluster RPC; expensive!) -4294967282 4294967125 0 cluster settings (RAM) -4294967281 4294967125 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967280 4294967125 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967284 4294967125 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967279 4294967125 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967278 4294967125 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967277 4294967125 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967233 4294967125 0 virtual table with cross db references -4294967276 4294967125 0 databases accessible by the current user (KV scan) -4294967229 4294967125 0 virtual table with default privileges -4294967275 4294967125 0 telemetry counters (RAM; local node only) -4294967274 4294967125 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967271 4294967125 0 locally known gossiped health alerts (RAM; local node only) -4294967270 4294967125 0 locally known gossiped node liveness (RAM; local node only) -4294967269 4294967125 0 locally known edges in the gossip network (RAM; local node only) -4294967272 4294967125 0 locally known gossiped node details (RAM; local node only) -4294967267 4294967125 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967266 4294967125 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967235 4294967125 0 virtual table to validate descriptors -4294967264 4294967125 0 decoded job metadata from system.jobs (KV scan) -4294967273 4294967125 0 node liveness status, as seen by kv -4294967263 4294967125 0 node details across the entire cluster (cluster RPC; expensive!) -4294967262 4294967125 0 store details and status (cluster RPC; expensive!) -4294967261 4294967125 0 acquired table leases (RAM; local node only) -4294967232 4294967125 0 virtual table with table descriptors that still have data -4294967293 4294967125 0 detailed identification strings (RAM, local node only) -4294967260 4294967125 0 contention information (RAM; local node only) -4294967259 4294967125 0 DistSQL remote flows information (RAM; local node only) -4294967265 4294967125 0 in-flight spans (RAM; local node only) -4294967254 4294967125 0 current values for metrics (RAM; local node only) -4294967257 4294967125 0 running queries visible by current user (RAM; local node only) -4294967247 4294967125 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967255 4294967125 0 running sessions visible by current user (RAM; local node only) -4294967253 4294967125 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967238 4294967125 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967256 4294967125 0 running user transactions visible by the current user (RAM; local node only) -4294967252 4294967125 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967251 4294967125 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967226 4294967125 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967250 4294967125 0 comments for predefined virtual tables (RAM/static) -4294967249 4294967125 0 range metadata without leaseholder details (KV join; expensive!) -4294967230 4294967125 0 available regions for the cluster -4294967246 4294967125 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967245 4294967125 0 session trace accumulated so far (RAM) -4294967244 4294967125 0 session variables (RAM) -4294967225 4294967125 0 list super regions of databases visible to the current user -4294967242 4294967125 0 details for all columns accessible by current user in current database (KV scan) -4294967241 4294967125 0 indexes accessible by current user in current database (KV scan) -4294967239 4294967125 0 stats for all tables accessible by current user in current database as of 10s ago -4294967240 4294967125 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967268 4294967125 0 cluster-wide transaction contention events. Querying this table is an -4294967236 4294967125 0 decoded zone configurations from system.zones (KV scan) -4294967223 4294967125 0 roles for which the current user has admin option -4294967222 4294967125 0 roles available to the current user -4294967221 4294967125 0 attributes was created for compatibility and is currently unimplemented -4294967220 4294967125 0 character sets available in the current database -4294967219 4294967125 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967218 4294967125 0 check constraints -4294967217 4294967125 0 identifies which character set the available collations are -4294967216 4294967125 0 shows the collations available in the current database -4294967215 4294967125 0 column_column_usage was created for compatibility and is currently unimplemented -4294967214 4294967125 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967213 4294967125 0 column_options was created for compatibility and is currently unimplemented -4294967212 4294967125 0 column privilege grants (incomplete) -4294967211 4294967125 0 column_statistics was created for compatibility and is currently unimplemented -4294967210 4294967125 0 columns with user defined types -4294967208 4294967125 0 table and view columns (incomplete) -4294967209 4294967125 0 columns_extensions was created for compatibility and is currently unimplemented -4294967207 4294967125 0 columns usage by constraints -4294967206 4294967125 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967205 4294967125 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967204 4294967125 0 domain_constraints was created for compatibility and is currently unimplemented -4294967203 4294967125 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967202 4294967125 0 domains was created for compatibility and is currently unimplemented -4294967201 4294967125 0 element_types was created for compatibility and is currently unimplemented -4294967200 4294967125 0 roles for the current user -4294967199 4294967125 0 engines was created for compatibility and is currently unimplemented -4294967198 4294967125 0 events was created for compatibility and is currently unimplemented -4294967197 4294967125 0 files was created for compatibility and is currently unimplemented -4294967196 4294967125 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967195 4294967125 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967194 4294967125 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967193 4294967125 0 foreign_servers was created for compatibility and is currently unimplemented -4294967192 4294967125 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967191 4294967125 0 foreign_tables was created for compatibility and is currently unimplemented -4294967190 4294967125 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967189 4294967125 0 column usage by indexes and key constraints -4294967188 4294967125 0 keywords was created for compatibility and is currently unimplemented -4294967187 4294967125 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967186 4294967125 0 built-in function parameters (empty - introspection not yet supported) -4294967185 4294967125 0 partitions was created for compatibility and is currently unimplemented -4294967184 4294967125 0 plugins was created for compatibility and is currently unimplemented -4294967183 4294967125 0 processlist was created for compatibility and is currently unimplemented -4294967182 4294967125 0 profiling was created for compatibility and is currently unimplemented -4294967181 4294967125 0 foreign key constraints -4294967180 4294967125 0 resource_groups was created for compatibility and is currently unimplemented -4294967179 4294967125 0 role_column_grants was created for compatibility and is currently unimplemented -4294967178 4294967125 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967177 4294967125 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967176 4294967125 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967175 4294967125 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967174 4294967125 0 routine_privileges was created for compatibility and is currently unimplemented -4294967173 4294967125 0 built-in functions (empty - introspection not yet supported) -4294967166 4294967125 0 schema privileges (incomplete; may contain excess users or roles) -4294967167 4294967125 0 database schemas (may contain schemata without permission) -4294967168 4294967125 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967165 4294967125 0 sequences -4294967164 4294967125 0 exposes the session variables. -4294967172 4294967125 0 sql_features was created for compatibility and is currently unimplemented -4294967171 4294967125 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967170 4294967125 0 sql_parts was created for compatibility and is currently unimplemented -4294967169 4294967125 0 sql_sizing was created for compatibility and is currently unimplemented -4294967163 4294967125 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967162 4294967125 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967161 4294967125 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967160 4294967125 0 index metadata and statistics (incomplete) -4294967159 4294967125 0 table constraints -4294967158 4294967125 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967157 4294967125 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967155 4294967125 0 tables and views -4294967156 4294967125 0 tables_extensions was created for compatibility and is currently unimplemented -4294967153 4294967125 0 tablespaces was created for compatibility and is currently unimplemented -4294967154 4294967125 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967152 4294967125 0 transforms was created for compatibility and is currently unimplemented -4294967151 4294967125 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967150 4294967125 0 triggers was created for compatibility and is currently unimplemented -4294967149 4294967125 0 type privileges (incomplete; may contain excess users or roles) -4294967148 4294967125 0 udt_privileges was created for compatibility and is currently unimplemented -4294967147 4294967125 0 usage_privileges was created for compatibility and is currently unimplemented -4294967146 4294967125 0 user_attributes was created for compatibility and is currently unimplemented -4294967145 4294967125 0 user_defined_types was created for compatibility and is currently unimplemented -4294967144 4294967125 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967143 4294967125 0 user_mappings was created for compatibility and is currently unimplemented -4294967142 4294967125 0 grantable privileges (incomplete) -4294967141 4294967125 0 view_column_usage was created for compatibility and is currently unimplemented -4294967140 4294967125 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967139 4294967125 0 view_table_usage was created for compatibility and is currently unimplemented -4294967138 4294967125 0 views (incomplete) -4294967136 4294967125 0 aggregated built-in functions (incomplete) -4294967135 4294967125 0 index access methods (incomplete) -4294967134 4294967125 0 pg_amop was created for compatibility and is currently unimplemented -4294967133 4294967125 0 pg_amproc was created for compatibility and is currently unimplemented -4294967132 4294967125 0 column default values -4294967131 4294967125 0 table columns (incomplete - see also information_schema.columns) -4294967129 4294967125 0 role membership -4294967130 4294967125 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967128 4294967125 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967127 4294967125 0 available extensions -4294967126 4294967125 0 casts (empty - needs filling out) -4294967125 4294967125 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967124 4294967125 0 available collations (incomplete) -4294967123 4294967125 0 pg_config was created for compatibility and is currently unimplemented -4294967122 4294967125 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967121 4294967125 0 encoding conversions (empty - unimplemented) -4294967120 4294967125 0 contains currently active SQL cursors created with DECLARE -4294967119 4294967125 0 available databases (incomplete) -4294967118 4294967125 0 contains the default values that have been configured for session variables -4294967117 4294967125 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967116 4294967125 0 dependency relationships (incomplete) -4294967115 4294967125 0 object comments -4294967114 4294967125 0 enum types and labels (empty - feature does not exist) -4294967113 4294967125 0 event triggers (empty - feature does not exist) -4294967112 4294967125 0 installed extensions (empty - feature does not exist) -4294967111 4294967125 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967110 4294967125 0 foreign data wrappers (empty - feature does not exist) -4294967109 4294967125 0 foreign servers (empty - feature does not exist) -4294967108 4294967125 0 foreign tables (empty - feature does not exist) -4294967107 4294967125 0 pg_group was created for compatibility and is currently unimplemented -4294967106 4294967125 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967105 4294967125 0 indexes (incomplete) -4294967104 4294967125 0 index creation statements -4294967103 4294967125 0 table inheritance hierarchy (empty - feature does not exist) -4294967102 4294967125 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967101 4294967125 0 available languages (empty - feature does not exist) -4294967099 4294967125 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967100 4294967125 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967098 4294967125 0 locks held by active processes (empty - feature does not exist) -4294967097 4294967125 0 available materialized views (empty - feature does not exist) -4294967096 4294967125 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967095 4294967125 0 opclass (empty - Operator classes not supported yet) -4294967094 4294967125 0 operators (incomplete) -4294967093 4294967125 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967092 4294967125 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967091 4294967125 0 pg_policies was created for compatibility and is currently unimplemented -4294967090 4294967125 0 pg_policy was created for compatibility and is currently unimplemented -4294967089 4294967125 0 prepared statements -4294967088 4294967125 0 prepared transactions (empty - feature does not exist) -4294967087 4294967125 0 built-in functions (incomplete) -4294967085 4294967125 0 pg_publication was created for compatibility and is currently unimplemented -4294967086 4294967125 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967084 4294967125 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967083 4294967125 0 range types (empty - feature does not exist) -4294967081 4294967125 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967082 4294967125 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967080 4294967125 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967079 4294967125 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967078 4294967125 0 database roles -4294967077 4294967125 0 pg_rules was created for compatibility and is currently unimplemented -4294967075 4294967125 0 security labels (empty - feature does not exist) -4294967076 4294967125 0 security labels (empty) -4294967074 4294967125 0 sequences (see also information_schema.sequences) -4294967073 4294967125 0 pg_sequences is very similar as pg_sequence. -4294967072 4294967125 0 session variables (incomplete) -4294967071 4294967125 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967068 4294967125 0 Shared Dependencies (Roles depending on objects). -4294967070 4294967125 0 shared object comments -4294967067 4294967125 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967069 4294967125 0 shared security labels (empty - feature not supported) -4294967066 4294967125 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967065 4294967125 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967064 4294967125 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967063 4294967125 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967062 4294967125 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967060 4294967125 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967061 4294967125 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967059 4294967125 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967058 4294967125 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967057 4294967125 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967056 4294967125 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967055 4294967125 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967054 4294967125 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967053 4294967125 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967052 4294967125 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967051 4294967125 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967050 4294967125 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967049 4294967125 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967048 4294967125 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967047 4294967125 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967046 4294967125 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967045 4294967125 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967044 4294967125 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967043 4294967125 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967042 4294967125 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967041 4294967125 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967040 4294967125 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967039 4294967125 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967038 4294967125 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967037 4294967125 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967036 4294967125 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967035 4294967125 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967034 4294967125 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967033 4294967125 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967032 4294967125 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967031 4294967125 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967028 4294967125 0 pg_statistic was created for compatibility and is currently unimplemented -4294967029 4294967125 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967030 4294967125 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967026 4294967125 0 pg_stats was created for compatibility and is currently unimplemented -4294967027 4294967125 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967024 4294967125 0 pg_subscription was created for compatibility and is currently unimplemented -4294967025 4294967125 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967023 4294967125 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967022 4294967125 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967021 4294967125 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967020 4294967125 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967019 4294967125 0 pg_transform was created for compatibility and is currently unimplemented -4294967018 4294967125 0 triggers (empty - feature does not exist) -4294967016 4294967125 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967017 4294967125 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967015 4294967125 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967014 4294967125 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967013 4294967125 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967012 4294967125 0 scalar types (incomplete) -4294967009 4294967125 0 database users -4294967011 4294967125 0 local to remote user mapping (empty - feature does not exist) -4294967010 4294967125 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967008 4294967125 0 view definitions (incomplete - see also information_schema.views) -4294967006 4294967125 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967005 4294967125 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967004 4294967125 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967227 4294967124 0 node-level table listing all currently running range feeds +4294967294 4294967124 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967124 0 built-in functions (RAM/static) +4294967288 4294967124 0 contention information (cluster RPC; expensive!) +4294967233 4294967124 0 virtual table with database privileges +4294967287 4294967124 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967230 4294967124 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967286 4294967124 0 cluster-wide locks held in lock tables. Querying this table is an +4294967285 4294967124 0 running queries visible by current user (cluster RPC; expensive!) +4294967283 4294967124 0 running sessions visible to current user (cluster RPC; expensive!) +4294967282 4294967124 0 cluster settings (RAM) +4294967281 4294967124 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967280 4294967124 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967284 4294967124 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967279 4294967124 0 CREATE statements for all user-defined functions +4294967278 4294967124 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967277 4294967124 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967276 4294967124 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967232 4294967124 0 virtual table with cross db references +4294967275 4294967124 0 databases accessible by the current user (KV scan) +4294967228 4294967124 0 virtual table with default privileges +4294967274 4294967124 0 telemetry counters (RAM; local node only) +4294967273 4294967124 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967270 4294967124 0 locally known gossiped health alerts (RAM; local node only) +4294967269 4294967124 0 locally known gossiped node liveness (RAM; local node only) +4294967268 4294967124 0 locally known edges in the gossip network (RAM; local node only) +4294967271 4294967124 0 locally known gossiped node details (RAM; local node only) +4294967266 4294967124 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967265 4294967124 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967234 4294967124 0 virtual table to validate descriptors +4294967263 4294967124 0 decoded job metadata from system.jobs (KV scan) +4294967272 4294967124 0 node liveness status, as seen by kv +4294967262 4294967124 0 node details across the entire cluster (cluster RPC; expensive!) +4294967261 4294967124 0 store details and status (cluster RPC; expensive!) +4294967260 4294967124 0 acquired table leases (RAM; local node only) +4294967231 4294967124 0 virtual table with table descriptors that still have data +4294967293 4294967124 0 detailed identification strings (RAM, local node only) +4294967259 4294967124 0 contention information (RAM; local node only) +4294967258 4294967124 0 DistSQL remote flows information (RAM; local node only) +4294967264 4294967124 0 in-flight spans (RAM; local node only) +4294967253 4294967124 0 current values for metrics (RAM; local node only) +4294967256 4294967124 0 running queries visible by current user (RAM; local node only) +4294967246 4294967124 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967254 4294967124 0 running sessions visible by current user (RAM; local node only) +4294967252 4294967124 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967237 4294967124 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967255 4294967124 0 running user transactions visible by the current user (RAM; local node only) +4294967251 4294967124 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967250 4294967124 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967225 4294967124 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967249 4294967124 0 comments for predefined virtual tables (RAM/static) +4294967248 4294967124 0 range metadata without leaseholder details (KV join; expensive!) +4294967229 4294967124 0 available regions for the cluster +4294967245 4294967124 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967244 4294967124 0 session trace accumulated so far (RAM) +4294967243 4294967124 0 session variables (RAM) +4294967224 4294967124 0 list super regions of databases visible to the current user +4294967241 4294967124 0 details for all columns accessible by current user in current database (KV scan) +4294967240 4294967124 0 indexes accessible by current user in current database (KV scan) +4294967238 4294967124 0 stats for all tables accessible by current user in current database as of 10s ago +4294967239 4294967124 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967267 4294967124 0 cluster-wide transaction contention events. Querying this table is an +4294967235 4294967124 0 decoded zone configurations from system.zones (KV scan) +4294967222 4294967124 0 roles for which the current user has admin option +4294967221 4294967124 0 roles available to the current user +4294967220 4294967124 0 attributes was created for compatibility and is currently unimplemented +4294967219 4294967124 0 character sets available in the current database +4294967218 4294967124 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967217 4294967124 0 check constraints +4294967216 4294967124 0 identifies which character set the available collations are +4294967215 4294967124 0 shows the collations available in the current database +4294967214 4294967124 0 column_column_usage was created for compatibility and is currently unimplemented +4294967213 4294967124 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967212 4294967124 0 column_options was created for compatibility and is currently unimplemented +4294967211 4294967124 0 column privilege grants (incomplete) +4294967210 4294967124 0 column_statistics was created for compatibility and is currently unimplemented +4294967209 4294967124 0 columns with user defined types +4294967207 4294967124 0 table and view columns (incomplete) +4294967208 4294967124 0 columns_extensions was created for compatibility and is currently unimplemented +4294967206 4294967124 0 columns usage by constraints +4294967205 4294967124 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967204 4294967124 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967203 4294967124 0 domain_constraints was created for compatibility and is currently unimplemented +4294967202 4294967124 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967201 4294967124 0 domains was created for compatibility and is currently unimplemented +4294967200 4294967124 0 element_types was created for compatibility and is currently unimplemented +4294967199 4294967124 0 roles for the current user +4294967198 4294967124 0 engines was created for compatibility and is currently unimplemented +4294967197 4294967124 0 events was created for compatibility and is currently unimplemented +4294967196 4294967124 0 files was created for compatibility and is currently unimplemented +4294967195 4294967124 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967194 4294967124 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967193 4294967124 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967192 4294967124 0 foreign_servers was created for compatibility and is currently unimplemented +4294967191 4294967124 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967190 4294967124 0 foreign_tables was created for compatibility and is currently unimplemented +4294967189 4294967124 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967188 4294967124 0 column usage by indexes and key constraints +4294967187 4294967124 0 keywords was created for compatibility and is currently unimplemented +4294967186 4294967124 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967185 4294967124 0 built-in function parameters (empty - introspection not yet supported) +4294967184 4294967124 0 partitions was created for compatibility and is currently unimplemented +4294967183 4294967124 0 plugins was created for compatibility and is currently unimplemented +4294967182 4294967124 0 processlist was created for compatibility and is currently unimplemented +4294967181 4294967124 0 profiling was created for compatibility and is currently unimplemented +4294967180 4294967124 0 foreign key constraints +4294967179 4294967124 0 resource_groups was created for compatibility and is currently unimplemented +4294967178 4294967124 0 role_column_grants was created for compatibility and is currently unimplemented +4294967177 4294967124 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967176 4294967124 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967175 4294967124 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967174 4294967124 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967173 4294967124 0 routine_privileges was created for compatibility and is currently unimplemented +4294967172 4294967124 0 built-in functions (empty - introspection not yet supported) +4294967165 4294967124 0 schema privileges (incomplete; may contain excess users or roles) +4294967166 4294967124 0 database schemas (may contain schemata without permission) +4294967167 4294967124 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967164 4294967124 0 sequences +4294967163 4294967124 0 exposes the session variables. +4294967171 4294967124 0 sql_features was created for compatibility and is currently unimplemented +4294967170 4294967124 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967169 4294967124 0 sql_parts was created for compatibility and is currently unimplemented +4294967168 4294967124 0 sql_sizing was created for compatibility and is currently unimplemented +4294967162 4294967124 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967161 4294967124 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967160 4294967124 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967159 4294967124 0 index metadata and statistics (incomplete) +4294967158 4294967124 0 table constraints +4294967157 4294967124 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967156 4294967124 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967154 4294967124 0 tables and views +4294967155 4294967124 0 tables_extensions was created for compatibility and is currently unimplemented +4294967152 4294967124 0 tablespaces was created for compatibility and is currently unimplemented +4294967153 4294967124 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967151 4294967124 0 transforms was created for compatibility and is currently unimplemented +4294967150 4294967124 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967149 4294967124 0 triggers was created for compatibility and is currently unimplemented +4294967148 4294967124 0 type privileges (incomplete; may contain excess users or roles) +4294967147 4294967124 0 udt_privileges was created for compatibility and is currently unimplemented +4294967146 4294967124 0 usage_privileges was created for compatibility and is currently unimplemented +4294967145 4294967124 0 user_attributes was created for compatibility and is currently unimplemented +4294967144 4294967124 0 user_defined_types was created for compatibility and is currently unimplemented +4294967143 4294967124 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967142 4294967124 0 user_mappings was created for compatibility and is currently unimplemented +4294967141 4294967124 0 grantable privileges (incomplete) +4294967140 4294967124 0 view_column_usage was created for compatibility and is currently unimplemented +4294967139 4294967124 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967138 4294967124 0 view_table_usage was created for compatibility and is currently unimplemented +4294967137 4294967124 0 views (incomplete) +4294967135 4294967124 0 aggregated built-in functions (incomplete) +4294967134 4294967124 0 index access methods (incomplete) +4294967133 4294967124 0 pg_amop was created for compatibility and is currently unimplemented +4294967132 4294967124 0 pg_amproc was created for compatibility and is currently unimplemented +4294967131 4294967124 0 column default values +4294967130 4294967124 0 table columns (incomplete - see also information_schema.columns) +4294967128 4294967124 0 role membership +4294967129 4294967124 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967127 4294967124 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967126 4294967124 0 available extensions +4294967125 4294967124 0 casts (empty - needs filling out) +4294967124 4294967124 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967123 4294967124 0 available collations (incomplete) +4294967122 4294967124 0 pg_config was created for compatibility and is currently unimplemented +4294967121 4294967124 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967120 4294967124 0 encoding conversions (empty - unimplemented) +4294967119 4294967124 0 contains currently active SQL cursors created with DECLARE +4294967118 4294967124 0 available databases (incomplete) +4294967117 4294967124 0 contains the default values that have been configured for session variables +4294967116 4294967124 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967115 4294967124 0 dependency relationships (incomplete) +4294967114 4294967124 0 object comments +4294967113 4294967124 0 enum types and labels (empty - feature does not exist) +4294967112 4294967124 0 event triggers (empty - feature does not exist) +4294967111 4294967124 0 installed extensions (empty - feature does not exist) +4294967110 4294967124 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967109 4294967124 0 foreign data wrappers (empty - feature does not exist) +4294967108 4294967124 0 foreign servers (empty - feature does not exist) +4294967107 4294967124 0 foreign tables (empty - feature does not exist) +4294967106 4294967124 0 pg_group was created for compatibility and is currently unimplemented +4294967105 4294967124 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967104 4294967124 0 indexes (incomplete) +4294967103 4294967124 0 index creation statements +4294967102 4294967124 0 table inheritance hierarchy (empty - feature does not exist) +4294967101 4294967124 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967100 4294967124 0 available languages (empty - feature does not exist) +4294967098 4294967124 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967099 4294967124 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967097 4294967124 0 locks held by active processes (empty - feature does not exist) +4294967096 4294967124 0 available materialized views (empty - feature does not exist) +4294967095 4294967124 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967094 4294967124 0 opclass (empty - Operator classes not supported yet) +4294967093 4294967124 0 operators (incomplete) +4294967092 4294967124 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967091 4294967124 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967090 4294967124 0 pg_policies was created for compatibility and is currently unimplemented +4294967089 4294967124 0 pg_policy was created for compatibility and is currently unimplemented +4294967088 4294967124 0 prepared statements +4294967087 4294967124 0 prepared transactions (empty - feature does not exist) +4294967086 4294967124 0 built-in functions (incomplete) +4294967084 4294967124 0 pg_publication was created for compatibility and is currently unimplemented +4294967085 4294967124 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967083 4294967124 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967082 4294967124 0 range types (empty - feature does not exist) +4294967080 4294967124 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967081 4294967124 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967079 4294967124 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967078 4294967124 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967077 4294967124 0 database roles +4294967076 4294967124 0 pg_rules was created for compatibility and is currently unimplemented +4294967074 4294967124 0 security labels (empty - feature does not exist) +4294967075 4294967124 0 security labels (empty) +4294967073 4294967124 0 sequences (see also information_schema.sequences) +4294967072 4294967124 0 pg_sequences is very similar as pg_sequence. +4294967071 4294967124 0 session variables (incomplete) +4294967070 4294967124 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967067 4294967124 0 Shared Dependencies (Roles depending on objects). +4294967069 4294967124 0 shared object comments +4294967066 4294967124 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967068 4294967124 0 shared security labels (empty - feature not supported) +4294967065 4294967124 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967064 4294967124 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967063 4294967124 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967062 4294967124 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967061 4294967124 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967059 4294967124 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967060 4294967124 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967058 4294967124 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967057 4294967124 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967056 4294967124 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967055 4294967124 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967054 4294967124 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967053 4294967124 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967052 4294967124 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967051 4294967124 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967050 4294967124 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967049 4294967124 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967048 4294967124 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967047 4294967124 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967046 4294967124 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967045 4294967124 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967044 4294967124 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967043 4294967124 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967042 4294967124 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967041 4294967124 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967040 4294967124 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967039 4294967124 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967038 4294967124 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967037 4294967124 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967036 4294967124 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967035 4294967124 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967034 4294967124 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967033 4294967124 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967032 4294967124 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967031 4294967124 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967030 4294967124 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967027 4294967124 0 pg_statistic was created for compatibility and is currently unimplemented +4294967028 4294967124 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967029 4294967124 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967025 4294967124 0 pg_stats was created for compatibility and is currently unimplemented +4294967026 4294967124 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967023 4294967124 0 pg_subscription was created for compatibility and is currently unimplemented +4294967024 4294967124 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967022 4294967124 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967021 4294967124 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967020 4294967124 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967019 4294967124 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967018 4294967124 0 pg_transform was created for compatibility and is currently unimplemented +4294967017 4294967124 0 triggers (empty - feature does not exist) +4294967015 4294967124 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967016 4294967124 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967014 4294967124 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967013 4294967124 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967012 4294967124 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967011 4294967124 0 scalar types (incomplete) +4294967008 4294967124 0 database users +4294967010 4294967124 0 local to remote user mapping (empty - feature does not exist) +4294967009 4294967124 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967007 4294967124 0 view definitions (incomplete - see also information_schema.views) +4294967005 4294967124 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967004 4294967124 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967003 4294967124 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -5362,7 +5368,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294967008 +test pg_views 4294967007 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index f9741e75d77a..d3eada2e2a4c 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -569,6 +569,7 @@ cluster_settings NULL cluster_statement_statistics NULL cluster_transaction_statistics NULL cluster_transactions NULL +create_function_statements NULL create_schema_statements NULL create_statements NULL create_type_statements NULL diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index ee08dd9a7a45..c0e9bec28296 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -389,6 +389,9 @@ CREATE FUNCTION proc_f(INT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; statement CREATE FUNCTION proc_f(STRING, b INT) RETURNS SETOF STRING STRICT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$ SELECT 'hello' $$; +statement ok +CREATE FUNCTION proc_implicit() RETURNS t_implicit_type IMMUTABLE LANGUAGE SQL AS $$ SELECT a, b from t_implicit_type $$ + statement ok CREATE SCHEMA sc; @@ -401,4 +404,95 @@ FROM pg_catalog.pg_proc WHERE proname IN ('proc_f', 'proc_f_2'); ---- 100115 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; 100116 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; -100118 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; +100119 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; + +subtest create_function_statements + +query TITITIT +SELECT create_statement, database_id, database_name, schema_id, schema_name, function_id, function_name +FROM crdb_internal.create_function_statements +WHERE function_name IN ('proc_f', 'proc_f_2') +ORDER BY function_name; +---- +CREATE FUNCTION public.proc_f(IN INT8) + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; +$$ 104 test 105 public 115 proc_f +CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) + RETURNS SETOF STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ 104 test 105 public 116 proc_f +CREATE FUNCTION sc.proc_f_2(IN STRING) + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ 104 test 118 sc 119 proc_f_2 + +subtest show_create_function + +query TT +SHOW CREATE FUNCTION proc_f; +---- +proc_f CREATE FUNCTION public.proc_f(IN INT8) +RETURNS INT8 +VOLATILE +NOT LEAKPROOF +CALLED ON NULL INPUT +AS $$ +SELECT 1; +$$ +proc_f CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) +RETURNS SETOF STRING +IMMUTABLE +LEAKPROOF +STRICT +AS $$ +SELECT 'hello'; +$$ + +statement error pq: unknown function: proc_f_2() +SHOW CREATE FUNCTION proc_f_2; + +query TT +SHOW CREATE FUNCTION sc.proc_f_2; +---- +proc_f_2 CREATE FUNCTION sc.proc_f_2(IN STRING) +RETURNS STRING +VOLATILE +NOT LEAKPROOF +CALLED ON NULL INPUT +AS $$ +SELECT 'hello'; +$$ + +statement ok +SET search_path = sc; + +query TT +SHOW CREATE FUNCTION proc_f_2; +---- +proc_f_2 CREATE FUNCTION sc.proc_f_2(IN STRING) +RETURNS STRING +VOLATILE +NOT LEAKPROOF +CALLED ON NULL INPUT +AS $$ +SELECT 'hello'; +$$ + +statement ok +SET search_path = public; diff --git a/pkg/sql/opt/optbuilder/testdata/create_function b/pkg/sql/opt/optbuilder/testdata/create_function index 1a4d11aa4d10..ee37ac75476b 100644 --- a/pkg/sql/opt/optbuilder/testdata/create_function +++ b/pkg/sql/opt/optbuilder/testdata/create_function @@ -15,21 +15,30 @@ build CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$ ---- create-function - ├── CREATE FUNCTION f() RETURNS INT8 LANGUAGE SQL AS $$SELECT 1;$$ + ├── CREATE FUNCTION f() + │ RETURNS INT8 + │ LANGUAGE SQL + │ AS $$SELECT 1;$$ └── no dependencies build CREATE FUNCTION f(a workday) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$ ---- create-function - ├── CREATE FUNCTION f(IN a workday) RETURNS INT8 LANGUAGE SQL AS $$SELECT 1;$$ + ├── CREATE FUNCTION f(IN a workday) + │ RETURNS INT8 + │ LANGUAGE SQL + │ AS $$SELECT 1;$$ └── no dependencies build CREATE FUNCTION f(a INT) RETURNS INT LANGUAGE SQL AS $$ SELECT a FROM ab $$ ---- create-function - ├── CREATE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL AS $$SELECT a FROM t.public.ab;$$ + ├── CREATE FUNCTION f(IN a INT8) + │ RETURNS INT8 + │ LANGUAGE SQL + │ AS $$SELECT a FROM t.public.ab;$$ └── dependencies └── ab [columns: a] @@ -37,7 +46,10 @@ build CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ SELECT b FROM ab@idx $$ ---- create-function - ├── CREATE FUNCTION f() RETURNS INT8 LANGUAGE SQL AS $$SELECT b FROM t.public.ab@idx;$$ + ├── CREATE FUNCTION f() + │ RETURNS INT8 + │ LANGUAGE SQL + │ AS $$SELECT b FROM t.public.ab@idx;$$ └── dependencies └── ab@idx [columns: b] @@ -48,7 +60,10 @@ CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS $$ $$ ---- create-function - ├── CREATE FUNCTION f() RETURNS INT8 LANGUAGE SQL AS $$SELECT a FROM t.public.ab; + ├── CREATE FUNCTION f() + │ RETURNS INT8 + │ LANGUAGE SQL + │ AS $$SELECT a FROM t.public.ab; │ SELECT nextval('s');$$ └── dependencies ├── ab [columns: a] diff --git a/pkg/sql/parser/testdata/create_function b/pkg/sql/parser/testdata/create_function index f6b2b6cc619d..c543b1820a4d 100644 --- a/pkg/sql/parser/testdata/create_function +++ b/pkg/sql/parser/testdata/create_function @@ -1,58 +1,178 @@ parse CREATE OR REPLACE FUNCTION f(a int = 7) RETURNS INT AS 'SELECT 1' LANGUAGE SQL ---- -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) RETURNS INT8 LANGUAGE SQL AS $$SELECT 1$$ -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) RETURNS INT8 LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) RETURNS INT8 LANGUAGE SQL AS $$SELECT 1$$ -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) RETURNS INT8 LANGUAGE SQL AS $$SELECT 1$$ -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) + RETURNS INT8 + LANGUAGE SQL + AS $$SELECT 1$$ -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) + RETURNS INT8 + LANGUAGE SQL + AS $$SELECT 1$$ -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) + RETURNS INT8 + LANGUAGE SQL + AS $$SELECT 1$$ -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) + RETURNS INT8 + LANGUAGE SQL + AS $$SELECT 1$$ -- identifiers removed parse CREATE OR REPLACE FUNCTION f(IN a INT=7) RETURNS INT CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS 'SELECT 1' ---- -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- identifiers removed parse CREATE OR REPLACE FUNCTION f(IN a INT=7) RETURNS INT AS 'SELECT 1' CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL ---- -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) RETURNS INT8 CALLED ON NULL INPUT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$SELECT 1$$ -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 7) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (7)) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 7) + RETURNS INT8 + CALLED ON NULL INPUT + IMMUTABLE + LEAKPROOF + LANGUAGE SQL + AS $$SELECT 1$$ -- identifiers removed parse CREATE OR REPLACE FUNCTION f(a INT DEFAULT 10) RETURNS INT RETURNS NULL ON NULL INPUT LANGUAGE SQL AS 'SELECT 1' ---- -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 10) RETURNS INT8 RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (10)) RETURNS INT8 RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) RETURNS INT8 RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 10) RETURNS INT8 RETURNS NULL ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT 10) + RETURNS INT8 + RETURNS NULL ON NULL INPUT + LANGUAGE SQL + AS $$SELECT 1$$ -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT (10)) + RETURNS INT8 + RETURNS NULL ON NULL INPUT + LANGUAGE SQL + AS $$SELECT 1$$ -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8 DEFAULT _) + RETURNS INT8 + RETURNS NULL ON NULL INPUT + LANGUAGE SQL + AS $$SELECT 1$$ -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8 DEFAULT 10) + RETURNS INT8 + RETURNS NULL ON NULL INPUT + LANGUAGE SQL + AS $$SELECT 1$$ -- identifiers removed parse CREATE OR REPLACE FUNCTION f(a INT) RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT a; END ---- -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT a; END -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT (1); SELECT (a); END -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT _; SELECT a; END -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT _; END -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; SELECT a; END -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT (1); SELECT (a); END -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT _; SELECT a; END -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; SELECT _; END -- identifiers removed parse CREATE OR REPLACE FUNCTION f(a INT) RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT $1; END ---- -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT $1; END -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT (1); SELECT ($1); END -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT _; SELECT $1; END -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; SELECT $1; END -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; SELECT $1; END -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT (1); SELECT ($1); END -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT _; SELECT $1; END -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; SELECT $1; END -- identifiers removed parse CREATE OR REPLACE FUNCTION f(a INT) RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1; CREATE OR REPLACE FUNCTION g() RETURNS INT BEGIN ATOMIC SELECT 2; END; END ---- -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; CREATE OR REPLACE FUNCTION g() RETURNS INT8 BEGIN ATOMIC SELECT 2; END; END -- normalized! -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT (1); CREATE OR REPLACE FUNCTION g() RETURNS INT8 BEGIN ATOMIC SELECT (2); END; END -- fully parenthesized -CREATE OR REPLACE FUNCTION f(IN a INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT _; CREATE OR REPLACE FUNCTION g() RETURNS INT8 BEGIN ATOMIC SELECT _; END; END -- literals removed -CREATE OR REPLACE FUNCTION _(IN _ INT8) RETURNS INT8 LANGUAGE SQL BEGIN ATOMIC SELECT 1; CREATE OR REPLACE FUNCTION _() RETURNS INT8 BEGIN ATOMIC SELECT 2; END; END -- identifiers removed +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; CREATE OR REPLACE FUNCTION g() + RETURNS INT8 + BEGIN ATOMIC SELECT 2; END; END -- normalized! +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT (1); CREATE OR REPLACE FUNCTION g() + RETURNS INT8 + BEGIN ATOMIC SELECT (2); END; END -- fully parenthesized +CREATE OR REPLACE FUNCTION f(IN a INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT _; CREATE OR REPLACE FUNCTION g() + RETURNS INT8 + BEGIN ATOMIC SELECT _; END; END -- literals removed +CREATE OR REPLACE FUNCTION _(IN _ INT8) + RETURNS INT8 + LANGUAGE SQL + BEGIN ATOMIC SELECT 1; CREATE OR REPLACE FUNCTION _() + RETURNS INT8 + BEGIN ATOMIC SELECT 2; END; END -- identifiers removed error CREATE OR REPLACE FUNCTION f(a INT) RETURNS INT LANGUAGE SQL BEGIN ATOMIC SELECT 1 END diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index 92c5005b18a0..1a85e45691c1 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -104,6 +104,7 @@ const ( CrdbInternalClusterSettingsTableID CrdbInternalClusterStmtStatsTableID CrdbInternalClusterTxnStatsTableID + CrdbInternalCreateFunctionStmtsTableID CrdbInternalCreateSchemaStmtsTableID CrdbInternalCreateStmtsTableID CrdbInternalCreateTypeStmtsTableID diff --git a/pkg/sql/sem/tree/udf.go b/pkg/sql/sem/tree/udf.go index 94b388a12ab0..a2a6d64b5107 100644 --- a/pkg/sql/sem/tree/udf.go +++ b/pkg/sql/sem/tree/udf.go @@ -66,13 +66,13 @@ func (node *CreateFunction) Format(ctx *FmtCtx) { ctx.FormatNode(&node.FuncName) ctx.WriteString("(") ctx.FormatNode(node.Args) - ctx.WriteString(") ") + ctx.WriteString(")\n\t") ctx.WriteString("RETURNS ") if node.ReturnType.IsSet { ctx.WriteString("SETOF ") } ctx.WriteString(node.ReturnType.Type.SQLString()) - ctx.WriteString(" ") + ctx.WriteString("\n\t") var funcBody FunctionBodyStr for _, option := range node.Options { switch t := option.(type) { @@ -81,7 +81,7 @@ func (node *CreateFunction) Format(ctx *FmtCtx) { continue } ctx.FormatNode(option) - ctx.WriteString(" ") + ctx.WriteString("\n\t") } if len(funcBody) > 0 { ctx.FormatNode(funcBody) From 521ff3c5d11026851ae1a02ff4deae94f0078c8d Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Fri, 5 Aug 2022 01:24:22 -0400 Subject: [PATCH 3/4] sql: add SHOW CREATE FUNCTION support Release note (sql change): This commit adds support for the SHOW CREATE FUNCTION statement. If given function name is qualified, the explicit schema will be searched. If function name is not qualified, the schemas on search path are searched and functions from the most significant schema are returned. --- pkg/sql/crdb_internal.go | 1 - pkg/sql/delegate/show_function.go | 36 +++++++++- .../logictest/testdata/logic_test/show_create | 5 -- pkg/sql/logictest/testdata/logic_test/udf | 72 ++++++++++--------- pkg/sql/opt_catalog.go | 4 +- 5 files changed, 74 insertions(+), 44 deletions(-) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index babac7687bd5..0bf2e198dcd7 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2629,7 +2629,6 @@ CREATE TABLE crdb_internal.create_function_statements ( ) }) }) - return nil }, } diff --git a/pkg/sql/delegate/show_function.go b/pkg/sql/delegate/show_function.go index 7fc125eba8a3..19f1f8bd26f4 100644 --- a/pkg/sql/delegate/show_function.go +++ b/pkg/sql/delegate/show_function.go @@ -11,10 +11,42 @@ package delegate import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/errors" ) func (d *delegator) delegateShowCreateFunction(n *tree.ShowCreateFunction) (tree.Statement, error) { - return nil, unimplemented.New("SHOW CREATE FUNCTION", "this statement is not yet supported") + // We don't need to filter by db since we don't allow cross-database + // references. + query := ` +SELECT function_name, create_statement +FROM crdb_internal.create_function_statements +WHERE schema_name = %[1]s +AND function_name = %[2]s +` + un, ok := n.Name.FunctionReference.(*tree.UnresolvedName) + if !ok { + return nil, errors.AssertionFailedf("not a valid function name") + } + + fn, err := d.catalog.ResolveFunction(d.ctx, un, &d.evalCtx.SessionData().SearchPath) + if err != nil { + return nil, err + } + + var udfSchema string + for _, o := range fn.Overloads { + if o.IsUDF { + udfSchema = o.Schema + } + } + if udfSchema == "" { + return nil, errors.Errorf("function %s does not exist", tree.AsString(un)) + } + + fullQuery := fmt.Sprintf(query, lexbase.EscapeSQLString(udfSchema), lexbase.EscapeSQLString(un.Parts[0])) + return parse(fullQuery) } diff --git a/pkg/sql/logictest/testdata/logic_test/show_create b/pkg/sql/logictest/testdata/logic_test/show_create index 4c2c1bc44b52..0e4d21073240 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_create +++ b/pkg/sql/logictest/testdata/logic_test/show_create @@ -140,8 +140,3 @@ CREATE TABLE public.t ( CONSTRAINT t_pkey PRIMARY KEY (rowid ASC) ); COMMENT ON COLUMN public.t.c IS 'first comment' - -subtest show_create_function - -statement error this statement is not yet supported -SHOW CREATE FUNCTION lower diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index c0e9bec28296..c94aeb2835e1 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -444,54 +444,58 @@ $$ 104 test 118 sc 119 proc_f_2 subtest show_create_function -query TT -SHOW CREATE FUNCTION proc_f; +query T +SELECT @2 FROM [SHOW CREATE FUNCTION proc_f]; ---- -proc_f CREATE FUNCTION public.proc_f(IN INT8) -RETURNS INT8 -VOLATILE -NOT LEAKPROOF -CALLED ON NULL INPUT -AS $$ -SELECT 1; +CREATE FUNCTION public.proc_f(IN INT8) + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; $$ -proc_f CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) -RETURNS SETOF STRING -IMMUTABLE -LEAKPROOF -STRICT -AS $$ -SELECT 'hello'; +CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) + RETURNS SETOF STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; $$ statement error pq: unknown function: proc_f_2() SHOW CREATE FUNCTION proc_f_2; -query TT -SHOW CREATE FUNCTION sc.proc_f_2; +query T +SELECT @2 FROM [SHOW CREATE FUNCTION sc.proc_f_2]; ---- -proc_f_2 CREATE FUNCTION sc.proc_f_2(IN STRING) -RETURNS STRING -VOLATILE -NOT LEAKPROOF -CALLED ON NULL INPUT -AS $$ -SELECT 'hello'; +CREATE FUNCTION sc.proc_f_2(IN STRING) + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; $$ statement ok SET search_path = sc; -query TT -SHOW CREATE FUNCTION proc_f_2; +query T +SELECT @2 FROM [SHOW CREATE FUNCTION proc_f_2]; ---- -proc_f_2 CREATE FUNCTION sc.proc_f_2(IN STRING) -RETURNS STRING -VOLATILE -NOT LEAKPROOF -CALLED ON NULL INPUT -AS $$ -SELECT 'hello'; +CREATE FUNCTION sc.proc_f_2(IN STRING) + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; $$ statement ok diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index a7c187476d47..ee2b01ff9396 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -331,13 +331,13 @@ func (oc *optCatalog) ResolveType( func (oc *optCatalog) ResolveFunction( ctx context.Context, name *tree.UnresolvedName, path tree.SearchPath, ) (*tree.ResolvedFunctionDefinition, error) { - return nil, errors.AssertionFailedf("unimplemented") + return oc.planner.ResolveFunction(ctx, name, path) } func (oc *optCatalog) ResolveFunctionByOID( ctx context.Context, oid oid.Oid, ) (*tree.Overload, error) { - return nil, errors.AssertionFailedf("unimplemented") + return oc.planner.ResolveFunctionByOID(ctx, oid) } func getDescFromCatalogObjectForPermissions(o cat.Object) (catalog.Descriptor, error) { From 4e4e5d93c8d88f3e48b3946c9b3883147015a1fc Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Fri, 5 Aug 2022 13:13:17 -0400 Subject: [PATCH 4/4] sql: add regproc support for UDF Release note (sql change): Previously, the `::regproc` casting only supported builtin functions. Now it's extened to support user-defined functions as well. --- .../changefeedccl/cdceval/func_resolver.go | 4 +- pkg/sql/faketreeeval/evalctx.go | 14 +++++ pkg/sql/function_resolver_test.go | 8 +-- pkg/sql/logictest/testdata/logic_test/udf | 60 +++++++++++++++++-- pkg/sql/opt/cat/catalog.go | 2 +- pkg/sql/opt/testutils/testcat/function.go | 6 +- pkg/sql/opt_catalog.go | 2 +- pkg/sql/pg_catalog.go | 1 - pkg/sql/schema_resolver.go | 14 ++--- pkg/sql/sem/eval/cast.go | 9 ++- pkg/sql/sem/eval/deps.go | 1 + pkg/sql/sem/eval/parse_doid.go | 6 +- pkg/sql/sem/tree/function_name.go | 5 +- pkg/sql/sem/tree/type_check.go | 2 +- 14 files changed, 97 insertions(+), 37 deletions(-) diff --git a/pkg/ccl/changefeedccl/cdceval/func_resolver.go b/pkg/ccl/changefeedccl/cdceval/func_resolver.go index af1a09b8684b..f5b0d0ee3fe7 100644 --- a/pkg/ccl/changefeedccl/cdceval/func_resolver.go +++ b/pkg/ccl/changefeedccl/cdceval/func_resolver.go @@ -72,8 +72,8 @@ func (rs *CDCFunctionResolver) WrapFunction(name string) (*tree.ResolvedFunction // ResolveFunctionByOID implements FunctionReferenceResolver interface. func (rs *CDCFunctionResolver) ResolveFunctionByOID( ctx context.Context, oid oid.Oid, -) (*tree.Overload, error) { +) (string, *tree.Overload, error) { // CDC doesn't support user defined function yet, so there's no need to // resolve function by OID. - return nil, errors.AssertionFailedf("unimplemented yet") + return "", nil, errors.AssertionFailedf("unimplemented yet") } diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index c661a7ab0192..c22c4a6ef51f 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -435,6 +435,20 @@ func (ep *DummyEvalPlanner) IsActive(_ context.Context, _ clusterversion.Key) bo return true } +// ResolveFunction implements FunctionReferenceResolver interface. +func (ep *DummyEvalPlanner) ResolveFunction( + ctx context.Context, name *tree.UnresolvedName, path tree.SearchPath, +) (*tree.ResolvedFunctionDefinition, error) { + return nil, errors.AssertionFailedf("ResolveFunction unimplemented") +} + +// ResolveFunctionByOID implements FunctionReferenceResolver interface. +func (ep *DummyEvalPlanner) ResolveFunctionByOID( + ctx context.Context, oid oid.Oid, +) (string, *tree.Overload, error) { + return "", nil, errors.AssertionFailedf("ResolveFunctionByOID unimplemented") +} + // DummyPrivilegedAccessor implements the tree.PrivilegedAccessor interface by returning errors. type DummyPrivilegedAccessor struct{} diff --git a/pkg/sql/function_resolver_test.go b/pkg/sql/function_resolver_test.go index 79f07c9a44bc..4a7d1ffd206a 100644 --- a/pkg/sql/function_resolver_test.go +++ b/pkg/sql/function_resolver_test.go @@ -117,7 +117,7 @@ CREATE FUNCTION f() RETURNS t IMMUTABLE LANGUAGE SQL AS $$ SELECT a, b, c FROM t require.Equal(t, types.TupleFamily, funcDef.Overloads[2].ReturnType([]tree.TypedExpr{}).Family()) require.NotZero(t, funcDef.Overloads[2].ReturnType([]tree.TypedExpr{}).TypeMeta) - overload, err := funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[0].Oid) + _, overload, err := funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[0].Oid) require.NoError(t, err) require.Equal(t, `SELECT a FROM defaultdb.public.t; SELECT b FROM defaultdb.public.t@t_idx_b; @@ -131,7 +131,7 @@ SELECT nextval(105:::REGCLASS);`, overload.Body) require.Equal(t, types.EnumFamily, overload.Types.Types()[0].Family()) require.Equal(t, types.Int, overload.ReturnType([]tree.TypedExpr{})) - overload, err = funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[1].Oid) + _, overload, err = funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[1].Oid) require.NoError(t, err) require.Equal(t, `SELECT 1;`, overload.Body) require.True(t, overload.IsUDF) @@ -139,7 +139,7 @@ SELECT nextval(105:::REGCLASS);`, overload.Body) require.Equal(t, 0, len(overload.Types.Types())) require.Equal(t, types.Void, overload.ReturnType([]tree.TypedExpr{})) - overload, err = funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[2].Oid) + _, overload, err = funcResolver.ResolveFunctionByOID(ctx, funcDef.Overloads[2].Oid) require.NoError(t, err) require.Equal(t, `SELECT a, b, c FROM defaultdb.public.t;`, overload.Body) require.True(t, overload.IsUDF) @@ -257,7 +257,7 @@ CREATE FUNCTION sc1.lower() RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT 3 $$ bodies := make([]string, len(funcDef.Overloads)) schemas := make([]string, len(funcDef.Overloads)) for i, o := range funcDef.Overloads { - overload, err := funcResolver.ResolveFunctionByOID(ctx, o.Oid) + _, overload, err := funcResolver.ResolveFunctionByOID(ctx, o.Oid) require.NoError(t, err) bodies[i] = overload.Body schemas[i] = o.Schema diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index c94aeb2835e1..ab2912694dd3 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -402,9 +402,9 @@ query TTTTTBBBTITTTTT SELECT oid, proname, pronamespace, proowner, prolang, proleakproof, proisstrict, proretset, provolatile, pronargs, prorettype, proargtypes, proargmodes, proargnames, prosrc FROM pg_catalog.pg_proc WHERE proname IN ('proc_f', 'proc_f_2'); ---- -100115 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; -100116 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; -100119 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; +100124 proc_f 4101115737 1546506610 14 false false false v 1 20 20 {i} NULL SELECT 1; +100125 proc_f 4101115737 1546506610 14 true true true i 2 25 25 20 {i,i} {"",b} SELECT 'hello'; +100128 proc_f_2 131273696 1546506610 14 false false false v 1 25 25 {i} NULL SELECT 'hello'; subtest create_function_statements @@ -422,7 +422,7 @@ CREATE FUNCTION public.proc_f(IN INT8) LANGUAGE SQL AS $$ SELECT 1; -$$ 104 test 105 public 115 proc_f +$$ 104 test 105 public 124 proc_f CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) RETURNS SETOF STRING IMMUTABLE @@ -431,7 +431,7 @@ CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 105 public 116 proc_f +$$ 104 test 105 public 125 proc_f CREATE FUNCTION sc.proc_f_2(IN STRING) RETURNS STRING VOLATILE @@ -440,7 +440,7 @@ CREATE FUNCTION sc.proc_f_2(IN STRING) LANGUAGE SQL AS $$ SELECT 'hello'; -$$ 104 test 118 sc 119 proc_f_2 +$$ 104 test 127 sc 128 proc_f_2 subtest show_create_function @@ -500,3 +500,51 @@ $$ statement ok SET search_path = public; + +subtest udf_regproc + +query T +SELECT 'proc_implicit'::REGPROC; +---- +proc_implicit + +query I +SELECT 'proc_implicit'::REGPROC::INT; +---- +100126 + +query T +SELECT '100126'::REGPROC; +---- +proc_implicit + +query T +SELECT 'sc.proc_f_2'::REGPROC; +---- +proc_f_2 + +query I +SELECT 'sc.proc_f_2'::REGPROC::INT; +---- +100128 + +statement error pq: unknown function: no_such_func() +SELECT 'no_such_func'::REGPROC; + +statement error pq: more than one function named 'proc_f' +SELECT 'proc_f'::REGPROC; + +query T +SELECT 100126::regproc; +---- +proc_implicit + +query I +SELECT 100117::regproc::INT; +---- +100117 + +query T +SELECT 999999::regproc; +---- +999999 diff --git a/pkg/sql/opt/cat/catalog.go b/pkg/sql/opt/cat/catalog.go index 4e571a9b2cf4..268563716a2a 100644 --- a/pkg/sql/opt/cat/catalog.go +++ b/pkg/sql/opt/cat/catalog.go @@ -143,7 +143,7 @@ type Catalog interface { ) (*tree.ResolvedFunctionDefinition, error) // ResolveFunctionByOID resolves a function overload by OID. - ResolveFunctionByOID(ctx context.Context, oid oid.Oid) (*tree.Overload, error) + ResolveFunctionByOID(ctx context.Context, oid oid.Oid) (string, *tree.Overload, error) // CheckPrivilege verifies that the current user has the given privilege on // the given catalog object. If not, then CheckPrivilege returns an error. diff --git a/pkg/sql/opt/testutils/testcat/function.go b/pkg/sql/opt/testutils/testcat/function.go index 6eda8baeeabd..80186d6eeb8d 100644 --- a/pkg/sql/opt/testutils/testcat/function.go +++ b/pkg/sql/opt/testutils/testcat/function.go @@ -51,8 +51,10 @@ func (tc *Catalog) ResolveFunction( } // ResolveFunctionByOID part of the tree.FunctionReferenceResolver interface. -func (tc *Catalog) ResolveFunctionByOID(ctx context.Context, oid oid.Oid) (*tree.Overload, error) { - return nil, errors.AssertionFailedf("ResolveFunctionByOID not supported in test catalog") +func (tc *Catalog) ResolveFunctionByOID( + ctx context.Context, oid oid.Oid, +) (string, *tree.Overload, error) { + return "", nil, errors.AssertionFailedf("ResolveFunctionByOID not supported in test catalog") } // CreateFunction handles the CREATE FUNCTION statement. diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index ee2b01ff9396..c274ee4b45e8 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -336,7 +336,7 @@ func (oc *optCatalog) ResolveFunction( func (oc *optCatalog) ResolveFunctionByOID( ctx context.Context, oid oid.Oid, -) (*tree.Overload, error) { +) (string, *tree.Overload, error) { return oc.planner.ResolveFunctionByOID(ctx, oid) } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 09d0568cddaa..b45f4be0cbe2 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -2457,7 +2457,6 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, } return addRow( - // TODO (chengxiong) fix this when we have the ID to OID thing tree.NewDOid(catid.FuncIDToOID(fnDesc.GetID())), // oid tree.NewDName(fnDesc.GetName()), // proname h.NamespaceOid(dbDesc.GetID(), scDesc.GetName()), // pronamespace diff --git a/pkg/sql/schema_resolver.go b/pkg/sql/schema_resolver.go index 8df779db4766..e5291746964f 100644 --- a/pkg/sql/schema_resolver.go +++ b/pkg/sql/schema_resolver.go @@ -441,16 +441,16 @@ func (sr *schemaResolver) ResolveFunction( func (sr *schemaResolver) ResolveFunctionByOID( ctx context.Context, oid oid.Oid, -) (*tree.Overload, error) { +) (name string, fn *tree.Overload, err error) { if !funcdesc.IsOIDUserDefinedFunc(oid) { name, ok := tree.OidToBuiltinName[oid] if !ok { - return nil, pgerror.Newf(pgcode.UndefinedFunction, "function %d not found", oid) + return "", nil, pgerror.Newf(pgcode.UndefinedFunction, "function %d not found", oid) } funcDef := tree.FunDefs[name] for _, o := range funcDef.Definition { if o.Oid == oid { - return o, nil + return funcDef.Name, o, nil } } } @@ -459,17 +459,17 @@ func (sr *schemaResolver) ResolveFunctionByOID( flags.AvoidLeased = sr.skipDescriptorCache descID, err := funcdesc.UserDefinedFunctionOIDToID(oid) if err != nil { - return nil, err + return "", nil, err } funcDesc, err := sr.descCollection.GetImmutableFunctionByID(ctx, sr.txn, descID, flags) if err != nil { - return nil, err + return "", nil, err } ret, err := funcDesc.ToOverload() if err != nil { - return nil, err + return "", nil, err } - return ret, nil + return funcDesc.GetName(), ret, nil } // NewSkippingCacheSchemaResolver constructs a schemaResolver which always skip diff --git a/pkg/sql/sem/eval/cast.go b/pkg/sql/sem/eval/cast.go index 094e15780cc3..38ba96256787 100644 --- a/pkg/sql/sem/eval/cast.go +++ b/pkg/sql/sem/eval/cast.go @@ -910,7 +910,7 @@ func performCastWithoutPrecisionTruncation( // performIntToOidCast casts the input integer to the OID type given by the // input types.T. func performIntToOidCast( - ctx context.Context, res TypeResolver, t *types.T, v tree.DInt, + ctx context.Context, res Planner, t *types.T, v tree.DInt, ) (tree.Datum, error) { // OIDs are always unsigned 32-bit integers. Some languages, like Java, // store OIDs as signed 32-bit integers, so we implement the cast @@ -940,13 +940,12 @@ func performIntToOidCast( return tree.NewDOidWithTypeAndName(o, t, name), nil case oid.T_regproc, oid.T_regprocedure: - // Mapping an dOid to a regproc is easy: we have a hardcoded map. - name, ok := tree.OidToBuiltinName[o] - if !ok { + name, _, err := res.ResolveFunctionByOID(ctx, oid.Oid(v)) + if err != nil { if v == 0 { return tree.WrapAsZeroOid(t), nil } - return tree.NewDOidWithType(o, t), nil + return tree.NewDOidWithType(o, t), nil //nolint:returnerrcheck } return tree.NewDOidWithTypeAndName(o, t, name), nil diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 31d72e8daf2e..1f39f6123527 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -190,6 +190,7 @@ type TypeResolver interface { type Planner interface { DatabaseCatalog TypeResolver + tree.FunctionReferenceResolver // ExecutorConfig returns *ExecutorConfig ExecutorConfig() interface{} diff --git a/pkg/sql/sem/eval/parse_doid.go b/pkg/sql/sem/eval/parse_doid.go index 5d87fcf13cf1..87e2d7a020e7 100644 --- a/pkg/sql/sem/eval/parse_doid.go +++ b/pkg/sql/sem/eval/parse_doid.go @@ -70,11 +70,7 @@ func ParseDOid(ctx *Context, s string, t *types.T) (*tree.DOid, error) { for i := 0; i < len(substrs); i++ { name.Parts[i] = substrs[len(substrs)-1-i] } - fn, err := name.ToFunctionName() - if err != nil { - return nil, err - } - funcDef, err := tree.GetBuiltinFuncDefinitionOrFail(fn, &ctx.SessionData().SearchPath) + funcDef, err := ctx.Planner.ResolveFunction(ctx.Ctx(), &name, &ctx.SessionData().SearchPath) if err != nil { return nil, err } diff --git a/pkg/sql/sem/tree/function_name.go b/pkg/sql/sem/tree/function_name.go index fb4f3c33b386..744ab62b3a6f 100644 --- a/pkg/sql/sem/tree/function_name.go +++ b/pkg/sql/sem/tree/function_name.go @@ -44,10 +44,11 @@ type FunctionReferenceResolver interface { ) (*ResolvedFunctionDefinition, error) // ResolveFunctionByOID looks up a function overload by using a given oid. - // Error is thrown if there is no function with the same oid. + // Function name is returned together with the overload. Error is thrown if + // there is no function with the same oid. ResolveFunctionByOID( ctx context.Context, oid oid.Oid, - ) (*Overload, error) + ) (string, *Overload, error) } // ResolvableFunctionReference implements the editable reference call of a diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 1befd8be190c..1f410004e958 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -1160,7 +1160,7 @@ func (expr *FuncExpr) TypeCheck( "%s()", errors.Safe(def.Name)) } if resolver != nil && overloadImpl.UDFContainsOnlySignature { - overloadImpl, err = resolver.ResolveFunctionByOID(ctx, overloadImpl.Oid) + _, overloadImpl, err = resolver.ResolveFunctionByOID(ctx, overloadImpl.Oid) if err != nil { return nil, err }