From 513b045ab5e5c62d86534a09fdce3a0b1594d277 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 2 Oct 2023 16:11:13 -0400 Subject: [PATCH 01/12] storage: embed roachpb.InternalTimeSeriesData on MVCCValueMerger Hang a roachpb.InternalTimeSeriesData off the MVCCValueMerger to avoid unnecessary allocations in Finish. Epic: none Informs #106567. Release note: none --- pkg/storage/pebble_merge.go | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/storage/pebble_merge.go b/pkg/storage/pebble_merge.go index 999d6b9cd184..7837bd314c76 100644 --- a/pkg/storage/pebble_merge.go +++ b/pkg/storage/pebble_merge.go @@ -176,6 +176,8 @@ type MVCCValueMerger struct { // Used to avoid heap allocations when passing pointer to `Unmarshal()`. meta enginepb.MVCCMetadata + // Used to avoid heap allocations in Finish(). + merged roachpb.InternalTimeSeriesData } const ( @@ -287,9 +289,11 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { // compatible with any version that supports row format only. Then we can drop support // for row format entirely. It requires significant cleanup effort as many tests target // the row format. - var merged roachpb.InternalTimeSeriesData - merged.StartTimestampNanos = t.timeSeriesOps[0].StartTimestampNanos - merged.SampleDurationNanos = t.timeSeriesOps[0].SampleDurationNanos + merged := &t.merged // avoid allocation + *merged = roachpb.InternalTimeSeriesData{ + StartTimestampNanos: t.timeSeriesOps[0].StartTimestampNanos, + SampleDurationNanos: t.timeSeriesOps[0].SampleDurationNanos, + } for _, timeSeriesOp := range t.timeSeriesOps { if timeSeriesOp.StartTimestampNanos != merged.StartTimestampNanos { return nil, nil, errors.Errorf("start timestamp mismatch") @@ -298,20 +302,20 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { return nil, nil, errors.Errorf("sample duration mismatch") } if !isColumnar && len(timeSeriesOp.Offset) > 0 { - ensureColumnar(&merged) + ensureColumnar(merged) ensureColumnar(&timeSeriesOp) isColumnar = true } else if isColumnar { ensureColumnar(&timeSeriesOp) } - proto.Merge(&merged, &timeSeriesOp) + proto.Merge(merged, &timeSeriesOp) } if isColumnar { - sortAndDeduplicateColumns(&merged) + sortAndDeduplicateColumns(merged) } else { - sortAndDeduplicateRows(&merged) + sortAndDeduplicateRows(merged) } - tsBytes, err := protoutil.Marshal(&merged) + tsBytes, err := protoutil.Marshal(merged) if err != nil { return nil, nil, err } From c52255aaff3897abbed3485bb139c4f6f7a6381b Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 2 Oct 2023 16:16:21 -0400 Subject: [PATCH 02/12] storage: don't let MVCCValueMerger's loop var escape Epic: none Informs #106567. Release note: none --- pkg/storage/pebble_merge.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pkg/storage/pebble_merge.go b/pkg/storage/pebble_merge.go index 7837bd314c76..27fd8c7859f3 100644 --- a/pkg/storage/pebble_merge.go +++ b/pkg/storage/pebble_merge.go @@ -294,7 +294,8 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { StartTimestampNanos: t.timeSeriesOps[0].StartTimestampNanos, SampleDurationNanos: t.timeSeriesOps[0].SampleDurationNanos, } - for _, timeSeriesOp := range t.timeSeriesOps { + for i := range t.timeSeriesOps { + timeSeriesOp := &t.timeSeriesOps[i] if timeSeriesOp.StartTimestampNanos != merged.StartTimestampNanos { return nil, nil, errors.Errorf("start timestamp mismatch") } @@ -303,12 +304,12 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { } if !isColumnar && len(timeSeriesOp.Offset) > 0 { ensureColumnar(merged) - ensureColumnar(&timeSeriesOp) + ensureColumnar(timeSeriesOp) isColumnar = true } else if isColumnar { - ensureColumnar(&timeSeriesOp) + ensureColumnar(timeSeriesOp) } - proto.Merge(merged, &timeSeriesOp) + proto.Merge(merged, timeSeriesOp) } if isColumnar { sortAndDeduplicateColumns(merged) From 40e2955fb3b8779f84bb78f6a546895f0f42bcac Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 2 Oct 2023 17:10:40 -0400 Subject: [PATCH 03/12] storage: avoid MVCCMetadataSubsetForMergeSerialization alloc Hang MVCCMetadataSubsetForMergeSerialization off the MVCCValueMerger type to avoid suffering its allocation during MVCCValueMerger.Finish. Epic: none Informs #106567. Release note: none --- pkg/storage/pebble_merge.go | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/pkg/storage/pebble_merge.go b/pkg/storage/pebble_merge.go index 27fd8c7859f3..3d02ef5d5c5c 100644 --- a/pkg/storage/pebble_merge.go +++ b/pkg/storage/pebble_merge.go @@ -176,8 +176,9 @@ type MVCCValueMerger struct { // Used to avoid heap allocations when passing pointer to `Unmarshal()`. meta enginepb.MVCCMetadata - // Used to avoid heap allocations in Finish(). - merged roachpb.InternalTimeSeriesData + // merged and metaSubset are used to avoid heap allocations in Finish(). + merged roachpb.InternalTimeSeriesData + metaSubset enginepb.MVCCMetadataSubsetForMergeSerialization } const ( @@ -271,13 +272,15 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { totalLen += len(rawByteOp) } // See the motivating comment in mvcc.proto. - var meta enginepb.MVCCMetadataSubsetForMergeSerialization - meta.RawBytes = make([]byte, mvccHeaderSize, mvccHeaderSize+totalLen) + meta := &t.metaSubset // avoid allocation + *meta = enginepb.MVCCMetadataSubsetForMergeSerialization{ + RawBytes: make([]byte, mvccHeaderSize, mvccHeaderSize+totalLen), + } meta.RawBytes[mvccTagPos] = byte(roachpb.ValueType_BYTES) for _, rawByteOp := range t.rawByteOps { meta.RawBytes = append(meta.RawBytes, rawByteOp...) } - res, err := protoutil.Marshal(&meta) + res, err := protoutil.Marshal(meta) if err != nil { return nil, nil, err } @@ -321,7 +324,7 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { return nil, nil, err } // See the motivating comment in mvcc.proto. - var meta enginepb.MVCCMetadataSubsetForMergeSerialization + meta := &t.metaSubset // avoid allocation if !(t.oldestMergeTS == hlc.LegacyTimestamp{}) { meta.MergeTimestamp = &t.oldestMergeTS } @@ -329,7 +332,7 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { header := make([]byte, mvccHeaderSize) header[mvccTagPos] = tsTag meta.RawBytes = append(header, tsBytes...) - res, err := protoutil.Marshal(&meta) + res, err := protoutil.Marshal(meta) if err != nil { return nil, nil, err } From 057e992ac8e3179b3824bb437646eb86ab64db2f Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Mon, 2 Oct 2023 17:30:08 -0400 Subject: [PATCH 04/12] storage: avoid intermediary allocs in MVCCValueMerger.Finish Marshal protocol buffer directly into an appropriately-sized RawBytes slice in MVCCValueMerger. Epic: none Release note: none Informs #106567 --- pkg/storage/pebble_merge.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/pkg/storage/pebble_merge.go b/pkg/storage/pebble_merge.go index 3d02ef5d5c5c..0f4b47527568 100644 --- a/pkg/storage/pebble_merge.go +++ b/pkg/storage/pebble_merge.go @@ -319,19 +319,20 @@ func (t *MVCCValueMerger) Finish(includesBase bool) ([]byte, io.Closer, error) { } else { sortAndDeduplicateRows(merged) } - tsBytes, err := protoutil.Marshal(merged) - if err != nil { - return nil, nil, err + + meta := &t.metaSubset // avoid allocation + *meta = enginepb.MVCCMetadataSubsetForMergeSerialization{ + RawBytes: make([]byte, mvccHeaderSize+merged.Size()), } + meta.RawBytes[mvccTagPos] = byte(roachpb.ValueType_TIMESERIES) // See the motivating comment in mvcc.proto. - meta := &t.metaSubset // avoid allocation if !(t.oldestMergeTS == hlc.LegacyTimestamp{}) { meta.MergeTimestamp = &t.oldestMergeTS } - tsTag := byte(roachpb.ValueType_TIMESERIES) - header := make([]byte, mvccHeaderSize) - header[mvccTagPos] = tsTag - meta.RawBytes = append(header, tsBytes...) + _, err := protoutil.MarshalToSizedBuffer(merged, meta.RawBytes[mvccHeaderSize:]) + if err != nil { + return nil, nil, err + } res, err := protoutil.Marshal(meta) if err != nil { return nil, nil, err From e5a692539fdbe974eda2d15d2254afad0362ad1b Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 24 Sep 2023 10:22:21 -0400 Subject: [PATCH 05/12] sql/tree: rename ShowCreateFunction to ShowCreateRoutine Release note: None --- pkg/sql/delegate/delegate.go | 2 +- pkg/sql/delegate/show_function.go | 2 +- pkg/sql/opt/testutils/testcat/test_catalog.go | 2 +- pkg/sql/parser/sql.y | 2 +- pkg/sql/sem/tree/show.go | 9 +++++---- pkg/sql/sem/tree/stmt.go | 8 ++++---- 6 files changed, 13 insertions(+), 12 deletions(-) diff --git a/pkg/sql/delegate/delegate.go b/pkg/sql/delegate/delegate.go index e7926d197200..1c988c89fcb3 100644 --- a/pkg/sql/delegate/delegate.go +++ b/pkg/sql/delegate/delegate.go @@ -64,7 +64,7 @@ func TryDelegate( case *tree.ShowCreate: return d.delegateShowCreate(t) - case *tree.ShowCreateFunction: + case *tree.ShowCreateRoutine: return d.delegateShowCreateFunction(t) case *tree.ShowCreateAllSchemas: diff --git a/pkg/sql/delegate/show_function.go b/pkg/sql/delegate/show_function.go index 64e45b3f6cf6..e061abe3b134 100644 --- a/pkg/sql/delegate/show_function.go +++ b/pkg/sql/delegate/show_function.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/errors" ) -func (d *delegator) delegateShowCreateFunction(n *tree.ShowCreateFunction) (tree.Statement, error) { +func (d *delegator) delegateShowCreateFunction(n *tree.ShowCreateRoutine) (tree.Statement, error) { // We don't need to filter by db since we don't allow cross-database // references. query := ` diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index f205aa7dc383..0221a6c787be 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -530,7 +530,7 @@ func (tc *Catalog) ExecuteDDLWithIndexVersion( } return ds.(fmt.Stringer).String(), nil - case *tree.ShowCreateFunction: + case *tree.ShowCreateRoutine: fn := stmt.Name.FunctionReference.(*tree.UnresolvedName) def, err := tc.ResolveFunction(context.Background(), fn, tree.EmptySearchPath) if err != nil { diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 53315f792d63..009513813e87 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -8562,7 +8562,7 @@ show_create_stmt: | SHOW CREATE FUNCTION db_object_name { /* SKIP DOC */ - $$.val = &tree.ShowCreateFunction{ + $$.val = &tree.ShowCreateRoutine{ Name: tree.ResolvableFunctionReference{ FunctionReference: $4.unresolvedObjectName().ToUnresolvedName(), }, diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 73e9a8859bce..013362fd22d3 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -1394,18 +1394,19 @@ func (s ShowCompletions) Format(ctx *FmtCtx) { var _ Statement = &ShowCompletions{} -// ShowCreateFunction represents a SHOW CREATE FUNCTION statement. -type ShowCreateFunction struct { +// ShowCreateRoutine represents a SHOW CREATE FUNCTION or SHOW CREATE PROCEDURE +// statement. +type ShowCreateRoutine struct { Name ResolvableFunctionReference } // Format implements the NodeFormatter interface. -func (node *ShowCreateFunction) Format(ctx *FmtCtx) { +func (node *ShowCreateRoutine) Format(ctx *FmtCtx) { ctx.WriteString("SHOW CREATE FUNCTION ") ctx.FormatNode(&node.Name) } -var _ Statement = &ShowCreateFunction{} +var _ Statement = &ShowCreateRoutine{} // ShowCreateExternalConnections represents a SHOW CREATE EXTERNAL CONNECTION // statement. diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 68bec9778bbc..36a07b4d9609 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -2007,13 +2007,13 @@ func (*ShowCompletions) observerStatement() {} func (*ShowCompletions) hiddenFromShowQueries() {} // StatementReturnType implements the Statement interface. -func (*ShowCreateFunction) StatementReturnType() StatementReturnType { return Rows } +func (*ShowCreateRoutine) StatementReturnType() StatementReturnType { return Rows } // StatementType implements the Statement interface. -func (*ShowCreateFunction) StatementType() StatementType { return TypeDML } +func (*ShowCreateRoutine) StatementType() StatementType { return TypeDML } // StatementTag returns a short string identifying the type of statement. -func (*ShowCreateFunction) StatementTag() string { return "SHOW CREATE FUNCTION" } +func (*ShowCreateRoutine) StatementTag() string { return "SHOW CREATE FUNCTION" } // StatementReturnType implements the Statement interface. func (*ShowCreateExternalConnections) StatementReturnType() StatementReturnType { return Rows } @@ -2333,7 +2333,7 @@ func (n *ShowDatabases) String() string { return AsString( func (n *ShowDatabaseIndexes) String() string { return AsString(n) } func (n *ShowEnums) String() string { return AsString(n) } func (n *ShowFullTableScans) String() string { return AsString(n) } -func (n *ShowCreateFunction) String() string { return AsString(n) } +func (n *ShowCreateRoutine) String() string { return AsString(n) } func (n *ShowCreateExternalConnections) String() string { return AsString(n) } func (n *ShowFunctions) String() string { return AsString(n) } func (n *ShowGrants) String() string { return AsString(n) } From f3e237e2206328c8ae87bbced08b9d513892570a Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 24 Sep 2023 11:44:15 -0400 Subject: [PATCH 06/12] sql/logictest: move create_function_statements tests to crdb_internal Release note: None --- .../testdata/logic_test/crdb_internal | 112 ++++++++++++++++++ pkg/sql/logictest/testdata/logic_test/udf | 86 -------------- 2 files changed, 112 insertions(+), 86 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 1459e6bd7958..0162cf0c361f 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -1443,3 +1443,115 @@ CREATE TABLE t76710_1 AS SELECT * FROM crdb_internal.statement_statistics; statement ok CREATE MATERIALIZED VIEW t76710_2 AS SELECT fingerprint_id FROM crdb_internal.cluster_statement_statistics; + +subtest end + + +subtest create_function_statements + +statement ok +CREATE FUNCTION f(INT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +statement ok +CREATE FUNCTION f(STRING, b INT) RETURNS STRING STRICT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$ SELECT 'hello' $$; + +statement ok +CREATE SCHEMA sc; + +statement +CREATE FUNCTION sc.f2(STRING) RETURNS STRING LANGUAGE SQL AS $$ SELECT 'hello' $$; + +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 ('f', 'f2') +ORDER BY function_id; +---- +CREATE FUNCTION public.f(IN INT8) 104 test 105 public 129 f + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; +$$ +CREATE FUNCTION public.f(IN STRING, IN b INT8) 104 test 105 public 130 f + RETURNS STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ +CREATE FUNCTION sc.f2(IN STRING) 104 test 131 sc 132 f2 + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ + +statement ok +CREATE DATABASE test_cross_db; +USE test_cross_db; +CREATE FUNCTION f_cross_db() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +USE test; + +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 ('f', 'f2', 'f_cross_db') +ORDER BY function_id; +---- +CREATE FUNCTION public.f(IN INT8) 104 test 105 public 129 f + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; +$$ +CREATE FUNCTION public.f(IN STRING, IN b INT8) 104 test 105 public 130 f + RETURNS STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ +CREATE FUNCTION sc.f2(IN STRING) 104 test 131 sc 132 f2 + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; +$$ +CREATE FUNCTION public.f_cross_db() 133 test_cross_db 134 public 135 f_cross_db + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; +$$ + +statement ok +DROP FUNCTION f(INT); +DROP FUNCTION f(STRING, INT); +DROP FUNCTION sc.f2(STRING); +DROP SCHEMA sc; +USE test_cross_db; +DROP FUNCTION f_cross_db; +USE test; +DROP DATABASE test_cross_db; + +subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 431c2aa29b0a..a867250b71f0 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -167,92 +167,6 @@ statement ok USE test; subtest end -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 118 proc_f -CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) - RETURNS STRING - IMMUTABLE - LEAKPROOF - STRICT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ 104 test 105 public 119 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 120 sc 121 proc_f_2 - -statement ok -CREATE DATABASE test_cross_db; -USE test_cross_db; -CREATE FUNCTION f_cross_db() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; -USE test; - -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', 'f_cross_db') -ORDER BY database_id, 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 118 proc_f -CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) - RETURNS STRING - IMMUTABLE - LEAKPROOF - STRICT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ 104 test 105 public 119 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 120 sc 121 proc_f_2 -CREATE FUNCTION public.f_cross_db() - RETURNS INT8 - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 1; -$$ 122 test_cross_db 123 public 124 f_cross_db -subtest end - subtest show_create_function query T From a42b150d68051b754c88e30447a8e6cdecae5928 Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Wed, 27 Sep 2023 01:04:01 -0600 Subject: [PATCH 07/12] plpgsql: add builtin function for closing cursors This patch adds the `crdb_internal.plpgsql_close` builtin, which closes the cursor with the given name. It returns a `34000` error if there is no cursor with the given name. A following commit will use this to implement the PLpgSQL CLOSE statement. Informs #109709 Release note: None --- pkg/sql/faketreeeval/evalctx.go | 5 ++++ .../testdata/logic_test/plpgsql_builtins | 23 +++++++++++++++++++ pkg/sql/sem/builtins/builtins.go | 18 +++++++++++++++ pkg/sql/sem/builtins/fixed_oids.go | 1 + pkg/sql/sem/eval/deps.go | 5 ++++ pkg/sql/sem/tree/eval.go | 3 ++- pkg/sql/sql_cursor.go | 5 ++++ 7 files changed, 59 insertions(+), 1 deletion(-) diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 032082fdb9cb..c4fa64c3bb55 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -338,6 +338,11 @@ func (*DummyEvalPlanner) GenUniqueCursorName() tree.Name { return "" } +// PLpgSQLCloseCursor is part of the eval.Planner interface. +func (*DummyEvalPlanner) PLpgSQLCloseCursor(_ tree.Name) error { + return errors.WithStack(errEvalPlanner) +} + var _ eval.Planner = &DummyEvalPlanner{} var errEvalPlanner = pgerror.New(pgcode.ScalarOperationCannotRunWithoutFullSessionContext, diff --git a/pkg/sql/logictest/testdata/logic_test/plpgsql_builtins b/pkg/sql/logictest/testdata/logic_test/plpgsql_builtins index 586d2ca017ea..da38dbd1e096 100644 --- a/pkg/sql/logictest/testdata/logic_test/plpgsql_builtins +++ b/pkg/sql/logictest/testdata/logic_test/plpgsql_builtins @@ -93,3 +93,26 @@ query T SELECT crdb_internal.plpgsql_gen_cursor_name(NULL); ---- + +# Testing crdb_internal.plpgsql_close. +statement ok +BEGIN; +DECLARE foo CURSOR FOR SELECT generate_series(1, 5); + +query T +SELECT name FROM pg_cursors; +---- +foo + +statement ok +SELECT crdb_internal.plpgsql_close('foo'); + +query T +SELECT name FROM pg_cursors; +---- + +statement ok +ABORT; + +statement error pgcode 34000 pq: cursor \"foo\" does not exist +SELECT crdb_internal.plpgsql_close('foo'); diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 0daf4393501e..11f8ac79a96f 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -8463,6 +8463,24 @@ specified store on the node it's run from. One of 'mvccGC', 'merge', 'split', CalledOnNullInput: true, }, ), + "crdb_internal.plpgsql_close": makeBuiltin(tree.FunctionProperties{ + Category: builtinconstants.CategoryString, + Undocumented: true, + }, + tree.Overload{ + Types: tree.ParamTypes{{Name: "name", Typ: types.String}}, + ReturnType: tree.FixedReturnType(types.Int), + Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + if args[0] == tree.DNull { + return nil, errors.AssertionFailedf("expected non-null argument for plpgsql_close") + } + return tree.DNull, evalCtx.Planner.PLpgSQLCloseCursor(tree.Name(tree.MustBeDString(args[0]))) + }, + Info: "This function is used internally to implement the PLpgSQL CLOSE statement.", + Volatility: volatility.Volatile, + CalledOnNullInput: true, + }, + ), } var lengthImpls = func(incBitOverload bool) builtinDefinition { diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index efe1e93963c9..a8d07a9a9e22 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -2464,6 +2464,7 @@ var builtinOidsArray = []string{ 2493: `date_trunc(element: string, input: timestamptz, timezone: string) -> timestamptz`, 2494: `make_date(year: int, month: int, day: int) -> date`, 2495: `crdb_internal.plpgsql_gen_cursor_name(name: string) -> string`, + 2496: `crdb_internal.plpgsql_close(name: string) -> int`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 64f60f4ea521..781fe140e7e2 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -415,6 +415,11 @@ type Planner interface { // the current list of cursors and portals. It is used to implement PLpgSQL // OPEN statements when used with an unnamed cursor. GenUniqueCursorName() tree.Name + + // PLpgSQLCloseCursor closes the cursor with the given name, returning an + // error if the cursor doesn't exist. It is used to implement the PLpgSQL + // CLOSE statement. + PLpgSQLCloseCursor(cursorName tree.Name) error } // InternalRows is an iterator interface that's exposed by the internal diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 586b55a2778c..8dc8b769c168 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -2052,7 +2052,8 @@ func (e *MultipleResultsError) Error() string { func (expr *FuncExpr) MaybeWrapError(err error) error { // If we are facing an explicit error, propagate it unchanged. fName := expr.Func.String() - if fName == `crdb_internal.force_error` || fName == `crdb_internal.plpgsql_raise` { + if fName == `crdb_internal.force_error` || fName == `crdb_internal.plpgsql_raise` || + fName == `crdb_internal.plpgsql_close` { return err } // Otherwise, wrap it with context. diff --git a/pkg/sql/sql_cursor.go b/pkg/sql/sql_cursor.go index 065f4a8a84ce..f4c1934e5014 100644 --- a/pkg/sql/sql_cursor.go +++ b/pkg/sql/sql_cursor.go @@ -273,6 +273,11 @@ func (p *planner) GenUniqueCursorName() tree.Name { return p.sqlCursors.genUniqueName() } +// PLpgSQLCloseCursor implements the eval.Planner interface. +func (p *planner) PLpgSQLCloseCursor(cursorName tree.Name) error { + return p.sqlCursors.closeCursor(cursorName) +} + type sqlCursor struct { isql.Rows // txn is the transaction object that the internal executor for this cursor From d2070e5c553f6c0d8f77e77cf7345fec5b70084d Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Wed, 27 Sep 2023 01:35:51 -0600 Subject: [PATCH 08/12] plpgsql: implement CLOSE statements This patch implements the PLpgSQL CLOSE statement, which allows a PLpgSQL routine to close a cursor with the name specified by a cursor variable. Closing the cursor is handled by the internal builtin function `crdb_internal.plpgsql_close`. Informs #109709 Release note (sql change): Added support for the PLpgSQL CLOSE statement. --- .../testdata/logic_test/plpgsql_cursor | 93 ++++++++++ pkg/sql/opt/optbuilder/plpgsql.go | 42 +++++ pkg/sql/opt/optbuilder/testdata/udf_plpgsql | 170 ++++++++++++++++++ 3 files changed, 305 insertions(+) diff --git a/pkg/sql/logictest/testdata/logic_test/plpgsql_cursor b/pkg/sql/logictest/testdata/logic_test/plpgsql_cursor index 42f88f00c31e..a971b123a00d 100644 --- a/pkg/sql/logictest/testdata/logic_test/plpgsql_cursor +++ b/pkg/sql/logictest/testdata/logic_test/plpgsql_cursor @@ -857,5 +857,98 @@ FETCH FORWARD 3 FROM ""; ---- 100 +# Testing CLOSE statements. +# +# Test CLOSE within the scope of a routine. statement ok ABORT; +DROP FUNCTION f(); +CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ + DECLARE + curs STRING := 'foo'; + curs2 STRING := 'bar'; + BEGIN + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + OPEN curs FOR SELECT 1; + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + OPEN curs2 FOR SELECT 2; + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + CLOSE curs; + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + OPEN curs FOR SELECT 3; + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + CLOSE curs; + CLOSE curs2; + RAISE NOTICE 'cursors: %', (SELECT count(*) FROM pg_cursors); + RETURN 0; + END +$$ LANGUAGE PLpgSQL; + +query T noticetrace +SELECT f(); +---- +NOTICE: cursors: 0 +NOTICE: cursors: 1 +NOTICE: cursors: 2 +NOTICE: cursors: 1 +NOTICE: cursors: 2 +NOTICE: cursors: 0 + +# Test CLOSE across statement boundaries. +statement ok +CREATE OR REPLACE FUNCTION f(curs STRING) RETURNS INT AS $$ + BEGIN + CLOSE curs; + RETURN 0; + END +$$ LANGUAGE PLpgSQL; + +statement ok +BEGIN; + +statement ok +DECLARE foo CURSOR FOR SELECT 1; + +query T +SELECT name FROM pg_cursors; +---- +foo + +statement ok +SELECT f('foo'); + +query T +SELECT name FROM pg_cursors; +---- + +statement ok +DECLARE foo CURSOR FOR SELECT 1; +DECLARE bar CURSOR FOR SELECT 2; + +query T rowsort +SELECT name FROM pg_cursors; +---- +foo +bar + +statement ok +SELECT f('bar'); + +query T +SELECT name FROM pg_cursors; +---- +foo + +statement ok +SELECT f('foo'); + +query T +SELECT name FROM pg_cursors; +---- + +statement ok +ABORT; + +# Test attempting to CLOSE a nonexistent cursor. +statement error pgcode 34000 pq: cursor \"foo\" does not exist +SELECT f('foo'); diff --git a/pkg/sql/opt/optbuilder/plpgsql.go b/pkg/sql/opt/optbuilder/plpgsql.go index 0152dddd7ca4..682359f0c4d5 100644 --- a/pkg/sql/opt/optbuilder/plpgsql.go +++ b/pkg/sql/opt/optbuilder/plpgsql.go @@ -581,6 +581,48 @@ func (b *plpgsqlBuilder) buildPLpgSQLStatements(stmts []ast.Statement, s *scope) b.appendBodyStmt(&nameCon, b.callContinuation(&openCon, nameScope)) return b.callContinuation(&nameCon, s) + case *ast.Close: + // CLOSE statements close the cursor with the name supplied by a PLpgSQL + // variable. The crdb_internal.plpgsql_close builtin function handles + // closing the cursor. Build a volatile (non-inlinable) continuation + // that calls the builtin function. + closeCon := b.makeContinuation("_stmt_close") + closeCon.def.Volatility = volatility.Volatile + const closeFnName = "crdb_internal.plpgsql_close" + props, overloads := builtinsregistry.GetBuiltinProperties(closeFnName) + if len(overloads) != 1 { + panic(errors.AssertionFailedf("expected one overload for %s", closeFnName)) + } + _, source, _, err := closeCon.s.FindSourceProvidingColumn(b.ob.ctx, t.CurVar) + if err != nil { + if pgerror.GetPGCode(err) == pgcode.UndefinedColumn { + panic(pgerror.Newf(pgcode.Syntax, "\"%s\" is not a known variable", t.CurVar)) + } + panic(err) + } + if !source.(*scopeColumn).typ.Equivalent(types.String) { + panic(pgerror.Newf(pgcode.DatatypeMismatch, + "variable \"%s\" must be of type cursor or refcursor", t.CurVar, + )) + } + closeCall := b.ob.factory.ConstructFunction( + memo.ScalarListExpr{b.ob.factory.ConstructVariable(source.(*scopeColumn).id)}, + &memo.FunctionPrivate{ + Name: closeFnName, + Typ: types.Int, + Properties: props, + Overload: &overloads[0], + }, + ) + closeColName := scopeColName("").WithMetadataName(b.makeIdentifier("stmt_close")) + closeScope := closeCon.s.push() + b.ensureScopeHasExpr(closeScope) + b.ob.synthesizeColumn(closeScope, closeColName, types.Int, nil /* expr */, closeCall) + b.ob.constructProjectForScope(closeCon.s, closeScope) + b.appendBodyStmt(&closeCon, closeScope) + b.appendPlpgSQLStmts(&closeCon, stmts[i+1:]) + return b.callContinuation(&closeCon, s) + default: panic(unimplemented.New( "unimplemented PL/pgSQL statement", diff --git a/pkg/sql/opt/optbuilder/testdata/udf_plpgsql b/pkg/sql/opt/optbuilder/testdata/udf_plpgsql index fc492ab99f5b..57bbd9d8fa2d 100644 --- a/pkg/sql/opt/optbuilder/testdata/udf_plpgsql +++ b/pkg/sql/opt/optbuilder/testdata/udf_plpgsql @@ -4939,3 +4939,173 @@ project │ └── projections │ └── const: 0 [as=stmt_return_4:16] └── const: 1 + +# Testing CLOSE statement. +exec-ddl +CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ + DECLARE + curs STRING := 'foo'; + BEGIN + OPEN curs FOR SELECT 1; + CLOSE curs; + RETURN 0; + END +$$ LANGUAGE PLpgSQL; +---- + +build format=show-all +SELECT f(); +---- +project + ├── columns: f:12(int) + ├── cardinality: [1 - 1] + ├── volatile + ├── stats: [rows=1] + ├── cost: 0.05 + ├── key: () + ├── fd: ()-->(12) + ├── prune: (12) + ├── values + │ ├── cardinality: [1 - 1] + │ ├── stats: [rows=1] + │ ├── cost: 0.02 + │ ├── key: () + │ └── tuple [type=tuple] + └── projections + └── udf: f [as=f:12, type=int, volatile, udf] + └── body + └── limit + ├── columns: "_gen_cursor_name_5":11(int) + ├── cardinality: [1 - 1] + ├── volatile + ├── stats: [rows=1] + ├── key: () + ├── fd: ()-->(11) + ├── project + │ ├── columns: "_gen_cursor_name_5":11(int) + │ ├── cardinality: [1 - 1] + │ ├── volatile + │ ├── stats: [rows=1] + │ ├── key: () + │ ├── fd: ()-->(11) + │ ├── project + │ │ ├── columns: curs:1(string!null) + │ │ ├── cardinality: [1 - 1] + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ ├── fd: ()-->(1) + │ │ ├── prune: (1) + │ │ ├── values + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── stats: [rows=1] + │ │ │ ├── key: () + │ │ │ └── tuple [type=tuple] + │ │ └── projections + │ │ └── const: 'foo' [as=curs:1, type=string] + │ └── projections + │ └── udf: _gen_cursor_name_5 [as="_gen_cursor_name_5":11, type=int, outer=(1), volatile, udf] + │ ├── args + │ │ └── variable: curs:1 [type=string] + │ ├── params: curs:8(string) + │ └── body + │ └── project + │ ├── columns: "_stmt_open_1":10(int) + │ ├── outer: (8) + │ ├── cardinality: [1 - 1] + │ ├── volatile + │ ├── stats: [rows=1] + │ ├── key: () + │ ├── fd: ()-->(10) + │ ├── project + │ │ ├── columns: curs:9(string) + │ │ ├── outer: (8) + │ │ ├── cardinality: [1 - 1] + │ │ ├── volatile + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ ├── fd: ()-->(9) + │ │ ├── prune: (9) + │ │ ├── values + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── stats: [rows=1] + │ │ │ ├── key: () + │ │ │ └── tuple [type=tuple] + │ │ └── projections + │ │ └── case [as=curs:9, type=string, outer=(8), volatile] + │ │ ├── true [type=bool] + │ │ ├── when [type=string] + │ │ │ ├── is [type=bool] + │ │ │ │ ├── variable: curs:8 [type=string] + │ │ │ │ └── null [type=unknown] + │ │ │ └── function: crdb_internal.plpgsql_gen_cursor_name [type=string] + │ │ │ └── variable: curs:8 [type=string] + │ │ └── variable: curs:8 [type=string] + │ └── projections + │ └── udf: _stmt_open_1 [as="_stmt_open_1":10, type=int, outer=(9), volatile, udf] + │ ├── args + │ │ └── variable: curs:9 [type=string] + │ ├── params: curs:2(string) + │ └── body + │ ├── open-cursor + │ │ └── project + │ │ ├── columns: "?column?":3(int!null) + │ │ ├── cardinality: [1 - 1] + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ ├── fd: ()-->(3) + │ │ ├── values + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── stats: [rows=1] + │ │ │ ├── key: () + │ │ │ └── tuple [type=tuple] + │ │ └── projections + │ │ └── const: 1 [as="?column?":3, type=int] + │ └── project + │ ├── columns: "_stmt_close_2":7(int) + │ ├── outer: (2) + │ ├── cardinality: [1 - 1] + │ ├── volatile + │ ├── stats: [rows=1] + │ ├── key: () + │ ├── fd: ()-->(7) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── udf: _stmt_close_2 [as="_stmt_close_2":7, type=int, outer=(2), volatile, udf] + │ ├── args + │ │ └── variable: curs:2 [type=string] + │ ├── params: curs:4(string) + │ └── body + │ ├── project + │ │ ├── columns: stmt_close_3:5(int) + │ │ ├── outer: (4) + │ │ ├── cardinality: [1 - 1] + │ │ ├── volatile + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ ├── fd: ()-->(5) + │ │ ├── values + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── stats: [rows=1] + │ │ │ ├── key: () + │ │ │ └── tuple [type=tuple] + │ │ └── projections + │ │ └── function: crdb_internal.plpgsql_close [as=stmt_close_3:5, type=int, outer=(4), volatile] + │ │ └── variable: curs:4 [type=string] + │ └── project + │ ├── columns: stmt_return_4:6(int!null) + │ ├── cardinality: [1 - 1] + │ ├── stats: [rows=1] + │ ├── key: () + │ ├── fd: ()-->(6) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── stats: [rows=1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── const: 0 [as=stmt_return_4:6, type=int] + └── const: 1 [type=int] From a62f127c76d14c5cef3eea62c36b83dfcdca0545 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Wed, 4 Oct 2023 10:50:30 -0400 Subject: [PATCH 09/12] backupccl,delegate: change SHOW SCHEDULES columns for schedule options For 23.2 we added two columns to `SHOW SCHEDULES` that surfaces the values for the schedule options corresponding to `on_previous_running` and `on_execution_failure`. These column names were previously named `on_wait` and `on_error` since those were the internal proto names. It makes more sense to align these with the publically documented option names. Release note (sql change): `SHOW SCHEDULES` has two columns that surface the schedule options. These columns have been renamed to align with the documented option names namely `on_previous_running` and `on_execution_failure`. --- .../testdata/backup-restore/alter-schedule/schedule-options | 6 +++--- pkg/sql/delegate/show_schedules.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options index e789101fda22..1e9afbbd715a 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options +++ b/pkg/ccl/backupccl/testdata/backup-restore/alter-schedule/schedule-options @@ -44,7 +44,7 @@ alter backup schedule $fullID set schedule option on_previous_running = 'start'; ---- query-sql -select on_error, on_wait +select on_execution_failure, on_previous_running from [show schedules for backup] where id in ($fullID, $incID) order by backup_type asc; @@ -57,7 +57,7 @@ alter backup schedule $fullID set schedule option on_previous_running = 'skip'; ---- query-sql -select on_error, on_wait +select on_execution_failure, on_previous_running from [show schedules for backup] where id in ($fullID, $incID) order by backup_type asc; @@ -70,7 +70,7 @@ alter backup schedule $fullID set schedule option on_previous_running = 'wait'; ---- query-sql -select on_error, on_wait +select on_execution_failure, on_previous_running from [show schedules for backup] where id in ($fullID, $incID) order by backup_type asc; diff --git a/pkg/sql/delegate/show_schedules.go b/pkg/sql/delegate/show_schedules.go index bf639918c06c..2318d30b4937 100644 --- a/pkg/sql/delegate/show_schedules.go +++ b/pkg/sql/delegate/show_schedules.go @@ -38,8 +38,8 @@ WHERE status='%s' AND created_by_type='%s' AND created_by_id=schedule_id ) AS jobsRunning`, jobs.StatusRunning, jobs.CreatedByScheduledJobs), "owner", "created", - "crdb_internal.pb_to_json('cockroach.jobs.jobspb.ScheduleDetails', schedule_details, true)->>'wait' as on_wait", - "crdb_internal.pb_to_json('cockroach.jobs.jobspb.ScheduleDetails', schedule_details, true)->>'onError' as on_error", + "crdb_internal.pb_to_json('cockroach.jobs.jobspb.ScheduleDetails', schedule_details, true)->>'wait' as on_previous_running", + "crdb_internal.pb_to_json('cockroach.jobs.jobspb.ScheduleDetails', schedule_details, true)->>'onError' as on_execution_failure", } var whereExprs []string From 04b9c363740d4e62e954cc8454faefff46af6897 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 24 Sep 2023 12:04:03 -0400 Subject: [PATCH 10/12] sql: add crdb_internal.show_procedure_statements This commit adds the `crdb_internal.show_procedure_statements` table. It is very similar to the `show_function_statements` table, but it shows procedures instead of user-defined functions. Release note: None --- .../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 + .../zip/testzip_exclude_goroutine_stacks | 1 + .../testzip_external_process_virtualization | 1 + .../zip/testzip_include_goroutine_stacks | 1 + .../testdata/zip/testzip_include_range_info | 1 + .../zip/testzip_shared_process_virtualization | 2 + pkg/cli/zip_table_registry.go | 10 + pkg/sql/alter_function.go | 2 +- pkg/sql/catalog/descriptor.go | 9 +- pkg/sql/catalog/funcdesc/func_desc.go | 12 +- pkg/sql/crdb_internal.go | 59 +- pkg/sql/create_function.go | 2 +- .../testdata/logic_test/crdb_internal | 232 +++- .../testdata/logic_test/crdb_internal_catalog | 1214 +++++++++-------- .../logictest/testdata/logic_test/grant_table | 3 +- .../testdata/logic_test/information_schema | 13 +- .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 252 ++-- pkg/sql/logictest/testdata/logic_test/table | 3 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 2 +- .../opt/xform/testdata/external/activerecord | 2 +- pkg/sql/opt/xform/testdata/external/liquibase | 2 +- pkg/sql/opt/xform/testdata/external/navicat | 2 +- pkg/sql/opt/xform/testdata/external/pgjdbc | 6 +- pkg/sql/parser/testdata/create_procedure | 8 - pkg/sql/resolver.go | 6 +- .../scexec/scmutationexec/references.go | 2 +- pkg/sql/sem/catconstants/constants.go | 1 + pkg/sql/sem/tree/create_routine.go | 4 + 34 files changed, 1048 insertions(+), 884 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 7687082525fe..83290fdebeae 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -42,6 +42,7 @@ crdb_internal cluster_transaction_statistics table node NULL NULL crdb_internal cluster_transactions table node NULL NULL crdb_internal cluster_txn_execution_insights table node NULL NULL crdb_internal create_function_statements table node NULL NULL +crdb_internal create_procedure_statements table node NULL NULL crdb_internal create_schema_statements table node NULL NULL crdb_internal create_statements table node NULL NULL crdb_internal create_type_statements table node NULL NULL diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index e80b9563ee26..014309a8e748 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index bb6040e4eb97..a1cfd380abf6 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -10,6 +10,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index a635c3e4c92b..d6ddc4bd52bb 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index e6b5e5f1333c..b3e8fff2cc5b 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index 3e29ec9bcf04..0e420180df2f 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -44,6 +44,9 @@ zip [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_procedure_statements... +[cluster] retrieving SQL data for "".crdb_internal.create_procedure_statements: done +[cluster] retrieving SQL data for "".crdb_internal.create_procedure_statements: writing output: debug/crdb_internal.create_procedure_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_exclude_goroutine_stacks b/pkg/cli/testdata/zip/testzip_exclude_goroutine_stacks index cf3e85a928c5..0e38d6616a69 100644 --- a/pkg/cli/testdata/zip/testzip_exclude_goroutine_stacks +++ b/pkg/cli/testdata/zip/testzip_exclude_goroutine_stacks @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s --include-goroutine-stacks=f [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip_external_process_virtualization b/pkg/cli/testdata/zip/testzip_external_process_virtualization index 183f5057b842..184a175a2429 100644 --- a/pkg/cli/testdata/zip/testzip_external_process_virtualization +++ b/pkg/cli/testdata/zip/testzip_external_process_virtualization @@ -12,6 +12,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/reports/problemranges: last request failed: rpc error: ... [cluster] requesting data for debug/reports/problemranges: creating error output: debug/reports/problemranges.json.err.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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip_include_goroutine_stacks b/pkg/cli/testdata/zip/testzip_include_goroutine_stacks index e6b5e5f1333c..b3e8fff2cc5b 100644 --- a/pkg/cli/testdata/zip/testzip_include_goroutine_stacks +++ b/pkg/cli/testdata/zip/testzip_include_goroutine_stacks @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip_include_range_info b/pkg/cli/testdata/zip/testzip_include_range_info index 838161180548..75d1d8f92a53 100644 --- a/pkg/cli/testdata/zip/testzip_include_range_info +++ b/pkg/cli/testdata/zip/testzip_include_range_info @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s --include-range-info /dev/nu [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 diff --git a/pkg/cli/testdata/zip/testzip_shared_process_virtualization b/pkg/cli/testdata/zip/testzip_shared_process_virtualization index 3b6db9e170ca..5aaad4c2caa5 100644 --- a/pkg/cli/testdata/zip/testzip_shared_process_virtualization +++ b/pkg/cli/testdata/zip/testzip_shared_process_virtualization @@ -10,6 +10,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/settings... received response... writing JSON output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... writing JSON output: debug/reports/problemranges.json... 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.create_procedure_statements... writing output: debug/crdb_internal.create_procedure_statements.txt... done [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 @@ -134,6 +135,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] requesting data for debug/cluster/test-tenant/reports/problemranges: last request failed: rpc error: ... [cluster] requesting data for debug/cluster/test-tenant/reports/problemranges: creating error output: debug/cluster/test-tenant/reports/problemranges.json.err.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_function_statements... writing output: debug/cluster/test-tenant/crdb_internal.create_function_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_procedure_statements... writing output: debug/cluster/test-tenant/crdb_internal.create_procedure_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_schema_statements... writing output: debug/cluster/test-tenant/crdb_internal.create_schema_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/cluster/test-tenant/crdb_internal.create_statements.txt... done [cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/cluster/test-tenant/crdb_internal.create_type_statements.txt... done diff --git a/pkg/cli/zip_table_registry.go b/pkg/cli/zip_table_registry.go index 05adf66fe443..32ef604d4f4a 100644 --- a/pkg/cli/zip_table_registry.go +++ b/pkg/cli/zip_table_registry.go @@ -298,6 +298,16 @@ var zipInternalTablesPerCluster = DebugZipTableRegistry{ "crdb_internal.hide_sql_constants(create_statement) as create_statement", }, }, + `"".crdb_internal.create_procedure_statements`: { + nonSensitiveCols: NonSensitiveColumns{ + "database_id", + "database_name", + "schema_id", + "procedure_id", + "procedure_name", + "crdb_internal.hide_sql_constants(create_statement) as create_statement", + }, + }, // The synthetic SQL CREATE statements for all tables. // Note the "". to collect across all databases. `"".crdb_internal.create_schema_statements`: { diff --git a/pkg/sql/alter_function.go b/pkg/sql/alter_function.go index 9eb96527683a..91234516207b 100644 --- a/pkg/sql/alter_function.go +++ b/pkg/sql/alter_function.go @@ -420,7 +420,7 @@ func toSchemaOverloadSignature(fnDesc *funcdesc.Mutable) descpb.SchemaDescriptor ArgTypes: make([]*types.T, len(fnDesc.GetParams())), ReturnType: fnDesc.ReturnType.Type, ReturnSet: fnDesc.ReturnType.ReturnSet, - IsProcedure: fnDesc.IsProcedure, + IsProcedure: fnDesc.IsProcedure(), } for i := range fnDesc.Params { ret.ArgTypes[i] = fnDesc.Params[i].Type diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 32bcda3b0efc..e4dcf452b610 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -974,9 +974,14 @@ 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 converts a function descriptor back to a CREATE FUNCTION or + // CREATE PROCEDURE statement. This is mainly used for formatting, e.g., + // SHOW CREATE FUNCTION and SHOW CREATE PROCEDURE. ToCreateExpr() (*tree.CreateRoutine, error) + + // IsProcedure returns true if the descriptor represents a procedure. It + // returns false if the descriptor represents a user-defined function. + IsProcedure() bool } // FilterDroppedDescriptor returns an error if the descriptor state is DROP. diff --git a/pkg/sql/catalog/funcdesc/func_desc.go b/pkg/sql/catalog/funcdesc/func_desc.go index 148c7f156c45..a918ab54f434 100644 --- a/pkg/sql/catalog/funcdesc/func_desc.go +++ b/pkg/sql/catalog/funcdesc/func_desc.go @@ -681,7 +681,7 @@ func (desc *immutable) GetObjectType() privilege.ObjectType { // GetObjectTypeString implements the Object interface. func (desc *immutable) GetObjectTypeString() string { - if desc.IsProcedure { + if desc.IsProcedure() { return "procedure" } return "function" @@ -699,7 +699,7 @@ func (desc *immutable) GetLanguage() catpb.Function_Language { func (desc *immutable) ToOverload() (ret *tree.Overload, err error) { routineType := tree.UDFRoutine - if desc.IsProcedure { + if desc.IsProcedure() { routineType = tree.ProcedureRoutine } ret = &tree.Overload{ @@ -773,7 +773,8 @@ func (desc *immutable) calledOnNullInput() (bool, error) { // ToCreateExpr implements the FunctionDescriptor interface. func (desc *immutable) ToCreateExpr() (ret *tree.CreateRoutine, err error) { ret = &tree.CreateRoutine{ - Name: tree.MakeRoutineNameFromPrefix(tree.ObjectNamePrefix{}, tree.Name(desc.Name)), + Name: tree.MakeRoutineNameFromPrefix(tree.ObjectNamePrefix{}, tree.Name(desc.Name)), + IsProcedure: desc.IsProcedure(), ReturnType: tree.RoutineReturnType{ Type: desc.ReturnType.Type, SetOf: desc.ReturnType.ReturnSet, @@ -804,6 +805,11 @@ func (desc *immutable) ToCreateExpr() (ret *tree.CreateRoutine, err error) { return ret, nil } +// IsProcedure implements the FunctionDescriptor interface. +func (desc *immutable) IsProcedure() bool { + return desc.FunctionDescriptor.IsProcedure +} + func (desc *immutable) getCreateExprLang() tree.RoutineLanguage { switch desc.Lang { case catpb.Function_SQL: diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index faa9746f4ee3..357ddc80e6b5 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -143,6 +143,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalClusterSettingsTableID: crdbInternalClusterSettingsTable, catconstants.CrdbInternalClusterStmtStatsTableID: crdbInternalClusterStmtStatsTable, catconstants.CrdbInternalCreateFunctionStmtsTableID: crdbInternalCreateFunctionStmtsTable, + catconstants.CrdbInternalCreateProcedureStmtsTableID: crdbInternalCreateProcedureStmtsTable, catconstants.CrdbInternalCreateSchemaStmtsTableID: crdbInternalCreateSchemaStmtsTable, catconstants.CrdbInternalCreateStmtsTableID: crdbInternalCreateStmtsTable, catconstants.CrdbInternalCreateTypeStmtsTableID: crdbInternalCreateTypeStmtsTable, @@ -3525,20 +3526,10 @@ 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 { +func createRoutinePopulate( + procedure bool, +) func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + return func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { var dbDescs []catalog.DatabaseDescriptor if db == nil { var err error @@ -3577,8 +3568,10 @@ CREATE TABLE crdb_internal.create_function_statements ( for _, desc := range fnDescs { fnDesc := desc.(catalog.FunctionDescriptor) - if err != nil { - return err + if procedure != fnDesc.IsProcedure() { + // Skip functions if procedure is true, and skip procedures + // otherwise. + continue } treeNode, err := fnDesc.ToCreateExpr() treeNode.Name.ObjectNamePrefix = tree.ObjectNamePrefix{ @@ -3622,7 +3615,39 @@ CREATE TABLE crdb_internal.create_function_statements ( } } return nil - }, + } +} + +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: createRoutinePopulate(false /* procedure */), +} + +var crdbInternalCreateProcedureStmtsTable = virtualSchemaTable{ + comment: "CREATE statements for all user-defined procedures", + schema: ` +CREATE TABLE crdb_internal.create_procedure_statements ( + database_id INT, + database_name STRING, + schema_id INT, + schema_name STRING, + procedure_id INT, + procedure_name STRING, + create_statement STRING +) +`, + populate: createRoutinePopulate(true /* procedure */), } // Prepare the row populate function. diff --git a/pkg/sql/create_function.go b/pkg/sql/create_function.go index f036d0ea07c2..fdf0ebaf7c5f 100644 --- a/pkg/sql/create_function.go +++ b/pkg/sql/create_function.go @@ -149,7 +149,7 @@ func (n *createFunctionNode) createNewFunction( ArgTypes: paramTypes, ReturnType: returnType, ReturnSet: udfDesc.ReturnType.ReturnSet, - IsProcedure: udfDesc.IsProcedure, + IsProcedure: udfDesc.IsProcedure(), }, ) if err := params.p.writeSchemaDescChange(params.ctx, scDesc, "Create Function"); err != nil { diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 0162cf0c361f..00751576cce9 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -34,6 +34,7 @@ crdb_internal cluster_transaction_statistics table node NULL NULL crdb_internal cluster_transactions table node NULL NULL crdb_internal cluster_txn_execution_insights table node NULL NULL crdb_internal create_function_statements table node NULL NULL +crdb_internal create_procedure_statements table node NULL NULL crdb_internal create_schema_statements table node NULL NULL crdb_internal create_statements table node NULL NULL crdb_internal create_type_statements table node NULL NULL @@ -1455,45 +1456,50 @@ CREATE FUNCTION f(INT) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; statement ok CREATE FUNCTION f(STRING, b INT) RETURNS STRING STRICT IMMUTABLE LEAKPROOF LANGUAGE SQL AS $$ SELECT 'hello' $$; +# Add a procedure to ensure that it doesn't show up in the +# create_function_statements table. +statement ok +CREATE PROCEDURE f(BOOL) LANGUAGE SQL AS $$ SELECT 1 $$; + statement ok CREATE SCHEMA sc; statement CREATE FUNCTION sc.f2(STRING) RETURNS STRING LANGUAGE SQL AS $$ SELECT 'hello' $$; -query TITITIT -SELECT create_statement, database_id, database_name, schema_id, schema_name, function_id, function_name +query ITITITT +SELECT database_id, database_name, schema_id, schema_name, function_id, function_name, create_statement FROM crdb_internal.create_function_statements WHERE function_name IN ('f', 'f2') ORDER BY function_id; ---- -CREATE FUNCTION public.f(IN INT8) 104 test 105 public 129 f - RETURNS INT8 - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 1; -$$ -CREATE FUNCTION public.f(IN STRING, IN b INT8) 104 test 105 public 130 f - RETURNS STRING - IMMUTABLE - LEAKPROOF - STRICT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ -CREATE FUNCTION sc.f2(IN STRING) 104 test 131 sc 132 f2 - RETURNS STRING - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ +104 test 105 public 129 f CREATE FUNCTION public.f(IN INT8) + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +104 test 105 public 130 f CREATE FUNCTION public.f(IN STRING, IN b INT8) + RETURNS STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +104 test 132 sc 133 f2 CREATE FUNCTION sc.f2(IN STRING) + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ statement ok CREATE DATABASE test_cross_db; @@ -1501,52 +1507,54 @@ USE test_cross_db; CREATE FUNCTION f_cross_db() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; USE test; -query TITITIT -SELECT create_statement, database_id, database_name, schema_id, schema_name, function_id, function_name +query ITITITT +SELECT database_id, database_name, schema_id, schema_name, function_id, function_name, create_statement FROM "".crdb_internal.create_function_statements WHERE function_name IN ('f', 'f2', 'f_cross_db') ORDER BY function_id; ---- -CREATE FUNCTION public.f(IN INT8) 104 test 105 public 129 f - RETURNS INT8 - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 1; -$$ -CREATE FUNCTION public.f(IN STRING, IN b INT8) 104 test 105 public 130 f - RETURNS STRING - IMMUTABLE - LEAKPROOF - STRICT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ -CREATE FUNCTION sc.f2(IN STRING) 104 test 131 sc 132 f2 - RETURNS STRING - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ -CREATE FUNCTION public.f_cross_db() 133 test_cross_db 134 public 135 f_cross_db - RETURNS INT8 - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 1; -$$ +104 test 105 public 129 f CREATE FUNCTION public.f(IN INT8) + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +104 test 105 public 130 f CREATE FUNCTION public.f(IN STRING, IN b INT8) + RETURNS STRING + IMMUTABLE + LEAKPROOF + STRICT + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +104 test 132 sc 133 f2 CREATE FUNCTION sc.f2(IN STRING) + RETURNS STRING + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +134 test_cross_db 135 public 136 f_cross_db CREATE FUNCTION public.f_cross_db() + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ statement ok DROP FUNCTION f(INT); DROP FUNCTION f(STRING, INT); +-- TODO(mgartner): This should use DROP PROCEDURE once it is supported. +DROP FUNCTION f(BOOL); DROP FUNCTION sc.f2(STRING); DROP SCHEMA sc; USE test_cross_db; @@ -1555,3 +1563,93 @@ USE test; DROP DATABASE test_cross_db; subtest end + + +subtest create_procedure_statements + +statement ok +CREATE PROCEDURE p(INT) LANGUAGE SQL AS $$ SELECT 1 $$; + +statement ok +CREATE PROCEDURE p(STRING, b INT) LANGUAGE SQL AS $$ SELECT 'hello' $$; + +# Add a function to ensure that it doesn't show up in the +# create_procedure_statements table. +statement ok +CREATE FUNCTION p(BOOL) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +statement ok +CREATE SCHEMA sc; + +statement +CREATE PROCEDURE sc.p2(STRING) LANGUAGE SQL AS $$ SELECT 'hello' $$; + +query ITITITT +SELECT database_id, database_name, schema_id, schema_name, procedure_id, procedure_name, create_statement +FROM crdb_internal.create_procedure_statements +WHERE procedure_name IN ('p', 'p2') +ORDER BY procedure_id; +---- +104 test 105 public 137 p CREATE PROCEDURE public.p(IN INT8) + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +104 test 105 public 138 p CREATE PROCEDURE public.p(IN STRING, IN b INT8) + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +104 test 140 sc 141 p2 CREATE PROCEDURE sc.p2(IN STRING) + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ + +statement ok +CREATE DATABASE test_cross_db; +USE test_cross_db; +CREATE PROCEDURE p_cross_db() LANGUAGE SQL AS $$ SELECT 1 $$; +USE test; + +query ITITITT +SELECT database_id, database_name, schema_id, schema_name, procedure_id, procedure_name, create_statement +FROM "".crdb_internal.create_procedure_statements +WHERE procedure_name IN ('p', 'p2', 'p_cross_db') +ORDER BY procedure_id; +---- +104 test 105 public 137 p CREATE PROCEDURE public.p(IN INT8) + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +104 test 105 public 138 p CREATE PROCEDURE public.p(IN STRING, IN b INT8) + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +104 test 140 sc 141 p2 CREATE PROCEDURE sc.p2(IN STRING) + LANGUAGE SQL + AS $$ + SELECT 'hello'; + $$ +142 test_cross_db 143 public 144 p_cross_db CREATE PROCEDURE public.p_cross_db() + LANGUAGE SQL + AS $$ + SELECT 1; + $$ + +# TODO(mgartner): These should use DROP PROCEDURE statements once it is +# supported. +statement ok +DROP FUNCTION p(INT); +DROP FUNCTION p(STRING, INT); +DROP FUNCTION p(BOOL); +DROP FUNCTION sc.p2(STRING); +DROP SCHEMA sc; +USE test_cross_db; +DROP FUNCTION p_cross_db; +USE test; +DROP DATABASE test_cross_db; + +subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 491b5a5245ca..1d2eab224d6c 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -172,300 +172,301 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor O 111 {"table": {"checks": [{"columnIds": [1], "constraintId": 2, "expr": "k > 0:::INT8", "name": "ck"}], "columns": [{"id": 1, "name": "k", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "dependedOnBy": [{"columnIds": [1, 2], "id": 112}], "formatVersion": 3, "id": 111, "name": "kv", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["k"], "name": "kv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "4"}} 112 {"table": {"columns": [{"id": 1, "name": "k", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "hidden": true, "id": 3, "name": "rowid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "dependsOn": [111], "formatVersion": 3, "id": 112, "indexes": [{"createdExplicitly": true, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["v"], "keySuffixColumnIds": [3], "name": "idx", "partitioning": {}, "sharded": {}, "version": 4}], "isMaterializedView": true, "name": "mv", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["rowid"], "name": "mv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2], "storeColumnNames": ["k", "v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "8", "viewQuery": "SELECT k, v FROM db.public.kv"}} 113 {"function": {"functionBody": "SELECT json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(d, ARRAY['table':::STRING, 'families':::STRING]:::STRING[]), ARRAY['table':::STRING, 'nextFamilyId':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '0':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '1':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '2':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'primaryIndex':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'createAsOfTime':::STRING]:::STRING[]), ARRAY['table':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['function':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['type':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['schema':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['database':::STRING, 'modificationTime':::STRING]:::STRING[]);", "id": 113, "lang": "SQL", "name": "strip_volatile", "nullInputBehavior": "CALLED_ON_NULL_INPUT", "params": [{"class": "IN", "name": "d", "type": {"family": "JsonFamily", "oid": 3802}}], "parentId": 104, "parentSchemaId": 105, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "1048576", "userProto": "public"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966977 {"table": {"columns": [{"id": 1, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "auth_name", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 256}}, {"id": 3, "name": "auth_srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "srtext", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}, {"id": 5, "name": "proj4text", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}], "formatVersion": 3, "id": 4294966977, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966980, "version": "1"}} -4294966978 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966978, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966980, "version": "1"}} -4294966979 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geography_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966979, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966980, "version": "1"}} -4294966980 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966980, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294966981 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "viewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "viewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966981, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966982 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966982, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966983 {"table": {"columns": [{"id": 1, "name": "umid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966983, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966984 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "umserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966984, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966985 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "typname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "typnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "typowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "typlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "typbyval", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "typtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 8, "name": "typcategory", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "typispreferred", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "typisdefined", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "typdelim", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "typrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "typelem", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "typarray", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "typinput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 16, "name": "typoutput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 17, "name": "typreceive", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 18, "name": "typsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 19, "name": "typmodin", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 20, "name": "typmodout", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 21, "name": "typanalyze", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 22, "name": "typalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 23, "name": "typstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 24, "name": "typnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "typbasetype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "typtypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 27, "name": "typndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 28, "name": "typcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "typdefaultbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "typdefault", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "typacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966985, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966986 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tmplname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tmplnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "tmplinit", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "tmpllexize", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966986, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966987 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prsname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "prsnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "prsstart", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "prstoken", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "prsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "prsheadline", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "prslextype", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966987, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966988 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "dictname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "dictnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "dictowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "dicttemplate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "dictinitoption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966988, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966989 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "cfgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "cfgnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "cfgowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "cfgparser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966989, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966990 {"table": {"columns": [{"id": 1, "name": "mapcfg", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "maptokentype", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "mapseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "mapdict", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966990, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966991 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tgrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "tgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tgfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "tgtype", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "tgenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "tgisinternal", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "tgconstrrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "tgconstrindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "tgconstraint", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "tgdeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "tginitdeferred", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "tgnargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 14, "name": "tgattr", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 15, "name": "tgargs", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "tgqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "tgoldtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 18, "name": "tgnewtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 19, "name": "tgparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966991, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966992 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "trftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "trflang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "trffromsql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "trftosql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966992, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966993 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 4, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966993, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966994 {"table": {"columns": [{"id": 1, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 3, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966994, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966995 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "spcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "spcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "spclocation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "spcacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 6, "name": "spcoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966995, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966996 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tableowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "hasrules", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "hastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rowsecurity", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966996, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966997 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subdbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "subowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "subenabled", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "subconninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "subslotname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "subsynccommit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "subpublications", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966997, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966998 {"table": {"columns": [{"id": 1, "name": "srsubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srsubstate", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 4, "name": "srsublsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966998, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294966999 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "inherited", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "null_frac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 6, "name": "avg_width", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "n_distinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 10, "name": "histogram_bounds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "correlation", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "most_common_elems", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "most_common_elem_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 14, "name": "elem_count_histogram", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}], "formatVersion": 3, "id": 4294966999, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967000 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "statistics_schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "statistics_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "statistics_owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "attnames", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 7, "name": "kinds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 8, "name": "n_distinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "dependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "most_common_val_nulls", "nullable": true, "type": {"arrayContents": {"oid": 16}, "arrayElemType": "BoolFamily", "family": "ArrayFamily", "oid": 1000}}, {"id": 12, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}, {"id": 13, "name": "most_common_base_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}], "formatVersion": 3, "id": 4294967000, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967001 {"table": {"columns": [{"id": 1, "name": "starelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "staattnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 3, "name": "stainherit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "stanullfrac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 5, "name": "stawidth", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "stadistinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "stakind1", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 8, "name": "stakind2", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 9, "name": "stakind3", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 10, "name": "stakind4", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "stakind5", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 12, "name": "staop1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "staop2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "staop3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "staop4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "staop5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "stacoll1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "stacoll2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "stacoll3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "stacoll4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "stacoll5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "stanumbers1", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 23, "name": "stanumbers2", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 24, "name": "stanumbers3", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 25, "name": "stanumbers4", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 26, "name": "stanumbers5", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 27, "name": "stavalues1", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "stavalues2", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "stavalues3", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "stavalues4", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "stavalues5", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967001, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967002 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "stxname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "stxnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "stxowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "stxstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "stxkeys", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 8, "name": "stxkind", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967002, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967003 {"table": {"columns": [{"id": 1, "name": "stxoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxdndistinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "stxddependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stxdmcv", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967003, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967004 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967004, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967005 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967005, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967006 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967006, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967007 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967007, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967008 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967008, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967009 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967009, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967010 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967010, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967011 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967011, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967012 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967012, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967013 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967013, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967014 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967014, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967015 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967015, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967016 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967016, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967017 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "receive_start_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "receive_start_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "written_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "flushed_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "received_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "slot_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "sender_host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "sender_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 15, "name": "conninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967017, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967018, "name": "pg_stat_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967019 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967019, "name": "pg_stat_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967020 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967020, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967021, "name": "pg_stat_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967022 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967022, "name": "pg_stat_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967023 {"table": {"columns": [{"id": 1, "name": "subid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "received_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967023, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967024 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "ssl", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "cipher", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "bits", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "compression", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "client_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_serial", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 9, "name": "issuer_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967024, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967025 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "blks_zeroed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_exists", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "flushes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "truncates", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967025, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967026 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 6, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "sent_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "write_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "replay_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "write_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "flush_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 17, "name": "replay_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 18, "name": "sync_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "sync_state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "reply_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967026, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967027 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "heap_blks_vacuumed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "index_vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "max_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "num_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967027, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967028 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "lockers_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lockers_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "current_locker_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "blocks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "blocks_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "tuples_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "tuples_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partitions_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "partitions_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967028, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967029 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cluster_index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "heap_tuples_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "heap_tuples_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "index_rebuild_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967029, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967030 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "backup_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "backup_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "tablespaces_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "tablespaces_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967030, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967031 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sample_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "sample_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "ext_stats_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "ext_stats_computed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "child_tables_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "child_tables_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "current_child_table_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967031, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967032 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "gss_authenticated", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "principal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "encrypted", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967032, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967033 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "numbackends", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "xact_commit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "xact_rollback", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tup_returned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "tup_fetched", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tup_inserted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tup_updated", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "tup_deleted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "conflicts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "temp_files", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "temp_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "deadlocks", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "checksum_failures", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "checksum_last_failure", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "blk_read_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "blk_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967033, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967034 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "confl_tablespace", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "confl_lock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "confl_snapshot", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "confl_bufferpin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "confl_deadlock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967034, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967035 {"table": {"columns": [{"id": 1, "name": "checkpoints_timed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "checkpoints_req", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "checkpoint_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 4, "name": "checkpoint_sync_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "buffers_checkpoint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "buffers_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "maxwritten_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "buffers_backend", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "buffers_backend_fsync", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "buffers_alloc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967035, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967036 {"table": {"columns": [{"id": 1, "name": "archived_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "last_archived_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "last_archived_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "failed_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "last_failed_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_failed_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967036, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967037 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967037, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967038 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967038, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967039 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 8, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "xact_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "state_change", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 14, "name": "wait_event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "wait_event", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "backend_xid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "backend_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "leader_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967039, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967040 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "off", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "allocated_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967040, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967041 {"table": {"columns": [{"id": 1, "name": "dbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967041, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967042 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967042, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967043 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967043, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1", "viewQuery": "SELECT objoid, classoid, description FROM \"\".crdb_internal.kv_catalog_comments WHERE classoid = 4294967092:::OID"}} -4294967044 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967044, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967045 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "short_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "extra_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "context", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "vartype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "source", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "min_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "max_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "enumvals", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "boot_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "reset_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "pending_restart", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967045, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967046 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "sequencename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "sequenceowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "OidFamily", "oid": 2206}}, {"id": 5, "name": "start_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "min_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "max_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "increment_by", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "cycle", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "cache_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967046, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967047 {"table": {"columns": [{"id": 1, "name": "seqrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "seqtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "seqstart", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "seqincrement", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seqmax", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "seqmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "seqcache", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "seqcycle", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967047, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967048 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967048, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967049 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "objtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "objnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "objname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967049, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967050 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967050, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967051 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcatupdate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "rolconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967051, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967052 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "ev_class", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "ev_type", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "ev_enabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "is_instead", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "ev_qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ev_action", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967052, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967053 {"table": {"columns": [{"id": 1, "name": "slot_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "plugin", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "slot_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "datoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "temporary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "active", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "active_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "catalog_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "restart_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "confirmed_flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "wal_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "safe_wal_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967053, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967054 {"table": {"columns": [{"id": 1, "name": "roident", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "roname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967054, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967055 {"table": {"columns": [{"id": 1, "name": "local_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "external_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "remote_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "local_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967055, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967056 {"table": {"columns": [{"id": 1, "name": "rngtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rngsubtype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "rngcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "rngsubopc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "rngcanonical", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "rngsubdiff", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967056, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967057 {"table": {"columns": [{"id": 1, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967057, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967058 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pubowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "puballtables", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "pubinsert", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "pubupdate", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "pubdelete", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "pubtruncate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "pubviaroot", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967058, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967059 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prpubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "prrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967059, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967060 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967060, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967061 {"table": {"columns": [{"id": 1, "name": "transaction", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "gid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepared", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967061, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967062 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepare_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "parameter_types", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 2206}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 2211}}, {"id": 5, "name": "from_sql", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967062, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967063 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "polname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "polrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "polcmd", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "polpermissive", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "polroles", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 7, "name": "polqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "polwithcheck", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967063, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967064 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "policyname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "permissive", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "roles", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 6, "name": "cmd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_check", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967064, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967065 {"table": {"columns": [{"id": 1, "name": "partrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "partstrat", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "partnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "partdefid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "partattrs", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 6, "name": "partclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 7, "name": "partcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 8, "name": "partexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967065, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967066 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opfmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opfname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opfnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opfowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967066, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967067 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "oprname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "oprnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "oprowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "oprkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "oprcanmerge", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "oprcanhash", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "oprleft", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "oprright", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "oprresult", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "oprcom", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "oprnegate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "oprcode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "oprrest", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "oprjoin", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967067, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967068 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opcmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opcnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "opcfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "opcintype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "opcdefault", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "opckeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967068, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967069 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "nspname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "nspowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "nspacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967069, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967070 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "matviewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "matviewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "ispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967070, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967071 {"table": {"columns": [{"id": 1, "name": "locktype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "database", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "relation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "page", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "tuple", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "virtualxid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "transactionid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "virtualtransaction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "fastpath", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967071, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967072 {"table": {"columns": [{"id": 1, "name": "loid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pageno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "data", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967072, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967073 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lomowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "lomacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967073, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967074 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lanname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "lanowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "lanispl", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "lanpltrusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "lanplcallfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "laninline", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "lanvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "lanacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967074, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967075 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "privtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "initprivs", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967075, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967076 {"table": {"columns": [{"id": 1, "name": "inhrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "inhparent", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "inhseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967076, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967077 {"table": {"columns": [{"id": 1, "name": "crdb_oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "indexname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "indexdef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967077, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967078 {"table": {"columns": [{"id": 1, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "indnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "indisunique", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "indnullsnotdistinct", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "indisprimary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "indisexclusion", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "indimmediate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "indisclustered", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "indisvalid", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "indcheckxmin", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "indisready", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "indislive", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "indisreplident", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "indkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 16, "name": "indcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 17, "name": "indclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 18, "name": "indoption", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 19, "name": "indexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "indpred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "indnkeyatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}], "formatVersion": 3, "id": 4294967078, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967079 {"table": {"columns": [{"id": 1, "name": "line_number", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "database", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "user_name", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "netmask", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "auth_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967079, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967080 {"table": {"columns": [{"id": 1, "name": "groname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "grosysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grolist", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}], "formatVersion": 3, "id": 4294967080, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967081 {"table": {"columns": [{"id": 1, "name": "ftrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "ftserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "ftoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967081, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967082 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "srvowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "srvfdw", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "srvtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "srvversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "srvacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "srvoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967082, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967083 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "fdwname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "fdwowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "fdwhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "fdwvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "fdwacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "fdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967083, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967084 {"table": {"columns": [{"id": 1, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "seqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "applied", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967084, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967085 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "extname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "extowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "extnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "extrelocatable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "extversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "extconfig", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "extcondition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967085, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967086 {"table": {"columns": [{"id": 1, "name": "evtname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "evtevent", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "evtowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "evtfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "evtenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "evttags", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967086, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967087 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "enumtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "enumsortorder", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 4, "name": "enumlabel", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967087, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967088 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967088, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1", "viewQuery": "SELECT objoid, classoid, objsubid, description FROM crdb_internal.kv_catalog_comments WHERE classoid != 4294967092 UNION ALL SELECT oid AS objoid, 4294967060:::OID AS classoid, 0:::INT4 AS objsubid, description AS description FROM crdb_internal.kv_builtin_function_comments"}} -4294967089 {"table": {"columns": [{"id": 1, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967089, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967090 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "defaclrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "defaclnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "defaclobjtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "defaclacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967090, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967091 {"table": {"columns": [{"id": 1, "name": "setconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 2, "name": "setdatabase", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "setrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967091, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967092 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "datdba", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "encoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "datcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "datctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "datistemplate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "datallowconn", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "datconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "datlastsysoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "datfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "datminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "dattablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "datacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967092, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967093 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_holdable", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "is_binary", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "is_scrollable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "creation_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967093, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967094 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "conowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "conforencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "contoencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "conproc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "condefault", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967094, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967095 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "contype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "condeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "condeferred", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "convalidated", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "conrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "contypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "conindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "confrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "confupdtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "confdeltype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 14, "name": "confmatchtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 15, "name": "conislocal", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "coninhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "connoinherit", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "conkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 19, "name": "confkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 20, "name": "conpfeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 21, "name": "conppeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "name": "conffeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "conexclop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 24, "name": "conbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "consrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "condef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "conparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967095, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967096 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967096, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967097 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "collname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "collowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "collencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "collcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "collctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "collprovider", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "collversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collisdeterministic", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967097, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967098 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "relname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "reltype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "reloftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "relowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "relam", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "relfilenode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "reltablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "relpages", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "reltuples", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "relallvisible", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "reltoastrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "relhasindex", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "relisshared", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "relpersistence", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "relistemp", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "relkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 19, "name": "relnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "relchecks", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 21, "name": "relhasoids", "nullable": true, "type": {"oid": 16}}, {"id": 22, "name": "relhaspkey", "nullable": true, "type": {"oid": 16}}, {"id": 23, "name": "relhasrules", "nullable": true, "type": {"oid": 16}}, {"id": 24, "name": "relhastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "relhassubclass", "nullable": true, "type": {"oid": 16}}, {"id": 26, "name": "relfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "relacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "reloptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "relforcerowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 30, "name": "relispartition", "nullable": true, "type": {"oid": 16}}, {"id": 31, "name": "relispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 32, "name": "relreplident", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 33, "name": "relrewrite", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 34, "name": "relrowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 35, "name": "relpartbound", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "relminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967098, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967099 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "castsource", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "casttarget", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "castfunc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "castcontext", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "castmethod", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967099, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967100 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "default_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967100, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967101 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "superuser", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "trusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "relocatable", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "requires", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 9, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967101, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967102 {"table": {"columns": [{"id": 1, "name": "roleid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "member", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grantor", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "admin_option", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967102, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967103 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967103, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967104 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967104, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967105 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "adrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "adnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "adbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "adsrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967105, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967106 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amprocfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amproclefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amprocrighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amprocnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amproc", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967106, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967107 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amopfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amoplefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amoprighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amopstrategy", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amoppurpose", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "amopopr", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "amopmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "amopsortfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967107, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967108 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "amstrategies", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "amsupport", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 5, "name": "amcanorder", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "amcanorderbyop", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "amcanbackward", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "amcanunique", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "amcanmulticol", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "amoptionalkey", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "amsearcharray", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "amsearchnulls", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "amstorage", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "amclusterable", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "ampredlocks", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "amkeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "aminsert", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "ambeginscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "amgettuple", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "amgetbitmap", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "amrescan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "amendscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 23, "name": "ammarkpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 24, "name": "amrestrpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 25, "name": "ambuild", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "ambuildempty", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 27, "name": "ambulkdelete", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 28, "name": "amvacuumcleanup", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "amcanreturn", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 30, "name": "amcostestimate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 31, "name": "amoptions", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 32, "name": "amhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 33, "name": "amtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967108, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967109 {"table": {"columns": [{"id": 1, "name": "aggfnoid", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 2, "name": "aggkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "aggnumdirectargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "aggtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "aggfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "aggcombinefn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "aggserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "aggdeserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 9, "name": "aggmtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "aggminvtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 11, "name": "aggmfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 12, "name": "aggfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "aggmfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "aggsortop", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "aggtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "aggtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "aggmtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "aggmtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "agginitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "aggminitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "aggfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 22, "name": "aggmfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967109, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967110, "version": "1"}} -4294967110 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967110, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294967111 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "view_definition", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "check_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_trigger_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "is_trigger_deletable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_trigger_insertable_into", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967111, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967112 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967112, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967113 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967113, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967114 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967114, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967115 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967115, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967116 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967116, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967117 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967117, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967118 {"table": {"columns": [{"id": 1, "name": "user_defined_type_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "user_defined_type_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_defined_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "user_defined_type_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_instantiable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_final", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "ordering_form", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordering_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "ordering_routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "ordering_routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "ordering_routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "reference_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 28, "name": "source_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "ref_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967118, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967119 {"table": {"columns": [{"id": 1, "name": "attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967119, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967120 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967120, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967121 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967121, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967122 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "type_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "type_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967122, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967123 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_manipulation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "action_order", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "action_condition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "action_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "action_orientation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "action_timing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "action_reference_old_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "action_reference_new_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "action_reference_old_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "action_reference_new_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967123, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967124 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967124, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967125 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "transform_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967125, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967126 {"table": {"columns": [{"id": 1, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "nodegroup_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "tablespace_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tablespace_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967126, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967127 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967127, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967128 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967128, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967129 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967129, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967130 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967130, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967131 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967131, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967132 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_deferrable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "initially_deferred", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967132, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967133 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "non_unique", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "seq_in_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "COLLATION", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "storing", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "implicit", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "is_visible", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "visibility", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967133, "name": "statistics", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967134 {"table": {"columns": [{"id": 1, "name": "conversion_factor", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 2, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unit_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967134, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967135 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "organization", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "organization_coordsys_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967135, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967136 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "geometry_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967136, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967137 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967137, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967138 {"table": {"columns": [{"id": 1, "name": "sequence_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sequence_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sequence_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "numeric_precision", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "numeric_precision_radix", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "numeric_scale", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "start_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "minimum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "maximum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "increment", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "cycle_option", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967138, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967139 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967139, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967140 {"table": {"columns": [{"id": 1, "name": "catalog_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "default_character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "crdb_is_user_defined", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967140, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967141 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "options", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967141, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967142 {"table": {"columns": [{"id": 1, "name": "sizing_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "sizing_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "supported_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967142, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967143 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967143, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967144 {"table": {"columns": [{"id": 1, "name": "implementation_info_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implementation_info_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "integer_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "character_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967144, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967145 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sub_feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sub_feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967145, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967146 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 36, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 48, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 55, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 57, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 61, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 63, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 69, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 73, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 75, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 81, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 82, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967146, "name": "routines", "nextColumnId": 83, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n.nspname AS STRING) AS specific_schema, CAST(nameconcatoid(p.proname, p.oid) AS STRING) AS specific_name, CAST(current_database() AS STRING) AS routine_catalog, CAST(n.nspname AS STRING) AS routine_schema, CAST(p.proname AS STRING) AS routine_name, CAST(CASE p.prokind WHEN 'f' THEN 'FUNCTION' WHEN 'p' THEN 'PROCEDURE' END AS STRING) AS routine_type, CAST(NULL AS STRING) AS module_catalog, CAST(NULL AS STRING) AS module_schema, CAST(NULL AS STRING) AS module_name, CAST(NULL AS STRING) AS udt_catalog, CAST(NULL AS STRING) AS udt_schema, CAST(NULL AS STRING) AS udt_name, CAST(CASE WHEN p.prokind = 'p' THEN NULL WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(CASE WHEN nt.nspname IS NOT NULL THEN current_database() END AS STRING) AS type_udt_catalog, CAST(nt.nspname AS STRING) AS type_udt_schema, CAST(t.typname AS STRING) AS type_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(CASE WHEN p.prokind != 'p' THEN 0 END AS STRING) AS dtd_identifier, CAST(CASE WHEN l.lanname = 'sql' THEN 'SQL' ELSE 'EXTERNAL' END AS STRING) AS routine_body, CAST(CASE WHEN pg_has_role(p.proowner, 'USAGE') THEN p.prosrc ELSE NULL END AS STRING) AS routine_definition, CAST(CASE WHEN l.lanname = 'c' THEN p.prosrc ELSE NULL END AS STRING) AS external_name, CAST(upper(l.lanname) AS STRING) AS external_language, CAST('GENERAL' AS STRING) AS parameter_style, CAST(CASE WHEN p.provolatile = 'i' THEN 'YES' ELSE 'NO' END AS STRING) AS is_deterministic, CAST('MODIFIES' AS STRING) AS sql_data_access, CAST(CASE WHEN p.prokind != 'p' THEN CASE WHEN p.proisstrict THEN 'YES' ELSE 'NO' END END AS STRING) AS is_null_call, CAST(NULL AS STRING) AS sql_path, CAST('YES' AS STRING) AS schema_level_routine, CAST(0 AS INT8) AS max_dynamic_result_sets, CAST(NULL AS STRING) AS is_user_defined_cast, CAST(NULL AS STRING) AS is_implicitly_invocable, CAST(CASE WHEN p.prosecdef THEN 'DEFINER' ELSE 'INVOKER' END AS STRING) AS security_type, CAST(NULL AS STRING) AS to_sql_specific_catalog, CAST(NULL AS STRING) AS to_sql_specific_schema, CAST(NULL AS STRING) AS to_sql_specific_name, CAST('NO' AS STRING) AS as_locator, CAST(NULL AS TIMESTAMPTZ) AS created, CAST(NULL AS TIMESTAMPTZ) AS last_altered, CAST(NULL AS STRING) AS new_savepoint_level, CAST('NO' AS STRING) AS is_udt_dependent, CAST(NULL AS STRING) AS result_cast_from_data_type, CAST(NULL AS STRING) AS result_cast_as_locator, CAST(NULL AS INT8) AS result_cast_char_max_length, CAST(NULL AS INT8) AS result_cast_char_octet_length, CAST(NULL AS STRING) AS result_cast_char_set_catalog, CAST(NULL AS STRING) AS result_cast_char_set_schema, CAST(NULL AS STRING) AS result_cast_char_set_name, CAST(NULL AS STRING) AS result_cast_collation_catalog, CAST(NULL AS STRING) AS result_cast_collation_schema, CAST(NULL AS STRING) AS result_cast_collation_name, CAST(NULL AS INT8) AS result_cast_numeric_precision, CAST(NULL AS INT8) AS result_cast_numeric_precision_radix, CAST(NULL AS INT8) AS result_cast_numeric_scale, CAST(NULL AS INT8) AS result_cast_datetime_precision, CAST(NULL AS STRING) AS result_cast_interval_type, CAST(NULL AS INT8) AS result_cast_interval_precision, CAST(NULL AS STRING) AS result_cast_type_udt_catalog, CAST(NULL AS STRING) AS result_cast_type_udt_schema, CAST(NULL AS STRING) AS result_cast_type_udt_name, CAST(NULL AS STRING) AS result_cast_scope_catalog, CAST(NULL AS STRING) AS result_cast_scope_schema, CAST(NULL AS STRING) AS result_cast_scope_name, CAST(NULL AS INT8) AS result_cast_maximum_cardinality, CAST(NULL AS STRING) AS result_cast_dtd_identifier FROM (pg_catalog.pg_namespace AS n JOIN pg_catalog.pg_proc AS p ON n.oid = p.pronamespace JOIN pg_catalog.pg_language AS l ON p.prolang = l.oid) LEFT JOIN (pg_catalog.pg_type AS t JOIN pg_catalog.pg_namespace AS nt ON t.typnamespace = nt.oid) ON (p.prorettype = t.oid) AND (p.prokind != 'p') WHERE (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))"}} -4294967147 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967147, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967148 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967148, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967149 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967149, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967150 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967150, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967151 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967151, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967152 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967153 {"table": {"columns": [{"id": 1, "name": "resource_group_enabled", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "resource_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "resource_group_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "thread_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "vcpu_ids", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967153, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967154 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unique_constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "unique_constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "unique_constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "match_option", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "update_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "delete_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "referenced_table_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967154, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967155 {"table": {"columns": [{"id": 1, "name": "cpu_system", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 2, "name": "messages_sent", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "swaps", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "block_ops_in", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "block_ops_out", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "context_voluntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "cpu_user", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "query_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "source_function", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "context_involuntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "duration", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 12, "name": "page_faults_major", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "page_faults_minor", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "seq", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "source_file", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "messages_received", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "source_line", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967155, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967156 {"table": {"columns": [{"id": 1, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "time", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "db", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967156, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967157 {"table": {"columns": [{"id": 1, "name": "plugin_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "load_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "plugin_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "plugin_library_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "plugin_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "plugin_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "plugin_type_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "plugin_author", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "plugin_library", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "plugin_license", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "plugin_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967157, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967158 {"table": {"columns": [{"id": 1, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "subpartition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_rows", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "avg_row_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "check_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "create_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "index_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "nodegroup", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "partition_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "partition_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "checksum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "partition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "subpartition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "update_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "max_data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "partition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "subpartition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "subpartition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967158, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967159 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "parameter_mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_result", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "parameter_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "parameter_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967159, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n_nspname AS STRING) AS specific_schema, CAST(nameconcatoid(proname, p_oid) AS STRING) AS specific_name, CAST((ss.x).n AS INT8) AS ordinal_position, CAST(CASE WHEN proargmodes IS NULL THEN 'IN' WHEN proargmodes[(ss.x).n] = 'i' THEN 'IN' WHEN proargmodes[(ss.x).n] = 'o' THEN 'OUT' WHEN proargmodes[(ss.x).n] = 'b' THEN 'INOUT' WHEN proargmodes[(ss.x).n] = 'v' THEN 'IN' WHEN proargmodes[(ss.x).n] = 't' THEN 'OUT' END AS STRING) AS parameter_mode, CAST('NO' AS STRING) AS is_result, CAST('NO' AS STRING) AS as_locator, CAST(NULLIF(proargnames[(ss.x).n], '') AS STRING) AS parameter_name, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS udt_catalog, CAST(nt.nspname AS STRING) AS udt_schema, CAST(t.typname AS STRING) AS udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST((ss.x).n AS STRING) AS dtd_identifier, CAST(CASE WHEN pg_has_role(proowner, 'USAGE') THEN pg_get_function_arg_default(p_oid, (ss.x).n) ELSE NULL END AS STRING) AS parameter_default FROM pg_type AS t, pg_namespace AS nt, (SELECT n.nspname AS n_nspname, p.proname, p.oid AS p_oid, p.proowner, p.proargnames, p.proargmodes, information_schema._pg_expandarray(COALESCE(p.proallargtypes, p.proargtypes::OID[])) AS x FROM pg_namespace AS n, pg_proc AS p WHERE (n.oid = p.pronamespace) AND (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))) AS ss WHERE (t.oid = (ss.x).x) AND (t.typnamespace = nt.oid)"}} -4294967160 {"table": {"columns": [{"id": 1, "name": "insufficient_privileges", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "missing_bytes_beyond_max_mem_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967160, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967161 {"table": {"columns": [{"id": 1, "name": "word", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967161, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967162 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "position_in_unique_constraint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967162, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967163 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967163, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967164 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967164, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967165 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967165, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967166 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "foreign_server_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967166, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967167 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967167, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967168 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "library_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_data_wrapper_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967168, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967169 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967169, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967170 {"table": {"columns": [{"id": 1, "name": "last_update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "table_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "check_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "checksum", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "extra", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "file_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "table_name", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "avg_row_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "file_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "free_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 14, "name": "table_schema", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 15, "name": "update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 17, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "create_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 20, "name": "initial_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "update_count", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 25, "name": "creation_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 26, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "fulltext_keys", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 28, "name": "row_format", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "total_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "index_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 32, "name": "last_access_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 33, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "transaction_counter", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 35, "name": "file_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "logfile_group_number", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 37, "name": "recover_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 38, "name": "deleted_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967170, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967171 {"table": {"columns": [{"id": 1, "name": "definer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "event_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "event_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "interval_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "on_completion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "originator", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "collation_connection", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "database_collation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "event_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "event_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "execute_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "interval_field", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "starts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 15, "name": "time_zone", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "character_set_client", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "ends", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "event_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "event_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "last_executed", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 22, "name": "sql_mode", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967171, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967172 {"table": {"columns": [{"id": 1, "name": "support", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "transactions", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "xa", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "savepoints", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967172, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967173 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967173, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967174 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "collection_type_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967174, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967175 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967175, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967176 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967176, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967177 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_deferrable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "initially_deferred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967177, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967178 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967178, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967179 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967180 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967180, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967181 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_nullable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 34, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "is_self_referencing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "is_identity", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "identity_generation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "identity_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "identity_increment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "identity_maximum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "identity_minimum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "identity_cycle", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_generated", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "generation_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "is_updatable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "is_hidden", "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "crdb_sql_type", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967181, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967182 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967182, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967183 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967183, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967184 {"table": {"columns": [{"id": 1, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "histogram", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967184, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967185 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967185, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967186 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967186, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967187 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967187, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967188 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dependent_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967188, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967189 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "pad_attribute", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967189, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967190 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967190, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967191 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "check_clause", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967191, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967192 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967192, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967193 {"table": {"columns": [{"id": 1, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_repertoire", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "form_of_use", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "default_collate_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "default_collate_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "default_collate_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967193, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967194 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attribute_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "attribute_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_nullable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "attribute_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "attribute_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "attribute_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "is_derived_reference_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967194, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967195 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967195, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967196 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967196, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967197, "version": "1"}} -4294967197 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967197, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294967198 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "ts", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 3, "name": "meta_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "meta", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "num_spans", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "spans", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 7, "name": "verified", "type": {"oid": 16}}, {"id": 8, "name": "target", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "decoded_meta", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "decoded_target", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "internal_meta", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "num_ranges", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_updated", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}], "formatVersion": 3, "id": 4294967198, "name": "kv_protected_ts_records", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967199 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "corruption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967199, "name": "kv_repairable_catalog_corruptions", "nextColumnId": 6, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH data AS (SELECT ns.\"parentID\" AS parent_id, ns.\"parentSchemaID\" AS parent_schema_id, ns.name, COALESCE(ns.id, d.id) AS id, d.descriptor, crdb_internal.descriptor_with_post_deserialization_changes(d.descriptor) AS updated_descriptor, crdb_internal.repaired_descriptor(d.descriptor, (SELECT array_agg(id) AS desc_id_array FROM system.descriptor), (SELECT array_agg(id) AS job_id_array FROM system.jobs WHERE status NOT IN ('failed', 'succeeded', 'canceled', 'revert-failed'))) AS repaired_descriptor FROM system.namespace AS ns FULL JOIN system.descriptor AS d ON ns.id = d.id), diag AS (SELECT *, CASE WHEN (descriptor IS NULL) AND (id != 29) THEN 'namespace' WHEN updated_descriptor != repaired_descriptor THEN 'descriptor' ELSE NULL END AS corruption FROM data) SELECT parent_id, parent_schema_id, name, id, corruption FROM diag WHERE corruption IS NOT NULL ORDER BY parent_id, parent_schema_id, name, id"}} -4294967200 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "log_term", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "log_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967200, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_token_deductions_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7], "storeColumnNames": ["tenant_id", "store_id", "priority", "log_term", "log_index", "tokens"], "version": 3}], "name": "kv_flow_token_deductions", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967201 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_tracked_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967201, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_control_handles_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["tenant_id", "store_id", "total_tracked_tokens"], "version": 3}], "name": "kv_flow_control_handles", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967202 {"table": {"columns": [{"id": 1, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "available_regular_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "available_elastic_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967202, "name": "kv_flow_controller", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967203 {"table": {"columns": [{"id": 1, "name": "username", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privileges", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "grant_options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "user_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967203, "name": "kv_system_privileges", "nextColumnId": 6, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT * FROM system.privileges"}} -4294967204 {"table": {"columns": [{"id": 1, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "inheriting_member", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "member_is_explicit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "member_is_admin", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967204, "name": "kv_inherited_role_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967205 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "capability_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "capability_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967205, "name": "node_tenant_capabilities_cache", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967206 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "ttl", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967206, "name": "kv_dropped_relations", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH dropped_relations AS (SELECT id, ((descriptor->'table')->>'name') AS name, ((descriptor->'table')->'parentId')::INT8 AS parent_id, ((descriptor->'table')->'unexposedParentSchemaId')::INT8 AS parent_schema_id, to_timestamp((((descriptor->'table')->>'dropTime')::DECIMAL * 0.000000001)::FLOAT8) AS drop_time FROM crdb_internal.kv_catalog_descriptor WHERE ((descriptor->'table')->>'state') = 'DROP'), gc_ttl AS (SELECT id, ((config->'gc')->'ttlSeconds')::INT8 AS ttl FROM crdb_internal.kv_catalog_zones) SELECT dr.parent_id, dr.parent_schema_id, dr.name, dr.id, dr.drop_time, COALESCE(gc.ttl, db_gc.ttl, root_gc.ttl) * '1 second'::INTERVAL AS ttl FROM dropped_relations AS dr LEFT JOIN gc_ttl AS gc ON gc.id = dr.id LEFT JOIN gc_ttl AS db_gc ON db_gc.id = dr.parent_id LEFT JOIN gc_ttl AS root_gc ON root_gc.id = 0 ORDER BY parent_id, parent_schema_id, id"}} -4294967207 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "super_region_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967207, "name": "super_regions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967208 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967208, "name": "pg_catalog_table_is_implemented", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967209 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "total_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 3, "name": "total_read_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_read_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_write_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "total_write_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "total_sql_pod_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "total_pgwire_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "total_external_io_ingress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "total_external_io_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "total_kv_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "total_cross_region_network_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967209, "name": "tenant_usage_details", "nextColumnId": 13, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT tenant_id, (j->>'rU')::FLOAT8 AS total_ru, (j->>'readBytes')::INT8 AS total_read_bytes, (j->>'readRequests')::INT8 AS total_read_requests, (j->>'writeBytes')::INT8 AS total_write_bytes, (j->>'writeRequests')::INT8 AS total_write_requests, (j->>'sqlPodsCpuSeconds')::FLOAT8 AS total_sql_pod_seconds, (j->>'pgwireEgressBytes')::INT8 AS total_pgwire_egress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'kvRU')::FLOAT8 AS total_kv_ru, (j->>'crossRegionNetworkRU')::FLOAT8 AS total_cross_region_network_ru FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} -4294967210 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tags", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "startts", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "diff", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "created", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "range_end", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "resolved", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_event_utc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "catchup", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967210, "name": "active_range_feeds", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967211 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "for_all_roles", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "object_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967211, "name": "default_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967212 {"table": {"columns": [{"id": 1, "name": "region", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "zones", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967212, "name": "regions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967213 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "root_op_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace_str", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "jaeger_json", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967213, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["trace_id"], "name": "cluster_inflight_traces_trace_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["node_id", "root_op_name", "trace_str", "jaeger_json"], "version": 3}], "name": "cluster_inflight_traces", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967214 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967214, "name": "lost_descriptors_with_data", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967215 {"table": {"columns": [{"id": 1, "name": "object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "referenced_object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "referenced_object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "referenced_object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cross_database_reference_description", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967215, "name": "cross_db_references", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967216 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967216, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["database_name"], "name": "cluster_database_privileges_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["grantee", "privilege_type", "is_grantable"], "version": 3}], "name": "cluster_database_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967217 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "obj_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"hidden": true, "id": 6, "name": "error_redactable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967217, "name": "invalid_objects", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967218 {"table": {"columns": [{"id": 1, "name": "zone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "subzone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "target", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "range_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "raw_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "raw_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "raw_config_protobuf", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 13, "name": "full_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "full_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967218, "name": "zones", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967219 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967219, "name": "transaction_statistics_persisted_v22_2", "nextColumnId": 8, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics FROM system.transaction_statistics"}} -4294967220 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "service_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "contention_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "total_estimated_execution_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "p99_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967220, "name": "transaction_statistics_persisted", "nextColumnId": 14, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics, execution_count, service_latency, cpu_sql_nanos, contention_time, total_estimated_execution_time, p99_latency FROM system.transaction_statistics"}} -4294967221 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967221, "name": "transaction_statistics", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, max(metadata), crdb_internal.merge_transaction_stats(array_agg(statistics)), aggregation_interval FROM (SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, aggregation_interval FROM crdb_internal.cluster_transaction_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, agg_interval FROM system.transaction_statistics) GROUP BY aggregated_ts, fingerprint_id, app_name, aggregation_interval"}} -4294967222 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "execution_total_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "execution_total_cluster_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "contention_time_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "cpu_sql_avg_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "service_latency_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "service_latency_p99_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967222, "name": "transaction_activity", "nextColumnId": 15, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, agg_interval, metadata, statistics, query, execution_count, execution_total_seconds, execution_total_cluster_seconds, contention_time_avg_seconds, cpu_sql_avg_nanos, service_latency_avg_seconds, service_latency_p99_seconds FROM system.transaction_activity"}} -4294967223 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "key", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_ids", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "max_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "retry_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "retry_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "commit_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "commit_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967223, "name": "node_transaction_statistics", "nextColumnId": 55, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967224 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "estimated_row_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967224, "name": "table_row_statistics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967225 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "version", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "mod_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "mod_time_logical", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "format_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "sc_lease_node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "sc_lease_expiration_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "audit_mode", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "parent_schema_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "locality", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967225, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["parent_id"], "name": "tables_parent_id_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "name", "database_name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["database_name"], "name": "tables_database_name_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "parent_id", "name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}], "name": "tables", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967226 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967226, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "table_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["start_key", "end_key"], "version": 3}], "name": "table_spans", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967227 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_unique", "type": {"oid": 16}}, {"id": 7, "name": "is_inverted", "type": {"oid": 16}}, {"id": 8, "name": "is_sharded", "type": {"oid": 16}}, {"id": 9, "name": "is_visible", "type": {"oid": 16}}, {"id": 10, "name": "visibility", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967227, "name": "table_indexes", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967228 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "nullable", "type": {"oid": 16}}, {"id": 7, "name": "default_expr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967228, "name": "table_columns", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967229 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967229, "name": "statement_statistics_persisted_v22_2", "nextColumnId": 12, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations FROM system.statement_statistics"}} -4294967230 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 12, "name": "indexes_usage", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "service_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "contention_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "total_estimated_execution_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "p99_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967230, "name": "statement_statistics_persisted", "nextColumnId": 19, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations, indexes_usage, execution_count, service_latency, cpu_sql_nanos, contention_time, total_estimated_execution_time, p99_latency FROM system.statement_statistics"}} -4294967231 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "sampled_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 10, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967231, "name": "statement_statistics", "nextColumnId": 11, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, max(metadata) AS metadata, crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)), max(sampled_plan), aggregation_interval, array_remove(array_cat_agg(index_recommendations), NULL) AS index_recommendations FROM (SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, sampled_plan, aggregation_interval, index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, plan, agg_interval, index_recommendations FROM system.statement_statistics) GROUP BY aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, aggregation_interval"}} -4294967232 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "execution_total_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "execution_total_cluster_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "cpu_sql_avg_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "contention_time_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "service_latency_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "service_latency_p99_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967232, "name": "statement_activity", "nextColumnId": 18, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, agg_interval, metadata, statistics, plan, index_recommendations, execution_count, execution_total_seconds, execution_total_cluster_seconds, contention_time_avg_seconds, cpu_sql_avg_nanos, service_latency_avg_seconds, service_latency_p99_seconds FROM system.statement_activity"}} -4294967233 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967233, "name": "session_variables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967234 {"table": {"columns": [{"id": 1, "name": "span_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "message_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "timestamp", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "loc", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "message", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "age", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967234, "name": "session_trace", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967235 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "target_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "target_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967235, "name": "schema_changes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967236 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "component", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "field", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967236, "name": "node_runtime_info", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967237 {"table": {"columns": [{"id": 1, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "start_pretty", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "end_key", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "end_pretty", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "replica_localities", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 9, "name": "non_voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 10, "name": "learner_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 11, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "lease_holder", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "range_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967237, "name": "ranges", "nextColumnId": 14, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT range_id, start_key, start_pretty, end_key, end_pretty, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (((crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8) + COALESCE((crdb_internal.range_stats(start_key)->>'range_key_bytes')::INT8, 0)) + COALESCE((crdb_internal.range_stats(start_key)->>'range_val_bytes')::INT8, 0) AS range_size FROM crdb_internal.ranges_no_leases"}} -4294967238 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "start_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "end_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "replica_localities", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 9, "name": "non_voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 10, "name": "learner_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 11, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967238, "name": "ranges_no_leases", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967239 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "parent_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "columns", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "column_names", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "list_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "range_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "zone_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "subzone_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967239, "name": "partitions", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967240 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "txn_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "txn_time_avg_sec", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "txn_time_var_sec", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "committed_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "implicit_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967240, "name": "node_txn_stats", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967241 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "flags", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "anonymized", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "first_attempt_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "max_retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "last_error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_error_code", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rows_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "rows_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "parse_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "parse_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "plan_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "plan_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "run_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "run_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "overhead_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "overhead_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "bytes_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "bytes_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "rows_written_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "rows_written_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 55, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 56, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 57, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 58, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 59, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 60, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 61, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 62, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 63, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 64, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 65, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 66, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 67, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 68, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 69, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 70, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 71, "name": "full_scan", "type": {"oid": 16}}, {"id": 72, "name": "sample_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 73, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 75, "name": "txn_fingerprint_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 77, "name": "latency_seconds_min", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 78, "name": "latency_seconds_max", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 79, "name": "latency_seconds_p50", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 80, "name": "latency_seconds_p90", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 81, "name": "latency_seconds_p99", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967241, "name": "node_statement_statistics", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967242 {"table": {"columns": [{"id": 1, "name": "level", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "used", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "reserved_used", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "reserved_reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967242, "name": "node_memory_monitors", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967243 {"table": {"columns": [{"id": 1, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967243, "name": "node_metrics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967244 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "active_queries", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "last_active_query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "num_txns_executed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "session_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "active_query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "kv_txn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "session_end", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967244, "name": "node_sessions", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967245 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "txn_string", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "num_stmts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "num_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "num_auto_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "last_auto_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "isolation_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "priority", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "quality_of_service", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967245, "name": "node_transactions", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967246 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967246, "name": "node_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967247 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 26, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "error_code", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "last_error_redactable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967247, "name": "node_execution_insights", "nextColumnId": 29, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967248 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967248, "name": "node_distsql_flows", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967249 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967249, "name": "node_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967250 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "deleted", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967250, "name": "leases", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967251 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 4, "name": "capacity", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "available", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "used", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "logical_bytes", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lease_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "writes_per_second", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "bytes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "writes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 14, "name": "properties", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967251, "name": "kv_store_status", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967252 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "go_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "time", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "revision", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "cgo_compiler", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "platform", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "distribution", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "dependencies", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "updated_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 18, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 19, "name": "args", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 20, "name": "env", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 21, "name": "activity", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967252, "name": "kv_node_status", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967253 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967253, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "system_jobs_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "name": "system_jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["id", "status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["status"], "name": "system_jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["id", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}], "name": "system_jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967254 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967254, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["job_type"], "name": "jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "description", "statement", "user_name", "descriptor_ids", "status", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967255 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "goroutine_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "finished", "type": {"oid": 16}}, {"id": 6, "name": "start_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967255, "name": "node_inflight_trace_spans", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967256 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967256, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967257 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967257, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967258 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "column_direction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "implicit", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967258, "name": "index_columns", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967259 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "contending_pretty_key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "waiting_stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "waiting_stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 11, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967259, "name": "transaction_contention_events", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967260 {"table": {"columns": [{"id": 1, "name": "source_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "target_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967260, "name": "gossip_network", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967261 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "decommissioning", "type": {"oid": 16}}, {"id": 6, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "updated_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967261, "name": "gossip_liveness", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967262 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "category", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967262, "name": "gossip_alerts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967263 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "advertise_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "sql_network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sql_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "advertise_sql_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cluster_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "build_tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 14, "name": "is_live", "type": {"oid": 16}}, {"id": 15, "name": "ranges", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "leases", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967263, "name": "gossip_nodes", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967264 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967264, "name": "kv_node_liveness", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967265 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "dependedonby_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "dependedonby_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "dependedonby_index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "dependedonby_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "dependedonby_details", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967265, "name": "forward_dependencies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967266 {"table": {"columns": [{"id": 1, "name": "feature_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "usage_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967266, "name": "feature_usage", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967267 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "owner", "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "primary_region", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_region", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "survival_goal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "placement_policy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967267, "name": "databases", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967268 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "descriptor_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "enum_members", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967268, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["descriptor_id"], "name": "create_type_statements_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7], "storeColumnNames": ["database_id", "database_name", "schema_name", "descriptor_name", "create_statement", "enum_members"], "version": 3}], "name": "create_type_statements", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967269 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "descriptor_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "create_nofks", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "alter_statements", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "validate_statements", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 12, "name": "create_redactable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "has_partitions", "type": {"oid": 16}}, {"id": 14, "name": "is_multi_region", "type": {"oid": 16}}, {"id": 15, "name": "is_virtual", "type": {"oid": 16}}, {"id": 16, "name": "is_temporary", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967269, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["descriptor_id"], "name": "create_statements_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["database_id", "database_name", "schema_name", "descriptor_type", "descriptor_name", "create_statement", "state", "create_nofks", "alter_statements", "validate_statements", "create_redactable", "has_partitions", "is_multi_region", "is_virtual", "is_temporary"], "version": 3}], "name": "create_statements", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967270 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967270, "name": "create_schema_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294966976 {"table": {"columns": [{"id": 1, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "auth_name", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 256}}, {"id": 3, "name": "auth_srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "srtext", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}, {"id": 5, "name": "proj4text", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}], "formatVersion": 3, "id": 4294966976, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966979, "version": "1"}} +4294966977 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966977, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966979, "version": "1"}} +4294966978 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geography_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966978, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966979, "version": "1"}} +4294966979 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966979, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294966980 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "viewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "viewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966980, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966981 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966981, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966982 {"table": {"columns": [{"id": 1, "name": "umid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966982, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966983 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "umserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966983, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966984 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "typname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "typnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "typowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "typlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "typbyval", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "typtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 8, "name": "typcategory", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "typispreferred", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "typisdefined", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "typdelim", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "typrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "typelem", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "typarray", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "typinput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 16, "name": "typoutput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 17, "name": "typreceive", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 18, "name": "typsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 19, "name": "typmodin", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 20, "name": "typmodout", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 21, "name": "typanalyze", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 22, "name": "typalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 23, "name": "typstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 24, "name": "typnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "typbasetype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "typtypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 27, "name": "typndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 28, "name": "typcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "typdefaultbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "typdefault", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "typacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966984, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966985 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tmplname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tmplnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "tmplinit", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "tmpllexize", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966985, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966986 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prsname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "prsnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "prsstart", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "prstoken", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "prsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "prsheadline", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "prslextype", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966986, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966987 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "dictname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "dictnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "dictowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "dicttemplate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "dictinitoption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966987, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966988 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "cfgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "cfgnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "cfgowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "cfgparser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966988, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966989 {"table": {"columns": [{"id": 1, "name": "mapcfg", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "maptokentype", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "mapseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "mapdict", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966989, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966990 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tgrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "tgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tgfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "tgtype", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "tgenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "tgisinternal", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "tgconstrrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "tgconstrindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "tgconstraint", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "tgdeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "tginitdeferred", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "tgnargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 14, "name": "tgattr", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 15, "name": "tgargs", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "tgqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "tgoldtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 18, "name": "tgnewtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 19, "name": "tgparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294966990, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966991 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "trftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "trflang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "trffromsql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "trftosql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966991, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966992 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 4, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966992, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966993 {"table": {"columns": [{"id": 1, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 3, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966993, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966994 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "spcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "spcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "spclocation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "spcacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 6, "name": "spcoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966994, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966995 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tableowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "hasrules", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "hastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rowsecurity", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294966995, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966996 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subdbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "subowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "subenabled", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "subconninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "subslotname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "subsynccommit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "subpublications", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966996, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966997 {"table": {"columns": [{"id": 1, "name": "srsubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srsubstate", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 4, "name": "srsublsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966997, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966998 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "inherited", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "null_frac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 6, "name": "avg_width", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "n_distinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 10, "name": "histogram_bounds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "correlation", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "most_common_elems", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "most_common_elem_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 14, "name": "elem_count_histogram", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}], "formatVersion": 3, "id": 4294966998, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294966999 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "statistics_schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "statistics_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "statistics_owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "attnames", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 7, "name": "kinds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 8, "name": "n_distinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "dependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "most_common_val_nulls", "nullable": true, "type": {"arrayContents": {"oid": 16}, "arrayElemType": "BoolFamily", "family": "ArrayFamily", "oid": 1000}}, {"id": 12, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}, {"id": 13, "name": "most_common_base_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}], "formatVersion": 3, "id": 4294966999, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967000 {"table": {"columns": [{"id": 1, "name": "starelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "staattnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 3, "name": "stainherit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "stanullfrac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 5, "name": "stawidth", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "stadistinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "stakind1", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 8, "name": "stakind2", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 9, "name": "stakind3", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 10, "name": "stakind4", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "stakind5", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 12, "name": "staop1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "staop2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "staop3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "staop4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "staop5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "stacoll1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "stacoll2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "stacoll3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "stacoll4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "stacoll5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "stanumbers1", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 23, "name": "stanumbers2", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 24, "name": "stanumbers3", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 25, "name": "stanumbers4", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 26, "name": "stanumbers5", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 27, "name": "stavalues1", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "stavalues2", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "stavalues3", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "stavalues4", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "stavalues5", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967000, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967001 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "stxname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "stxnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "stxowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "stxstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "stxkeys", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 8, "name": "stxkind", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967001, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967002 {"table": {"columns": [{"id": 1, "name": "stxoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxdndistinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "stxddependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stxdmcv", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967002, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967003 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967003, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967004 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967004, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967005 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967005, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967006 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967006, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967007 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967007, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967008 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967008, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967009 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967009, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967010 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967010, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967011 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967011, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967012 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967012, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967013 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967013, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967014 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967014, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967015 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967015, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967016 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "receive_start_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "receive_start_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "written_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "flushed_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "received_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "slot_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "sender_host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "sender_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 15, "name": "conninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967016, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967017 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967017, "name": "pg_stat_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967018, "name": "pg_stat_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967019 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967019, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967020 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967020, "name": "pg_stat_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967021, "name": "pg_stat_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967022 {"table": {"columns": [{"id": 1, "name": "subid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "received_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967022, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967023 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "ssl", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "cipher", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "bits", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "compression", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "client_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_serial", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 9, "name": "issuer_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967023, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967024 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "blks_zeroed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_exists", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "flushes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "truncates", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967024, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967025 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 6, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "sent_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "write_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "replay_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "write_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "flush_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 17, "name": "replay_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 18, "name": "sync_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "sync_state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "reply_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967025, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967026 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "heap_blks_vacuumed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "index_vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "max_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "num_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967026, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967027 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "lockers_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lockers_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "current_locker_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "blocks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "blocks_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "tuples_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "tuples_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partitions_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "partitions_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967027, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967028 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cluster_index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "heap_tuples_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "heap_tuples_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "index_rebuild_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967028, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967029 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "backup_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "backup_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "tablespaces_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "tablespaces_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967029, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967030 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sample_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "sample_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "ext_stats_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "ext_stats_computed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "child_tables_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "child_tables_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "current_child_table_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967030, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967031 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "gss_authenticated", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "principal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "encrypted", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967031, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967032 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "numbackends", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "xact_commit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "xact_rollback", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tup_returned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "tup_fetched", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tup_inserted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tup_updated", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "tup_deleted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "conflicts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "temp_files", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "temp_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "deadlocks", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "checksum_failures", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "checksum_last_failure", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "blk_read_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "blk_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967032, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967033 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "confl_tablespace", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "confl_lock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "confl_snapshot", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "confl_bufferpin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "confl_deadlock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967033, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967034 {"table": {"columns": [{"id": 1, "name": "checkpoints_timed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "checkpoints_req", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "checkpoint_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 4, "name": "checkpoint_sync_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "buffers_checkpoint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "buffers_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "maxwritten_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "buffers_backend", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "buffers_backend_fsync", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "buffers_alloc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967034, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967035 {"table": {"columns": [{"id": 1, "name": "archived_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "last_archived_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "last_archived_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "failed_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "last_failed_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_failed_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967035, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967036 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967036, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967037 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967037, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967038 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 8, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "xact_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "state_change", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 14, "name": "wait_event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "wait_event", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "backend_xid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "backend_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "leader_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967038, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967039 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "off", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "allocated_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967039, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967040 {"table": {"columns": [{"id": 1, "name": "dbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967040, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967041 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967041, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967042 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967042, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1", "viewQuery": "SELECT objoid, classoid, description FROM \"\".crdb_internal.kv_catalog_comments WHERE classoid = 4294967091:::OID"}} +4294967043 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967043, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967044 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "short_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "extra_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "context", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "vartype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "source", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "min_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "max_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "enumvals", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "boot_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "reset_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "pending_restart", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967044, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967045 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "sequencename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "sequenceowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "OidFamily", "oid": 2206}}, {"id": 5, "name": "start_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "min_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "max_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "increment_by", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "cycle", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "cache_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967045, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967046 {"table": {"columns": [{"id": 1, "name": "seqrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "seqtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "seqstart", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "seqincrement", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seqmax", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "seqmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "seqcache", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "seqcycle", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967046, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967047 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967047, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967048 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "objtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "objnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "objname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967048, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967049 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967049, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967050 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcatupdate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "rolconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967050, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967051 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "ev_class", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "ev_type", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "ev_enabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "is_instead", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "ev_qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ev_action", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967051, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967052 {"table": {"columns": [{"id": 1, "name": "slot_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "plugin", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "slot_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "datoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "temporary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "active", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "active_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "catalog_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "restart_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "confirmed_flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "wal_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "safe_wal_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967052, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967053 {"table": {"columns": [{"id": 1, "name": "roident", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "roname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967053, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967054 {"table": {"columns": [{"id": 1, "name": "local_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "external_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "remote_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "local_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967054, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967055 {"table": {"columns": [{"id": 1, "name": "rngtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rngsubtype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "rngcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "rngsubopc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "rngcanonical", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "rngsubdiff", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967055, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967056 {"table": {"columns": [{"id": 1, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967056, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967057 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pubowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "puballtables", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "pubinsert", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "pubupdate", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "pubdelete", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "pubtruncate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "pubviaroot", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967057, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967058 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prpubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "prrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967058, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967059 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967059, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967060 {"table": {"columns": [{"id": 1, "name": "transaction", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "gid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepared", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967060, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967061 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepare_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "parameter_types", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 2206}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 2211}}, {"id": 5, "name": "from_sql", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967061, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967062 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "polname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "polrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "polcmd", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "polpermissive", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "polroles", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 7, "name": "polqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "polwithcheck", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967062, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967063 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "policyname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "permissive", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "roles", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 6, "name": "cmd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_check", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967063, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967064 {"table": {"columns": [{"id": 1, "name": "partrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "partstrat", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "partnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "partdefid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "partattrs", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 6, "name": "partclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 7, "name": "partcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 8, "name": "partexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967064, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967065 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opfmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opfname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opfnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opfowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967065, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967066 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "oprname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "oprnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "oprowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "oprkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "oprcanmerge", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "oprcanhash", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "oprleft", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "oprright", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "oprresult", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "oprcom", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "oprnegate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "oprcode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "oprrest", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "oprjoin", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967066, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967067 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opcmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opcnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "opcfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "opcintype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "opcdefault", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "opckeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967067, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967068 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "nspname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "nspowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "nspacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967068, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967069 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "matviewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "matviewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "ispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967069, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967070 {"table": {"columns": [{"id": 1, "name": "locktype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "database", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "relation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "page", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "tuple", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "virtualxid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "transactionid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "virtualtransaction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "fastpath", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967070, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967071 {"table": {"columns": [{"id": 1, "name": "loid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pageno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "data", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967071, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967072 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lomowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "lomacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967072, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967073 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lanname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "lanowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "lanispl", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "lanpltrusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "lanplcallfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "laninline", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "lanvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "lanacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967073, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967074 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "privtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "initprivs", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967074, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967075 {"table": {"columns": [{"id": 1, "name": "inhrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "inhparent", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "inhseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967075, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967076 {"table": {"columns": [{"id": 1, "name": "crdb_oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "indexname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "indexdef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967076, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967077 {"table": {"columns": [{"id": 1, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "indnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "indisunique", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "indnullsnotdistinct", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "indisprimary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "indisexclusion", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "indimmediate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "indisclustered", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "indisvalid", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "indcheckxmin", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "indisready", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "indislive", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "indisreplident", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "indkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 16, "name": "indcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 17, "name": "indclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 18, "name": "indoption", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 19, "name": "indexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "indpred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "indnkeyatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}], "formatVersion": 3, "id": 4294967077, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967078 {"table": {"columns": [{"id": 1, "name": "line_number", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "database", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "user_name", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "netmask", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "auth_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967078, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967079 {"table": {"columns": [{"id": 1, "name": "groname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "grosysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grolist", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}], "formatVersion": 3, "id": 4294967079, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967080 {"table": {"columns": [{"id": 1, "name": "ftrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "ftserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "ftoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967080, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967081 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "srvowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "srvfdw", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "srvtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "srvversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "srvacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "srvoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967081, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967082 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "fdwname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "fdwowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "fdwhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "fdwvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "fdwacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "fdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967082, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967083 {"table": {"columns": [{"id": 1, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "seqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "applied", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967083, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967084 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "extname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "extowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "extnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "extrelocatable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "extversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "extconfig", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "extcondition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967084, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967085 {"table": {"columns": [{"id": 1, "name": "evtname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "evtevent", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "evtowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "evtfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "evtenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "evttags", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967085, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967086 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "enumtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "enumsortorder", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 4, "name": "enumlabel", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967086, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967087 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967087, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1", "viewQuery": "SELECT objoid, classoid, objsubid, description FROM crdb_internal.kv_catalog_comments WHERE classoid != 4294967091 UNION ALL SELECT oid AS objoid, 4294967059:::OID AS classoid, 0:::INT4 AS objsubid, description AS description FROM crdb_internal.kv_builtin_function_comments"}} +4294967088 {"table": {"columns": [{"id": 1, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967088, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967089 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "defaclrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "defaclnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "defaclobjtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "defaclacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967089, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967090 {"table": {"columns": [{"id": 1, "name": "setconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 2, "name": "setdatabase", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "setrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967090, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967091 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "datdba", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "encoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "datcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "datctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "datistemplate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "datallowconn", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "datconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "datlastsysoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "datfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "datminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "dattablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "datacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967091, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967092 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_holdable", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "is_binary", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "is_scrollable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "creation_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967092, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967093 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "conowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "conforencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "contoencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "conproc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "condefault", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967093, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967094 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "contype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "condeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "condeferred", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "convalidated", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "conrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "contypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "conindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "confrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "confupdtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "confdeltype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 14, "name": "confmatchtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 15, "name": "conislocal", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "coninhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "connoinherit", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "conkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 19, "name": "confkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 20, "name": "conpfeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 21, "name": "conppeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "name": "conffeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "conexclop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 24, "name": "conbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "consrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "condef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "conparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967094, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967095 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967095, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967096 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "collname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "collowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "collencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "collcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "collctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "collprovider", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "collversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collisdeterministic", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967096, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967097 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "relname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "reltype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "reloftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "relowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "relam", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "relfilenode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "reltablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "relpages", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "reltuples", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "relallvisible", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "reltoastrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "relhasindex", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "relisshared", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "relpersistence", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "relistemp", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "relkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 19, "name": "relnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "relchecks", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 21, "name": "relhasoids", "nullable": true, "type": {"oid": 16}}, {"id": 22, "name": "relhaspkey", "nullable": true, "type": {"oid": 16}}, {"id": 23, "name": "relhasrules", "nullable": true, "type": {"oid": 16}}, {"id": 24, "name": "relhastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "relhassubclass", "nullable": true, "type": {"oid": 16}}, {"id": 26, "name": "relfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "relacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "reloptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "relforcerowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 30, "name": "relispartition", "nullable": true, "type": {"oid": 16}}, {"id": 31, "name": "relispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 32, "name": "relreplident", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 33, "name": "relrewrite", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 34, "name": "relrowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 35, "name": "relpartbound", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "relminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967097, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967098 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "castsource", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "casttarget", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "castfunc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "castcontext", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "castmethod", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967098, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967099 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "default_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967099, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967100 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "superuser", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "trusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "relocatable", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "requires", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 9, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967100, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967101 {"table": {"columns": [{"id": 1, "name": "roleid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "member", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grantor", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "admin_option", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967101, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967102 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967102, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967103 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967103, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967104 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "adrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "adnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "adbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "adsrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967104, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967105 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amprocfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amproclefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amprocrighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amprocnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amproc", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967105, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967106 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amopfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amoplefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amoprighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amopstrategy", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amoppurpose", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "amopopr", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "amopmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "amopsortfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967106, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967107 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "amstrategies", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "amsupport", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 5, "name": "amcanorder", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "amcanorderbyop", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "amcanbackward", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "amcanunique", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "amcanmulticol", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "amoptionalkey", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "amsearcharray", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "amsearchnulls", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "amstorage", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "amclusterable", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "ampredlocks", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "amkeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "aminsert", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "ambeginscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "amgettuple", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "amgetbitmap", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "amrescan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "amendscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 23, "name": "ammarkpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 24, "name": "amrestrpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 25, "name": "ambuild", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "ambuildempty", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 27, "name": "ambulkdelete", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 28, "name": "amvacuumcleanup", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "amcanreturn", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 30, "name": "amcostestimate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 31, "name": "amoptions", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 32, "name": "amhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 33, "name": "amtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967107, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967108 {"table": {"columns": [{"id": 1, "name": "aggfnoid", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 2, "name": "aggkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "aggnumdirectargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "aggtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "aggfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "aggcombinefn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "aggserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "aggdeserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 9, "name": "aggmtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "aggminvtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 11, "name": "aggmfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 12, "name": "aggfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "aggmfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "aggsortop", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "aggtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "aggtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "aggmtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "aggmtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "agginitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "aggminitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "aggfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 22, "name": "aggmfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967108, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967109, "version": "1"}} +4294967109 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967109, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967110 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "view_definition", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "check_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_trigger_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "is_trigger_deletable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_trigger_insertable_into", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967110, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967111 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967111, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967112 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967112, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967113 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967113, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967114 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967114, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967115 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967115, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967116 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967116, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967117 {"table": {"columns": [{"id": 1, "name": "user_defined_type_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "user_defined_type_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_defined_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "user_defined_type_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_instantiable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_final", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "ordering_form", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordering_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "ordering_routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "ordering_routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "ordering_routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "reference_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 28, "name": "source_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "ref_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967117, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967118 {"table": {"columns": [{"id": 1, "name": "attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967118, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967119 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967119, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967120 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967120, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967121 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "type_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "type_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967121, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967122 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_manipulation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "action_order", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "action_condition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "action_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "action_orientation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "action_timing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "action_reference_old_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "action_reference_new_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "action_reference_old_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "action_reference_new_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967122, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967123 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967123, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967124 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "transform_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967124, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967125 {"table": {"columns": [{"id": 1, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "nodegroup_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "tablespace_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tablespace_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967125, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967126 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967126, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967127 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967127, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967128 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967128, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967129 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967129, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967130 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967130, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967131 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_deferrable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "initially_deferred", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967131, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967132 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "non_unique", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "seq_in_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "COLLATION", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "storing", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "implicit", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "is_visible", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "visibility", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967132, "name": "statistics", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967133 {"table": {"columns": [{"id": 1, "name": "conversion_factor", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 2, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unit_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967133, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967134 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "organization", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "organization_coordsys_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967134, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967135 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "geometry_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967135, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967136 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967136, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967137 {"table": {"columns": [{"id": 1, "name": "sequence_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sequence_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sequence_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "numeric_precision", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "numeric_precision_radix", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "numeric_scale", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "start_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "minimum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "maximum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "increment", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "cycle_option", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967137, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967138 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967138, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967139 {"table": {"columns": [{"id": 1, "name": "catalog_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "default_character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "crdb_is_user_defined", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967139, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967140 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "options", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967140, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967141 {"table": {"columns": [{"id": 1, "name": "sizing_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "sizing_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "supported_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967141, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967142 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967142, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967143 {"table": {"columns": [{"id": 1, "name": "implementation_info_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implementation_info_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "integer_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "character_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967143, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967144 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sub_feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sub_feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967144, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967145 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 36, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 48, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 55, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 57, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 61, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 63, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 69, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 73, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 75, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 81, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 82, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967145, "name": "routines", "nextColumnId": 83, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n.nspname AS STRING) AS specific_schema, CAST(nameconcatoid(p.proname, p.oid) AS STRING) AS specific_name, CAST(current_database() AS STRING) AS routine_catalog, CAST(n.nspname AS STRING) AS routine_schema, CAST(p.proname AS STRING) AS routine_name, CAST(CASE p.prokind WHEN 'f' THEN 'FUNCTION' WHEN 'p' THEN 'PROCEDURE' END AS STRING) AS routine_type, CAST(NULL AS STRING) AS module_catalog, CAST(NULL AS STRING) AS module_schema, CAST(NULL AS STRING) AS module_name, CAST(NULL AS STRING) AS udt_catalog, CAST(NULL AS STRING) AS udt_schema, CAST(NULL AS STRING) AS udt_name, CAST(CASE WHEN p.prokind = 'p' THEN NULL WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(CASE WHEN nt.nspname IS NOT NULL THEN current_database() END AS STRING) AS type_udt_catalog, CAST(nt.nspname AS STRING) AS type_udt_schema, CAST(t.typname AS STRING) AS type_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(CASE WHEN p.prokind != 'p' THEN 0 END AS STRING) AS dtd_identifier, CAST(CASE WHEN l.lanname = 'sql' THEN 'SQL' ELSE 'EXTERNAL' END AS STRING) AS routine_body, CAST(CASE WHEN pg_has_role(p.proowner, 'USAGE') THEN p.prosrc ELSE NULL END AS STRING) AS routine_definition, CAST(CASE WHEN l.lanname = 'c' THEN p.prosrc ELSE NULL END AS STRING) AS external_name, CAST(upper(l.lanname) AS STRING) AS external_language, CAST('GENERAL' AS STRING) AS parameter_style, CAST(CASE WHEN p.provolatile = 'i' THEN 'YES' ELSE 'NO' END AS STRING) AS is_deterministic, CAST('MODIFIES' AS STRING) AS sql_data_access, CAST(CASE WHEN p.prokind != 'p' THEN CASE WHEN p.proisstrict THEN 'YES' ELSE 'NO' END END AS STRING) AS is_null_call, CAST(NULL AS STRING) AS sql_path, CAST('YES' AS STRING) AS schema_level_routine, CAST(0 AS INT8) AS max_dynamic_result_sets, CAST(NULL AS STRING) AS is_user_defined_cast, CAST(NULL AS STRING) AS is_implicitly_invocable, CAST(CASE WHEN p.prosecdef THEN 'DEFINER' ELSE 'INVOKER' END AS STRING) AS security_type, CAST(NULL AS STRING) AS to_sql_specific_catalog, CAST(NULL AS STRING) AS to_sql_specific_schema, CAST(NULL AS STRING) AS to_sql_specific_name, CAST('NO' AS STRING) AS as_locator, CAST(NULL AS TIMESTAMPTZ) AS created, CAST(NULL AS TIMESTAMPTZ) AS last_altered, CAST(NULL AS STRING) AS new_savepoint_level, CAST('NO' AS STRING) AS is_udt_dependent, CAST(NULL AS STRING) AS result_cast_from_data_type, CAST(NULL AS STRING) AS result_cast_as_locator, CAST(NULL AS INT8) AS result_cast_char_max_length, CAST(NULL AS INT8) AS result_cast_char_octet_length, CAST(NULL AS STRING) AS result_cast_char_set_catalog, CAST(NULL AS STRING) AS result_cast_char_set_schema, CAST(NULL AS STRING) AS result_cast_char_set_name, CAST(NULL AS STRING) AS result_cast_collation_catalog, CAST(NULL AS STRING) AS result_cast_collation_schema, CAST(NULL AS STRING) AS result_cast_collation_name, CAST(NULL AS INT8) AS result_cast_numeric_precision, CAST(NULL AS INT8) AS result_cast_numeric_precision_radix, CAST(NULL AS INT8) AS result_cast_numeric_scale, CAST(NULL AS INT8) AS result_cast_datetime_precision, CAST(NULL AS STRING) AS result_cast_interval_type, CAST(NULL AS INT8) AS result_cast_interval_precision, CAST(NULL AS STRING) AS result_cast_type_udt_catalog, CAST(NULL AS STRING) AS result_cast_type_udt_schema, CAST(NULL AS STRING) AS result_cast_type_udt_name, CAST(NULL AS STRING) AS result_cast_scope_catalog, CAST(NULL AS STRING) AS result_cast_scope_schema, CAST(NULL AS STRING) AS result_cast_scope_name, CAST(NULL AS INT8) AS result_cast_maximum_cardinality, CAST(NULL AS STRING) AS result_cast_dtd_identifier FROM (pg_catalog.pg_namespace AS n JOIN pg_catalog.pg_proc AS p ON n.oid = p.pronamespace JOIN pg_catalog.pg_language AS l ON p.prolang = l.oid) LEFT JOIN (pg_catalog.pg_type AS t JOIN pg_catalog.pg_namespace AS nt ON t.typnamespace = nt.oid) ON (p.prorettype = t.oid) AND (p.prokind != 'p') WHERE (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))"}} +4294967146 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967146, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967147 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967147, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967148 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967148, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967149 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967149, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967150 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967150, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967151 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967151, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967152 {"table": {"columns": [{"id": 1, "name": "resource_group_enabled", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "resource_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "resource_group_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "thread_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "vcpu_ids", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967152, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967153 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unique_constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "unique_constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "unique_constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "match_option", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "update_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "delete_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "referenced_table_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967153, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967154 {"table": {"columns": [{"id": 1, "name": "cpu_system", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 2, "name": "messages_sent", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "swaps", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "block_ops_in", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "block_ops_out", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "context_voluntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "cpu_user", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "query_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "source_function", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "context_involuntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "duration", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 12, "name": "page_faults_major", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "page_faults_minor", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "seq", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "source_file", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "messages_received", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "source_line", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967154, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967155 {"table": {"columns": [{"id": 1, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "time", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "db", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967155, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967156 {"table": {"columns": [{"id": 1, "name": "plugin_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "load_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "plugin_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "plugin_library_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "plugin_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "plugin_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "plugin_type_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "plugin_author", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "plugin_library", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "plugin_license", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "plugin_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967156, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967157 {"table": {"columns": [{"id": 1, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "subpartition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_rows", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "avg_row_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "check_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "create_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "index_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "nodegroup", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "partition_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "partition_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "checksum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "partition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "subpartition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "update_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "max_data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "partition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "subpartition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "subpartition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967157, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967158 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "parameter_mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_result", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "parameter_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "parameter_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967158, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n_nspname AS STRING) AS specific_schema, CAST(nameconcatoid(proname, p_oid) AS STRING) AS specific_name, CAST((ss.x).n AS INT8) AS ordinal_position, CAST(CASE WHEN proargmodes IS NULL THEN 'IN' WHEN proargmodes[(ss.x).n] = 'i' THEN 'IN' WHEN proargmodes[(ss.x).n] = 'o' THEN 'OUT' WHEN proargmodes[(ss.x).n] = 'b' THEN 'INOUT' WHEN proargmodes[(ss.x).n] = 'v' THEN 'IN' WHEN proargmodes[(ss.x).n] = 't' THEN 'OUT' END AS STRING) AS parameter_mode, CAST('NO' AS STRING) AS is_result, CAST('NO' AS STRING) AS as_locator, CAST(NULLIF(proargnames[(ss.x).n], '') AS STRING) AS parameter_name, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS udt_catalog, CAST(nt.nspname AS STRING) AS udt_schema, CAST(t.typname AS STRING) AS udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST((ss.x).n AS STRING) AS dtd_identifier, CAST(CASE WHEN pg_has_role(proowner, 'USAGE') THEN pg_get_function_arg_default(p_oid, (ss.x).n) ELSE NULL END AS STRING) AS parameter_default FROM pg_type AS t, pg_namespace AS nt, (SELECT n.nspname AS n_nspname, p.proname, p.oid AS p_oid, p.proowner, p.proargnames, p.proargmodes, information_schema._pg_expandarray(COALESCE(p.proallargtypes, p.proargtypes::OID[])) AS x FROM pg_namespace AS n, pg_proc AS p WHERE (n.oid = p.pronamespace) AND (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))) AS ss WHERE (t.oid = (ss.x).x) AND (t.typnamespace = nt.oid)"}} +4294967159 {"table": {"columns": [{"id": 1, "name": "insufficient_privileges", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "missing_bytes_beyond_max_mem_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967159, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967160 {"table": {"columns": [{"id": 1, "name": "word", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967160, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967161 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "position_in_unique_constraint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967161, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967162 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967162, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967163 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967163, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967164 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967164, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967165 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "foreign_server_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967165, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967166 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967166, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967167 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "library_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_data_wrapper_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967167, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967168 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967168, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967169 {"table": {"columns": [{"id": 1, "name": "last_update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "table_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "check_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "checksum", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "extra", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "file_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "table_name", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "avg_row_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "file_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "free_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 14, "name": "table_schema", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 15, "name": "update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 17, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "create_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 20, "name": "initial_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "update_count", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 25, "name": "creation_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 26, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "fulltext_keys", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 28, "name": "row_format", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "total_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "index_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 32, "name": "last_access_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 33, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "transaction_counter", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 35, "name": "file_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "logfile_group_number", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 37, "name": "recover_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 38, "name": "deleted_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967169, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967170 {"table": {"columns": [{"id": 1, "name": "definer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "event_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "event_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "interval_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "on_completion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "originator", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "collation_connection", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "database_collation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "event_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "event_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "execute_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "interval_field", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "starts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 15, "name": "time_zone", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "character_set_client", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "ends", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "event_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "event_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "last_executed", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 22, "name": "sql_mode", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967170, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967171 {"table": {"columns": [{"id": 1, "name": "support", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "transactions", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "xa", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "savepoints", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967171, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967172 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967172, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967173 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "collection_type_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967173, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967174 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967174, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967175 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967175, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967176 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_deferrable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "initially_deferred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967176, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967177 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967177, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967178 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967178, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967179 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967180 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_nullable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 34, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "is_self_referencing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "is_identity", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "identity_generation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "identity_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "identity_increment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "identity_maximum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "identity_minimum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "identity_cycle", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_generated", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "generation_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "is_updatable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "is_hidden", "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "crdb_sql_type", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967180, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967181 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967181, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967182 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967182, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967183 {"table": {"columns": [{"id": 1, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "histogram", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967183, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967184 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967184, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967185 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967185, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967186 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967186, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967187 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dependent_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967187, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967188 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "pad_attribute", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967188, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967189 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967189, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967190 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "check_clause", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967190, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967191 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967191, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967192 {"table": {"columns": [{"id": 1, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_repertoire", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "form_of_use", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "default_collate_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "default_collate_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "default_collate_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967192, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967193 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attribute_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "attribute_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_nullable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "attribute_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "attribute_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "attribute_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "is_derived_reference_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967193, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967194 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967194, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967195 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967195, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967196, "version": "1"}} +4294967196 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967196, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967197 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "ts", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 3, "name": "meta_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "meta", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "num_spans", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "spans", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 7, "name": "verified", "type": {"oid": 16}}, {"id": 8, "name": "target", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "decoded_meta", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "decoded_target", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "internal_meta", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "num_ranges", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_updated", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}], "formatVersion": 3, "id": 4294967197, "name": "kv_protected_ts_records", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967198 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "corruption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967198, "name": "kv_repairable_catalog_corruptions", "nextColumnId": 6, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH data AS (SELECT ns.\"parentID\" AS parent_id, ns.\"parentSchemaID\" AS parent_schema_id, ns.name, COALESCE(ns.id, d.id) AS id, d.descriptor, crdb_internal.descriptor_with_post_deserialization_changes(d.descriptor) AS updated_descriptor, crdb_internal.repaired_descriptor(d.descriptor, (SELECT array_agg(id) AS desc_id_array FROM system.descriptor), (SELECT array_agg(id) AS job_id_array FROM system.jobs WHERE status NOT IN ('failed', 'succeeded', 'canceled', 'revert-failed'))) AS repaired_descriptor FROM system.namespace AS ns FULL JOIN system.descriptor AS d ON ns.id = d.id), diag AS (SELECT *, CASE WHEN (descriptor IS NULL) AND (id != 29) THEN 'namespace' WHEN updated_descriptor != repaired_descriptor THEN 'descriptor' ELSE NULL END AS corruption FROM data) SELECT parent_id, parent_schema_id, name, id, corruption FROM diag WHERE corruption IS NOT NULL ORDER BY parent_id, parent_schema_id, name, id"}} +4294967199 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "log_term", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "log_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967199, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_token_deductions_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7], "storeColumnNames": ["tenant_id", "store_id", "priority", "log_term", "log_index", "tokens"], "version": 3}], "name": "kv_flow_token_deductions", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967200 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_tracked_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967200, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["range_id"], "name": "kv_flow_control_handles_range_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["tenant_id", "store_id", "total_tracked_tokens"], "version": 3}], "name": "kv_flow_control_handles", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967201 {"table": {"columns": [{"id": 1, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "available_regular_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "available_elastic_tokens", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967201, "name": "kv_flow_controller", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967202 {"table": {"columns": [{"id": 1, "name": "username", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privileges", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "grant_options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "user_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967202, "name": "kv_system_privileges", "nextColumnId": 6, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT * FROM system.privileges"}} +4294967203 {"table": {"columns": [{"id": 1, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "inheriting_member", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "member_is_explicit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "member_is_admin", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967203, "name": "kv_inherited_role_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967204 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "capability_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "capability_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967204, "name": "node_tenant_capabilities_cache", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967205 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "ttl", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967205, "name": "kv_dropped_relations", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH dropped_relations AS (SELECT id, ((descriptor->'table')->>'name') AS name, ((descriptor->'table')->'parentId')::INT8 AS parent_id, ((descriptor->'table')->'unexposedParentSchemaId')::INT8 AS parent_schema_id, to_timestamp((((descriptor->'table')->>'dropTime')::DECIMAL * 0.000000001)::FLOAT8) AS drop_time FROM crdb_internal.kv_catalog_descriptor WHERE ((descriptor->'table')->>'state') = 'DROP'), gc_ttl AS (SELECT id, ((config->'gc')->'ttlSeconds')::INT8 AS ttl FROM crdb_internal.kv_catalog_zones) SELECT dr.parent_id, dr.parent_schema_id, dr.name, dr.id, dr.drop_time, COALESCE(gc.ttl, db_gc.ttl, root_gc.ttl) * '1 second'::INTERVAL AS ttl FROM dropped_relations AS dr LEFT JOIN gc_ttl AS gc ON gc.id = dr.id LEFT JOIN gc_ttl AS db_gc ON db_gc.id = dr.parent_id LEFT JOIN gc_ttl AS root_gc ON root_gc.id = 0 ORDER BY parent_id, parent_schema_id, id"}} +4294967206 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "super_region_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967206, "name": "super_regions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967207 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967207, "name": "pg_catalog_table_is_implemented", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967208 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "total_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 3, "name": "total_read_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_read_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_write_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "total_write_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "total_sql_pod_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "total_pgwire_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "total_external_io_ingress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "total_external_io_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "total_kv_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "total_cross_region_network_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967208, "name": "tenant_usage_details", "nextColumnId": 13, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT tenant_id, (j->>'rU')::FLOAT8 AS total_ru, (j->>'readBytes')::INT8 AS total_read_bytes, (j->>'readRequests')::INT8 AS total_read_requests, (j->>'writeBytes')::INT8 AS total_write_bytes, (j->>'writeRequests')::INT8 AS total_write_requests, (j->>'sqlPodsCpuSeconds')::FLOAT8 AS total_sql_pod_seconds, (j->>'pgwireEgressBytes')::INT8 AS total_pgwire_egress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'kvRU')::FLOAT8 AS total_kv_ru, (j->>'crossRegionNetworkRU')::FLOAT8 AS total_cross_region_network_ru FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} +4294967209 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tags", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "startts", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "diff", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "created", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "range_end", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "resolved", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_event_utc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "catchup", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967209, "name": "active_range_feeds", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967210 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "for_all_roles", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "object_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967210, "name": "default_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967211 {"table": {"columns": [{"id": 1, "name": "region", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "zones", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967211, "name": "regions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967212 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "root_op_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace_str", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "jaeger_json", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967212, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["trace_id"], "name": "cluster_inflight_traces_trace_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["node_id", "root_op_name", "trace_str", "jaeger_json"], "version": 3}], "name": "cluster_inflight_traces", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967213 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967213, "name": "lost_descriptors_with_data", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967214 {"table": {"columns": [{"id": 1, "name": "object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "referenced_object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "referenced_object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "referenced_object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cross_database_reference_description", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967214, "name": "cross_db_references", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967215 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967215, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["database_name"], "name": "cluster_database_privileges_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["grantee", "privilege_type", "is_grantable"], "version": 3}], "name": "cluster_database_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967216 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "obj_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"hidden": true, "id": 6, "name": "error_redactable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967216, "name": "invalid_objects", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967217 {"table": {"columns": [{"id": 1, "name": "zone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "subzone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "target", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "range_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "raw_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "raw_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "raw_config_protobuf", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 13, "name": "full_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "full_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967217, "name": "zones", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967218 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967218, "name": "transaction_statistics_persisted_v22_2", "nextColumnId": 8, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics FROM system.transaction_statistics"}} +4294967219 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "service_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "contention_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "total_estimated_execution_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "p99_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967219, "name": "transaction_statistics_persisted", "nextColumnId": 14, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics, execution_count, service_latency, cpu_sql_nanos, contention_time, total_estimated_execution_time, p99_latency FROM system.transaction_statistics"}} +4294967220 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967220, "name": "transaction_statistics", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, max(metadata), crdb_internal.merge_transaction_stats(array_agg(statistics)), aggregation_interval FROM (SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, aggregation_interval FROM crdb_internal.cluster_transaction_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, agg_interval FROM system.transaction_statistics) GROUP BY aggregated_ts, fingerprint_id, app_name, aggregation_interval"}} +4294967221 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "execution_total_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "execution_total_cluster_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "contention_time_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "cpu_sql_avg_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "service_latency_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "service_latency_p99_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967221, "name": "transaction_activity", "nextColumnId": 15, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, agg_interval, metadata, statistics, query, execution_count, execution_total_seconds, execution_total_cluster_seconds, contention_time_avg_seconds, cpu_sql_avg_nanos, service_latency_avg_seconds, service_latency_p99_seconds FROM system.transaction_activity"}} +4294967222 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "key", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_ids", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "max_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "retry_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "retry_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "commit_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "commit_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967222, "name": "node_transaction_statistics", "nextColumnId": 55, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967223 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "estimated_row_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967223, "name": "table_row_statistics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967224 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "version", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "mod_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "mod_time_logical", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "format_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "sc_lease_node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "sc_lease_expiration_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "audit_mode", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "parent_schema_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "locality", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967224, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["parent_id"], "name": "tables_parent_id_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "name", "database_name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["database_name"], "name": "tables_database_name_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "parent_id", "name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}], "name": "tables", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967225 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967225, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "table_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["start_key", "end_key"], "version": 3}], "name": "table_spans", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967226 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_unique", "type": {"oid": 16}}, {"id": 7, "name": "is_inverted", "type": {"oid": 16}}, {"id": 8, "name": "is_sharded", "type": {"oid": 16}}, {"id": 9, "name": "is_visible", "type": {"oid": 16}}, {"id": 10, "name": "visibility", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967226, "name": "table_indexes", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967227 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "nullable", "type": {"oid": 16}}, {"id": 7, "name": "default_expr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967227, "name": "table_columns", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967228 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967228, "name": "statement_statistics_persisted_v22_2", "nextColumnId": 12, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations FROM system.statement_statistics"}} +4294967229 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 12, "name": "indexes_usage", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "service_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "contention_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "total_estimated_execution_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "p99_latency", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967229, "name": "statement_statistics_persisted", "nextColumnId": 19, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations, indexes_usage, execution_count, service_latency, cpu_sql_nanos, contention_time, total_estimated_execution_time, p99_latency FROM system.statement_statistics"}} +4294967230 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "sampled_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 10, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967230, "name": "statement_statistics", "nextColumnId": 11, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, max(metadata) AS metadata, crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)), max(sampled_plan), aggregation_interval, array_remove(array_cat_agg(index_recommendations), NULL) AS index_recommendations FROM (SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, sampled_plan, aggregation_interval, index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, plan, agg_interval, index_recommendations FROM system.statement_statistics) GROUP BY aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, aggregation_interval"}} +4294967231 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "execution_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "execution_total_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "execution_total_cluster_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "cpu_sql_avg_nanos", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "contention_time_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "service_latency_avg_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "service_latency_p99_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967231, "name": "statement_activity", "nextColumnId": 18, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, agg_interval, metadata, statistics, plan, index_recommendations, execution_count, execution_total_seconds, execution_total_cluster_seconds, contention_time_avg_seconds, cpu_sql_avg_nanos, service_latency_avg_seconds, service_latency_p99_seconds FROM system.statement_activity"}} +4294967232 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967232, "name": "session_variables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967233 {"table": {"columns": [{"id": 1, "name": "span_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "message_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "timestamp", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "loc", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "message", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "age", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967233, "name": "session_trace", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967234 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "target_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "target_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967234, "name": "schema_changes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967235 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "component", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "field", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967235, "name": "node_runtime_info", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967236 {"table": {"columns": [{"id": 1, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "start_pretty", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "end_key", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "end_pretty", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "replica_localities", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 9, "name": "non_voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 10, "name": "learner_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 11, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "lease_holder", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "range_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967236, "name": "ranges", "nextColumnId": 14, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT range_id, start_key, start_pretty, end_key, end_pretty, replicas, replica_localities, voting_replicas, non_voting_replicas, learner_replicas, split_enforced_until, crdb_internal.lease_holder(start_key) AS lease_holder, (((crdb_internal.range_stats(start_key)->>'key_bytes')::INT8 + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT8) + COALESCE((crdb_internal.range_stats(start_key)->>'range_key_bytes')::INT8, 0)) + COALESCE((crdb_internal.range_stats(start_key)->>'range_val_bytes')::INT8, 0) AS range_size FROM crdb_internal.ranges_no_leases"}} +4294967237 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "start_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "end_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "replica_localities", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 9, "name": "non_voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 10, "name": "learner_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 11, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967237, "name": "ranges_no_leases", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967238 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "parent_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "columns", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "column_names", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "list_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "range_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "zone_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "subzone_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967238, "name": "partitions", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967239 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "txn_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "txn_time_avg_sec", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "txn_time_var_sec", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "committed_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "implicit_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967239, "name": "node_txn_stats", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967240 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "flags", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "anonymized", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "first_attempt_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "max_retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "last_error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_error_code", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rows_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "rows_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "parse_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "parse_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "plan_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "plan_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "run_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "run_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "overhead_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "overhead_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "bytes_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "bytes_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "rows_written_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "rows_written_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 55, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 56, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 57, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 58, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 59, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 60, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 61, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 62, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 63, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 64, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 65, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 66, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 67, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 68, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 69, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 70, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 71, "name": "full_scan", "type": {"oid": 16}}, {"id": 72, "name": "sample_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 73, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 75, "name": "txn_fingerprint_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 77, "name": "latency_seconds_min", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 78, "name": "latency_seconds_max", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 79, "name": "latency_seconds_p50", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 80, "name": "latency_seconds_p90", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 81, "name": "latency_seconds_p99", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967240, "name": "node_statement_statistics", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967241 {"table": {"columns": [{"id": 1, "name": "level", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "used", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "reserved_used", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "reserved_reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967241, "name": "node_memory_monitors", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967242 {"table": {"columns": [{"id": 1, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967242, "name": "node_metrics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967243 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "active_queries", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "last_active_query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "num_txns_executed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "session_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "active_query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "kv_txn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_alloc_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "session_end", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967243, "name": "node_sessions", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967244 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "txn_string", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "num_stmts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "num_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "num_auto_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "last_auto_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "isolation_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "priority", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "quality_of_service", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967244, "name": "node_transactions", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967245 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967245, "name": "node_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967246 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 26, "name": "cpu_sql_nanos", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "error_code", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "last_error_redactable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967246, "name": "node_execution_insights", "nextColumnId": 29, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967247 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967247, "name": "node_distsql_flows", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967248 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967248, "name": "node_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967249 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "deleted", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967249, "name": "leases", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967250 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 4, "name": "capacity", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "available", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "used", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "logical_bytes", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lease_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "writes_per_second", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "bytes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "writes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 14, "name": "properties", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967250, "name": "kv_store_status", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967251 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "go_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "time", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "revision", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "cgo_compiler", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "platform", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "distribution", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "dependencies", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "updated_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 18, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 19, "name": "args", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 20, "name": "env", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 21, "name": "activity", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967251, "name": "kv_node_status", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967252 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967252, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "system_jobs_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "name": "system_jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["id", "status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["status"], "name": "system_jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["id", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}], "name": "system_jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967253 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967253, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["job_type"], "name": "jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "description", "statement", "user_name", "descriptor_ids", "status", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967254 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "goroutine_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "finished", "type": {"oid": 16}}, {"id": 6, "name": "start_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967254, "name": "node_inflight_trace_spans", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967255 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967255, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967256 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967256, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967257 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "column_direction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "implicit", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967257, "name": "index_columns", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967258 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "contending_pretty_key", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "waiting_stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "waiting_stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 11, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967258, "name": "transaction_contention_events", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967259 {"table": {"columns": [{"id": 1, "name": "source_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "target_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967259, "name": "gossip_network", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967260 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "decommissioning", "type": {"oid": 16}}, {"id": 6, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "updated_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967260, "name": "gossip_liveness", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967261 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "category", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "value", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967261, "name": "gossip_alerts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967262 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "advertise_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "sql_network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sql_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "advertise_sql_address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cluster_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "build_tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 14, "name": "is_live", "type": {"oid": 16}}, {"id": 15, "name": "ranges", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "leases", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967262, "name": "gossip_nodes", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967263 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "expiration", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "draining", "type": {"oid": 16}}, {"id": 5, "name": "membership", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967263, "name": "kv_node_liveness", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967264 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "dependedonby_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "dependedonby_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "dependedonby_index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "dependedonby_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "dependedonby_details", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967264, "name": "forward_dependencies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967265 {"table": {"columns": [{"id": 1, "name": "feature_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "usage_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967265, "name": "feature_usage", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967266 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "owner", "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "primary_region", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_region", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "survival_goal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "placement_policy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967266, "name": "databases", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967267 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "descriptor_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "enum_members", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967267, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["descriptor_id"], "name": "create_type_statements_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7], "storeColumnNames": ["database_id", "database_name", "schema_name", "descriptor_name", "create_statement", "enum_members"], "version": 3}], "name": "create_type_statements", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967268 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "descriptor_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "create_nofks", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "alter_statements", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "validate_statements", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 12, "name": "create_redactable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "has_partitions", "type": {"oid": 16}}, {"id": 14, "name": "is_multi_region", "type": {"oid": 16}}, {"id": 15, "name": "is_virtual", "type": {"oid": 16}}, {"id": 16, "name": "is_temporary", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967268, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["descriptor_id"], "name": "create_statements_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["database_id", "database_name", "schema_name", "descriptor_type", "descriptor_name", "create_statement", "state", "create_nofks", "alter_statements", "validate_statements", "create_redactable", "has_partitions", "is_multi_region", "is_virtual", "is_temporary"], "version": 3}], "name": "create_statements", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967269 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967269, "name": "create_schema_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967270 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "procedure_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "procedure_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967270, "name": "create_procedure_statements", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967271 {"table": {"columns": [{"id": 1, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "function_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "function_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "create_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967271, "name": "create_function_statements", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967272 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "metadata", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "statistics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "aggregation_interval", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967272, "name": "cluster_transaction_statistics", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967273 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "metadata", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "sampled_plan", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "aggregation_interval", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 10, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967273, "name": "cluster_statement_statistics", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -594,316 +595,317 @@ SELECT FROM "".crdb_internal.kv_catalog_comments ORDER BY 1,2,3,4 ---- -4294967069 107 0 "this is the public schema" -4294967069 108 0 "this is a schema" -4294967092 104 0 "this is the test database" -4294967095 3328135398 0 "this is a primary key constraint" -4294967095 3441531627 0 "this is a check constraint" -4294967098 111 0 "this is a table" -4294967098 5181039 0 "this is an index" -4294967098 4294966977 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." -4294967098 4294966978 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." -4294967098 4294966979 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." -4294967098 4294966981 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" -4294967098 4294966982 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" -4294967098 4294966983 0 "pg_user_mappings was created for compatibility and is currently unimplemented" -4294967098 4294966984 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" -4294967098 4294966985 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" -4294967098 4294966986 0 "pg_ts_template was created for compatibility and is currently unimplemented" -4294967098 4294966987 0 "pg_ts_parser was created for compatibility and is currently unimplemented" -4294967098 4294966988 0 "pg_ts_dict was created for compatibility and is currently unimplemented" -4294967098 4294966989 0 "pg_ts_config was created for compatibility and is currently unimplemented" -4294967098 4294966990 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" -4294967098 4294966991 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" -4294967098 4294966992 0 "pg_transform was created for compatibility and is currently unimplemented" -4294967098 4294966993 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" -4294967098 4294966994 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" -4294967098 4294966995 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" -4294967098 4294966996 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" -4294967098 4294966997 0 "pg_subscription was created for compatibility and is currently unimplemented" -4294967098 4294966998 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" -4294967098 4294966999 0 "pg_stats was created for compatibility and is currently unimplemented" -4294967098 4294967000 0 "pg_stats_ext was created for compatibility and is currently unimplemented" -4294967098 4294967001 0 "pg_statistic was created for compatibility and is currently unimplemented" -4294967098 4294967002 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" -4294967098 4294967003 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" -4294967098 4294967004 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" -4294967098 4294967005 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" -4294967098 4294967006 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967007 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" -4294967098 4294967008 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" -4294967098 4294967009 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967010 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" -4294967098 4294967011 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" -4294967098 4294967012 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967013 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" -4294967098 4294967014 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" -4294967098 4294967015 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" -4294967098 4294967016 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" -4294967098 4294967017 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" -4294967098 4294967018 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" -4294967098 4294967019 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967020 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" -4294967098 4294967021 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" -4294967098 4294967022 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967023 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" -4294967098 4294967024 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" -4294967098 4294967025 0 "pg_stat_slru was created for compatibility and is currently unimplemented" -4294967098 4294967026 0 "pg_stat_replication was created for compatibility and is currently unimplemented" -4294967098 4294967027 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" -4294967098 4294967028 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" -4294967098 4294967029 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" -4294967098 4294967030 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" -4294967098 4294967031 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" -4294967098 4294967032 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" -4294967098 4294967033 0 "pg_stat_database was created for compatibility and is currently unimplemented" -4294967098 4294967034 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" -4294967098 4294967035 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" -4294967098 4294967036 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" -4294967098 4294967037 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" -4294967098 4294967038 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" -4294967098 4294967039 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" -4294967098 4294967040 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" -4294967098 4294967041 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" -4294967098 4294967042 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" -4294967098 4294967043 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" -4294967098 4294967044 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" -4294967098 4294967045 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" -4294967098 4294967046 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" -4294967098 4294967047 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" -4294967098 4294967048 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" -4294967098 4294967049 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" -4294967098 4294967050 0 "pg_rules was created for compatibility and is currently unimplemented" -4294967098 4294967051 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" -4294967098 4294967052 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" -4294967098 4294967053 0 "pg_replication_slots was created for compatibility and is currently unimplemented" -4294967098 4294967054 0 "pg_replication_origin was created for compatibility and is currently unimplemented" -4294967098 4294967055 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" -4294967098 4294967056 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" -4294967098 4294967057 0 "pg_publication_tables was created for compatibility and is currently unimplemented" -4294967098 4294967058 0 "pg_publication was created for compatibility and is currently unimplemented" -4294967098 4294967059 0 "pg_publication_rel was created for compatibility and is currently unimplemented" -4294967098 4294967060 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" -4294967098 4294967061 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" -4294967098 4294967062 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" -4294967098 4294967063 0 "pg_policy was created for compatibility and is currently unimplemented" -4294967098 4294967064 0 "pg_policies was created for compatibility and is currently unimplemented" -4294967098 4294967065 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" -4294967098 4294967066 0 "pg_opfamily was created for compatibility and is currently unimplemented" -4294967098 4294967067 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" -4294967098 4294967068 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" -4294967098 4294967069 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" -4294967098 4294967070 0 "available materialized views\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" -4294967098 4294967071 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" -4294967098 4294967072 0 "pg_largeobject was created for compatibility and is currently unimplemented" -4294967098 4294967073 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" -4294967098 4294967074 0 "available languages\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" -4294967098 4294967075 0 "pg_init_privs was created for compatibility and is currently unimplemented" -4294967098 4294967076 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" -4294967098 4294967077 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" -4294967098 4294967078 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" -4294967098 4294967079 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" -4294967098 4294967080 0 "pg_group was created for compatibility and is currently unimplemented" -4294967098 4294967081 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" -4294967098 4294967082 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" -4294967098 4294967083 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" -4294967098 4294967084 0 "pg_file_settings was created for compatibility and is currently unimplemented" -4294967098 4294967085 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" -4294967098 4294967086 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" -4294967098 4294967087 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" -4294967098 4294967088 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" -4294967098 4294967089 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" -4294967098 4294967090 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" -4294967098 4294967091 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" -4294967098 4294967092 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" -4294967098 4294967093 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" -4294967098 4294967094 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" -4294967098 4294967095 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" -4294967098 4294967096 0 "pg_config was created for compatibility and is currently unimplemented" -4294967098 4294967097 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" -4294967098 4294967098 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" -4294967098 4294967099 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" -4294967098 4294967100 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" -4294967098 4294967101 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" -4294967098 4294967102 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" -4294967098 4294967103 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" -4294967098 4294967104 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" -4294967098 4294967105 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" -4294967098 4294967106 0 "pg_amproc was created for compatibility and is currently unimplemented" -4294967098 4294967107 0 "pg_amop was created for compatibility and is currently unimplemented" -4294967098 4294967108 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" -4294967098 4294967109 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" -4294967098 4294967111 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" -4294967098 4294967112 0 "view_table_usage was created for compatibility and is currently unimplemented" -4294967098 4294967113 0 "view_routine_usage was created for compatibility and is currently unimplemented" -4294967098 4294967114 0 "view_column_usage was created for compatibility and is currently unimplemented" -4294967098 4294967115 0 "grantable privileges (incomplete)" -4294967098 4294967116 0 "user_mappings was created for compatibility and is currently unimplemented" -4294967098 4294967117 0 "user_mapping_options was created for compatibility and is currently unimplemented" -4294967098 4294967118 0 "user_defined_types was created for compatibility and is currently unimplemented" -4294967098 4294967119 0 "user_attributes was created for compatibility and is currently unimplemented" -4294967098 4294967120 0 "usage_privileges was created for compatibility and is currently unimplemented" -4294967098 4294967121 0 "udt_privileges was created for compatibility and is currently unimplemented" -4294967098 4294967122 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" -4294967098 4294967123 0 "triggers was created for compatibility and is currently unimplemented" -4294967098 4294967124 0 "triggered_update_columns was created for compatibility and is currently unimplemented" -4294967098 4294967125 0 "transforms was created for compatibility and is currently unimplemented" -4294967098 4294967126 0 "tablespaces was created for compatibility and is currently unimplemented" -4294967098 4294967127 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" -4294967098 4294967128 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" -4294967098 4294967129 0 "tables_extensions was created for compatibility and is currently unimplemented" -4294967098 4294967130 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" -4294967098 4294967131 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" -4294967098 4294967132 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" -4294967098 4294967133 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" -4294967098 4294967134 0 "st_units_of_measure was created for compatibility and is currently unimplemented" -4294967098 4294967135 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" -4294967098 4294967136 0 "st_geometry_columns was created for compatibility and is currently unimplemented" -4294967098 4294967137 0 "exposes the session variables." -4294967098 4294967138 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" -4294967098 4294967139 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" -4294967098 4294967140 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" -4294967098 4294967141 0 "schemata_extensions was created for compatibility and is currently unimplemented" -4294967098 4294967142 0 "sql_sizing was created for compatibility and is currently unimplemented" -4294967098 4294967143 0 "sql_parts was created for compatibility and is currently unimplemented" -4294967098 4294967144 0 "sql_implementation_info was created for compatibility and is currently unimplemented" -4294967098 4294967145 0 "sql_features was created for compatibility and is currently unimplemented" -4294967098 4294967146 0 "built-in functions and user-defined functions\nhttps://www.postgresql.org/docs/15/infoschema-routines.html" -4294967098 4294967147 0 "routine_privileges was created for compatibility and is currently unimplemented" -4294967098 4294967148 0 "role_usage_grants was created for compatibility and is currently unimplemented" -4294967098 4294967149 0 "role_udt_grants was created for compatibility and is currently unimplemented" -4294967098 4294967150 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" -4294967098 4294967151 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" -4294967098 4294967152 0 "role_column_grants was created for compatibility and is currently unimplemented" -4294967098 4294967153 0 "resource_groups was created for compatibility and is currently unimplemented" -4294967098 4294967154 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" -4294967098 4294967155 0 "profiling was created for compatibility and is currently unimplemented" -4294967098 4294967156 0 "processlist was created for compatibility and is currently unimplemented" -4294967098 4294967157 0 "plugins was created for compatibility and is currently unimplemented" -4294967098 4294967158 0 "partitions was created for compatibility and is currently unimplemented" -4294967098 4294967159 0 "function parameters\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" -4294967098 4294967160 0 "optimizer_trace was created for compatibility and is currently unimplemented" -4294967098 4294967161 0 "keywords was created for compatibility and is currently unimplemented" -4294967098 4294967162 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" -4294967098 4294967163 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" -4294967098 4294967164 0 "foreign_tables was created for compatibility and is currently unimplemented" -4294967098 4294967165 0 "foreign_table_options was created for compatibility and is currently unimplemented" -4294967098 4294967166 0 "foreign_servers was created for compatibility and is currently unimplemented" -4294967098 4294967167 0 "foreign_server_options was created for compatibility and is currently unimplemented" -4294967098 4294967168 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" -4294967098 4294967169 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" -4294967098 4294967170 0 "files was created for compatibility and is currently unimplemented" -4294967098 4294967171 0 "events was created for compatibility and is currently unimplemented" -4294967098 4294967172 0 "engines was created for compatibility and is currently unimplemented" -4294967098 4294967173 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" -4294967098 4294967174 0 "element_types was created for compatibility and is currently unimplemented" -4294967098 4294967175 0 "domains was created for compatibility and is currently unimplemented" -4294967098 4294967176 0 "domain_udt_usage was created for compatibility and is currently unimplemented" -4294967098 4294967177 0 "domain_constraints was created for compatibility and is currently unimplemented" -4294967098 4294967178 0 "data_type_privileges was created for compatibility and is currently unimplemented" -4294967098 4294967179 0 "constraint_table_usage was created for compatibility and is currently unimplemented" -4294967098 4294967180 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" -4294967098 4294967181 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" -4294967098 4294967182 0 "columns_extensions was created for compatibility and is currently unimplemented" -4294967098 4294967183 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" -4294967098 4294967184 0 "column_statistics was created for compatibility and is currently unimplemented" -4294967098 4294967185 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" -4294967098 4294967186 0 "column_options was created for compatibility and is currently unimplemented" -4294967098 4294967187 0 "column_domain_usage was created for compatibility and is currently unimplemented" -4294967098 4294967188 0 "column_column_usage was created for compatibility and is currently unimplemented" -4294967098 4294967189 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" -4294967098 4294967190 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" -4294967098 4294967191 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" -4294967098 4294967192 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" -4294967098 4294967193 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" -4294967098 4294967194 0 "attributes was created for compatibility and is currently unimplemented" -4294967098 4294967195 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" -4294967098 4294967196 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" -4294967098 4294967198 0 "decoded protected timestamp metadata from system.protected_ts_records (KV scan). does not decode" -4294967098 4294967199 0 "known corruptions in the catalog which can be repaired using builtin functions like crdb_internal.repaired_descriptor" -4294967098 4294967200 0 "node-level view of tracked kv flow tokens" -4294967098 4294967201 0 "node-level view of active kv flow control handles, their underlying streams, and tracked state" -4294967098 4294967202 0 "node-level view of the kv flow controller, its active streams and available tokens state" -4294967098 4294967203 0 "this vtable is a view on system.privileges with the root user and is used to back SHOW SYSTEM GRANTS" -4294967098 4294967204 0 "table listing transitive closure of system.role_members" -4294967098 4294967205 0 "eventually consistent in-memory tenant capability cache for this node" -4294967098 4294967206 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" -4294967098 4294967207 0 "list super regions of databases visible to the current user" -4294967098 4294967208 0 "which entries of pg_catalog are implemented in this version of CockroachDB" -4294967098 4294967210 0 "node-level table listing all currently running range feeds" -4294967098 4294967211 0 "virtual table with default privileges" -4294967098 4294967212 0 "available regions for the cluster" -4294967098 4294967213 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" -4294967098 4294967214 0 "virtual table with table descriptors that still have data" -4294967098 4294967215 0 "virtual table with cross db references" -4294967098 4294967216 0 "virtual table with database privileges" -4294967098 4294967217 0 "virtual table to validate descriptors" -4294967098 4294967218 0 "decoded zone configurations from system.zones (KV scan)" -4294967098 4294967223 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)." -4294967098 4294967224 0 "stats for all tables accessible by current user in current database as of 10s ago" -4294967098 4294967225 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" -4294967098 4294967226 0 "key spans per SQL object" -4294967098 4294967227 0 "indexes accessible by current user in current database (KV scan)" -4294967098 4294967228 0 "details for all columns accessible by current user in current database (KV scan)" -4294967098 4294967233 0 "session variables (RAM)" -4294967098 4294967234 0 "session trace accumulated so far (RAM)" -4294967098 4294967235 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" -4294967098 4294967236 0 "server parameters, useful to construct connection URLs (RAM, local node only)" -4294967098 4294967237 0 "ranges is a view which queries ranges_no_leases for system ranges" -4294967098 4294967238 0 "range metadata without leaseholder details (KV join; expensive!)" -4294967098 4294967239 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" -4294967098 4294967240 0 "per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours)" -4294967098 4294967241 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)." -4294967098 4294967242 0 "node-level table listing all currently active memory monitors" -4294967098 4294967243 0 "current values for metrics (RAM; local node only)" -4294967098 4294967244 0 "running sessions visible by current user (RAM; local node only)" -4294967098 4294967245 0 "running user transactions visible by the current user (RAM; local node only)" -4294967098 4294967246 0 "running queries visible by current user (RAM; local node only)" -4294967098 4294967247 0 "Node statement execution insights" -4294967098 4294967248 0 "DistSQL remote flows information (RAM; local node only)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on this node. The local flows (those that are\nrunning on the same node as the query originated on) are not included." -4294967098 4294967249 0 "contention information (RAM; local node only)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." -4294967098 4294967250 0 "acquired table leases (RAM; local node only)" -4294967098 4294967251 0 "store details and status (cluster RPC; expensive!)" -4294967098 4294967252 0 "node details across the entire cluster (cluster RPC; expensive!)" -4294967098 4294967253 0 "wrapper over system.jobs with row access control (KV scan)" -4294967098 4294967254 0 "decoded job metadata from crdb_internal.system_jobs (KV scan)" -4294967098 4294967255 0 "in-flight spans (RAM; local node only)" -4294967098 4294967256 0 "cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout." -4294967098 4294967257 0 "key spans per table index" -4294967098 4294967258 0 "index columns for all indexes accessible by current user in current database (KV scan)" -4294967098 4294967259 0 "cluster-wide transaction contention events. Querying this table is an\n\t\texpensive operation since it creates a cluster-wide RPC-fanout." -4294967098 4294967260 0 "locally known edges in the gossip network (RAM; local node only)" -4294967098 4294967261 0 "locally known gossiped node liveness (RAM; local node only)" -4294967098 4294967262 0 "locally known gossiped health alerts (RAM; local node only)" -4294967098 4294967263 0 "locally known gossiped node details (RAM; local node only)" -4294967098 4294967264 0 "node liveness status, as seen by kv" -4294967098 4294967265 0 "forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)" -4294967098 4294967266 0 "telemetry counters (RAM; local node only)" -4294967098 4294967267 0 "databases accessible by the current user (KV scan)" -4294967098 4294967268 0 "CREATE statements for all user defined types accessible by the current user in current database (KV scan)" -4294967098 4294967269 0 "CREATE and ALTER statements for all tables accessible by current user in current database (KV scan)" -4294967098 4294967270 0 "CREATE statements for all user defined schemas accessible by the current user in current database (KV scan)" -4294967098 4294967271 0 "CREATE statements for all user-defined functions" -4294967098 4294967272 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." -4294967098 4294967273 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." -4294967098 4294967274 0 "cluster settings (RAM)" -4294967098 4294967275 0 "running sessions visible to current user (cluster RPC; expensive!)" -4294967098 4294967276 0 "running user transactions visible by the current user (cluster RPC; expensive!)" -4294967098 4294967277 0 "running queries visible by current user (cluster RPC; expensive!)" -4294967098 4294967278 0 "cluster-wide locks held in lock tables. Querying this table is an\n\t\texpensive operation since it creates a cluster-wide RPC-fanout." -4294967098 4294967279 0 "Node transaction execution insights" -4294967098 4294967280 0 "Cluster transaction execution insights" -4294967098 4294967281 0 "Cluster-wide statement execution insights" -4294967098 4294967282 0 "DistSQL remote flows information (cluster RPC; expensive!)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on any node in the cluster. The local flows (those that are\nrunning on the same node as the query originated on) are not included." -4294967098 4294967283 0 "contention information (cluster RPC; expensive!)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." -4294967098 4294967287 0 "like system.zones but overlaid with in-txn in-memory changes" -4294967098 4294967288 0 "like system.namespace but overlaid with in-txn in-memory changes" -4294967098 4294967289 0 "like system.descriptor but overlaid with in-txn in-memory changes and including virtual objects" -4294967098 4294967290 0 "like system.comments but overlaid with in-txn in-memory changes and including virtual objects" -4294967098 4294967291 0 "built-in functions (RAM/static)" -4294967098 4294967292 0 "built-in functions (RAM/static)" -4294967098 4294967293 0 "detailed identification strings (RAM, local node only)" -4294967098 4294967294 0 "backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)" +4294967068 107 0 "this is the public schema" +4294967068 108 0 "this is a schema" +4294967091 104 0 "this is the test database" +4294967094 3328135398 0 "this is a primary key constraint" +4294967094 3441531627 0 "this is a check constraint" +4294967097 111 0 "this is a table" +4294967097 5181039 0 "this is an index" +4294967097 4294966976 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." +4294967097 4294966977 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." +4294967097 4294966978 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." +4294967097 4294966980 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" +4294967097 4294966981 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" +4294967097 4294966982 0 "pg_user_mappings was created for compatibility and is currently unimplemented" +4294967097 4294966983 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" +4294967097 4294966984 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" +4294967097 4294966985 0 "pg_ts_template was created for compatibility and is currently unimplemented" +4294967097 4294966986 0 "pg_ts_parser was created for compatibility and is currently unimplemented" +4294967097 4294966987 0 "pg_ts_dict was created for compatibility and is currently unimplemented" +4294967097 4294966988 0 "pg_ts_config was created for compatibility and is currently unimplemented" +4294967097 4294966989 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" +4294967097 4294966990 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" +4294967097 4294966991 0 "pg_transform was created for compatibility and is currently unimplemented" +4294967097 4294966992 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" +4294967097 4294966993 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" +4294967097 4294966994 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" +4294967097 4294966995 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" +4294967097 4294966996 0 "pg_subscription was created for compatibility and is currently unimplemented" +4294967097 4294966997 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" +4294967097 4294966998 0 "pg_stats was created for compatibility and is currently unimplemented" +4294967097 4294966999 0 "pg_stats_ext was created for compatibility and is currently unimplemented" +4294967097 4294967000 0 "pg_statistic was created for compatibility and is currently unimplemented" +4294967097 4294967001 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" +4294967097 4294967002 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" +4294967097 4294967003 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" +4294967097 4294967004 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" +4294967097 4294967005 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967006 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" +4294967097 4294967007 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" +4294967097 4294967008 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967009 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" +4294967097 4294967010 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" +4294967097 4294967011 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967012 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" +4294967097 4294967013 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" +4294967097 4294967014 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" +4294967097 4294967015 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" +4294967097 4294967016 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" +4294967097 4294967017 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" +4294967097 4294967018 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967019 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" +4294967097 4294967020 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" +4294967097 4294967021 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967022 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" +4294967097 4294967023 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" +4294967097 4294967024 0 "pg_stat_slru was created for compatibility and is currently unimplemented" +4294967097 4294967025 0 "pg_stat_replication was created for compatibility and is currently unimplemented" +4294967097 4294967026 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" +4294967097 4294967027 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" +4294967097 4294967028 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" +4294967097 4294967029 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" +4294967097 4294967030 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" +4294967097 4294967031 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" +4294967097 4294967032 0 "pg_stat_database was created for compatibility and is currently unimplemented" +4294967097 4294967033 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" +4294967097 4294967034 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" +4294967097 4294967035 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" +4294967097 4294967036 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" +4294967097 4294967037 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" +4294967097 4294967038 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" +4294967097 4294967039 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" +4294967097 4294967040 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" +4294967097 4294967041 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" +4294967097 4294967042 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" +4294967097 4294967043 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" +4294967097 4294967044 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" +4294967097 4294967045 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" +4294967097 4294967046 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" +4294967097 4294967047 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" +4294967097 4294967048 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" +4294967097 4294967049 0 "pg_rules was created for compatibility and is currently unimplemented" +4294967097 4294967050 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" +4294967097 4294967051 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" +4294967097 4294967052 0 "pg_replication_slots was created for compatibility and is currently unimplemented" +4294967097 4294967053 0 "pg_replication_origin was created for compatibility and is currently unimplemented" +4294967097 4294967054 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" +4294967097 4294967055 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" +4294967097 4294967056 0 "pg_publication_tables was created for compatibility and is currently unimplemented" +4294967097 4294967057 0 "pg_publication was created for compatibility and is currently unimplemented" +4294967097 4294967058 0 "pg_publication_rel was created for compatibility and is currently unimplemented" +4294967097 4294967059 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" +4294967097 4294967060 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" +4294967097 4294967061 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" +4294967097 4294967062 0 "pg_policy was created for compatibility and is currently unimplemented" +4294967097 4294967063 0 "pg_policies was created for compatibility and is currently unimplemented" +4294967097 4294967064 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" +4294967097 4294967065 0 "pg_opfamily was created for compatibility and is currently unimplemented" +4294967097 4294967066 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" +4294967097 4294967067 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" +4294967097 4294967068 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" +4294967097 4294967069 0 "available materialized views\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" +4294967097 4294967070 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" +4294967097 4294967071 0 "pg_largeobject was created for compatibility and is currently unimplemented" +4294967097 4294967072 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" +4294967097 4294967073 0 "available languages\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" +4294967097 4294967074 0 "pg_init_privs was created for compatibility and is currently unimplemented" +4294967097 4294967075 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" +4294967097 4294967076 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" +4294967097 4294967077 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" +4294967097 4294967078 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" +4294967097 4294967079 0 "pg_group was created for compatibility and is currently unimplemented" +4294967097 4294967080 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" +4294967097 4294967081 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" +4294967097 4294967082 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" +4294967097 4294967083 0 "pg_file_settings was created for compatibility and is currently unimplemented" +4294967097 4294967084 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" +4294967097 4294967085 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" +4294967097 4294967086 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" +4294967097 4294967087 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" +4294967097 4294967088 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" +4294967097 4294967089 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" +4294967097 4294967090 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" +4294967097 4294967091 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" +4294967097 4294967092 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" +4294967097 4294967093 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" +4294967097 4294967094 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" +4294967097 4294967095 0 "pg_config was created for compatibility and is currently unimplemented" +4294967097 4294967096 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" +4294967097 4294967097 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" +4294967097 4294967098 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" +4294967097 4294967099 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" +4294967097 4294967100 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" +4294967097 4294967101 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" +4294967097 4294967102 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" +4294967097 4294967103 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" +4294967097 4294967104 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" +4294967097 4294967105 0 "pg_amproc was created for compatibility and is currently unimplemented" +4294967097 4294967106 0 "pg_amop was created for compatibility and is currently unimplemented" +4294967097 4294967107 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" +4294967097 4294967108 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" +4294967097 4294967110 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" +4294967097 4294967111 0 "view_table_usage was created for compatibility and is currently unimplemented" +4294967097 4294967112 0 "view_routine_usage was created for compatibility and is currently unimplemented" +4294967097 4294967113 0 "view_column_usage was created for compatibility and is currently unimplemented" +4294967097 4294967114 0 "grantable privileges (incomplete)" +4294967097 4294967115 0 "user_mappings was created for compatibility and is currently unimplemented" +4294967097 4294967116 0 "user_mapping_options was created for compatibility and is currently unimplemented" +4294967097 4294967117 0 "user_defined_types was created for compatibility and is currently unimplemented" +4294967097 4294967118 0 "user_attributes was created for compatibility and is currently unimplemented" +4294967097 4294967119 0 "usage_privileges was created for compatibility and is currently unimplemented" +4294967097 4294967120 0 "udt_privileges was created for compatibility and is currently unimplemented" +4294967097 4294967121 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" +4294967097 4294967122 0 "triggers was created for compatibility and is currently unimplemented" +4294967097 4294967123 0 "triggered_update_columns was created for compatibility and is currently unimplemented" +4294967097 4294967124 0 "transforms was created for compatibility and is currently unimplemented" +4294967097 4294967125 0 "tablespaces was created for compatibility and is currently unimplemented" +4294967097 4294967126 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" +4294967097 4294967127 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" +4294967097 4294967128 0 "tables_extensions was created for compatibility and is currently unimplemented" +4294967097 4294967129 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" +4294967097 4294967130 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" +4294967097 4294967131 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" +4294967097 4294967132 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" +4294967097 4294967133 0 "st_units_of_measure was created for compatibility and is currently unimplemented" +4294967097 4294967134 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" +4294967097 4294967135 0 "st_geometry_columns was created for compatibility and is currently unimplemented" +4294967097 4294967136 0 "exposes the session variables." +4294967097 4294967137 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" +4294967097 4294967138 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" +4294967097 4294967139 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" +4294967097 4294967140 0 "schemata_extensions was created for compatibility and is currently unimplemented" +4294967097 4294967141 0 "sql_sizing was created for compatibility and is currently unimplemented" +4294967097 4294967142 0 "sql_parts was created for compatibility and is currently unimplemented" +4294967097 4294967143 0 "sql_implementation_info was created for compatibility and is currently unimplemented" +4294967097 4294967144 0 "sql_features was created for compatibility and is currently unimplemented" +4294967097 4294967145 0 "built-in functions and user-defined functions\nhttps://www.postgresql.org/docs/15/infoschema-routines.html" +4294967097 4294967146 0 "routine_privileges was created for compatibility and is currently unimplemented" +4294967097 4294967147 0 "role_usage_grants was created for compatibility and is currently unimplemented" +4294967097 4294967148 0 "role_udt_grants was created for compatibility and is currently unimplemented" +4294967097 4294967149 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" +4294967097 4294967150 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" +4294967097 4294967151 0 "role_column_grants was created for compatibility and is currently unimplemented" +4294967097 4294967152 0 "resource_groups was created for compatibility and is currently unimplemented" +4294967097 4294967153 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" +4294967097 4294967154 0 "profiling was created for compatibility and is currently unimplemented" +4294967097 4294967155 0 "processlist was created for compatibility and is currently unimplemented" +4294967097 4294967156 0 "plugins was created for compatibility and is currently unimplemented" +4294967097 4294967157 0 "partitions was created for compatibility and is currently unimplemented" +4294967097 4294967158 0 "function parameters\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" +4294967097 4294967159 0 "optimizer_trace was created for compatibility and is currently unimplemented" +4294967097 4294967160 0 "keywords was created for compatibility and is currently unimplemented" +4294967097 4294967161 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" +4294967097 4294967162 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" +4294967097 4294967163 0 "foreign_tables was created for compatibility and is currently unimplemented" +4294967097 4294967164 0 "foreign_table_options was created for compatibility and is currently unimplemented" +4294967097 4294967165 0 "foreign_servers was created for compatibility and is currently unimplemented" +4294967097 4294967166 0 "foreign_server_options was created for compatibility and is currently unimplemented" +4294967097 4294967167 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" +4294967097 4294967168 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" +4294967097 4294967169 0 "files was created for compatibility and is currently unimplemented" +4294967097 4294967170 0 "events was created for compatibility and is currently unimplemented" +4294967097 4294967171 0 "engines was created for compatibility and is currently unimplemented" +4294967097 4294967172 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" +4294967097 4294967173 0 "element_types was created for compatibility and is currently unimplemented" +4294967097 4294967174 0 "domains was created for compatibility and is currently unimplemented" +4294967097 4294967175 0 "domain_udt_usage was created for compatibility and is currently unimplemented" +4294967097 4294967176 0 "domain_constraints was created for compatibility and is currently unimplemented" +4294967097 4294967177 0 "data_type_privileges was created for compatibility and is currently unimplemented" +4294967097 4294967178 0 "constraint_table_usage was created for compatibility and is currently unimplemented" +4294967097 4294967179 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" +4294967097 4294967180 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" +4294967097 4294967181 0 "columns_extensions was created for compatibility and is currently unimplemented" +4294967097 4294967182 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" +4294967097 4294967183 0 "column_statistics was created for compatibility and is currently unimplemented" +4294967097 4294967184 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" +4294967097 4294967185 0 "column_options was created for compatibility and is currently unimplemented" +4294967097 4294967186 0 "column_domain_usage was created for compatibility and is currently unimplemented" +4294967097 4294967187 0 "column_column_usage was created for compatibility and is currently unimplemented" +4294967097 4294967188 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" +4294967097 4294967189 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" +4294967097 4294967190 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" +4294967097 4294967191 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" +4294967097 4294967192 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" +4294967097 4294967193 0 "attributes was created for compatibility and is currently unimplemented" +4294967097 4294967194 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" +4294967097 4294967195 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" +4294967097 4294967197 0 "decoded protected timestamp metadata from system.protected_ts_records (KV scan). does not decode" +4294967097 4294967198 0 "known corruptions in the catalog which can be repaired using builtin functions like crdb_internal.repaired_descriptor" +4294967097 4294967199 0 "node-level view of tracked kv flow tokens" +4294967097 4294967200 0 "node-level view of active kv flow control handles, their underlying streams, and tracked state" +4294967097 4294967201 0 "node-level view of the kv flow controller, its active streams and available tokens state" +4294967097 4294967202 0 "this vtable is a view on system.privileges with the root user and is used to back SHOW SYSTEM GRANTS" +4294967097 4294967203 0 "table listing transitive closure of system.role_members" +4294967097 4294967204 0 "eventually consistent in-memory tenant capability cache for this node" +4294967097 4294967205 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" +4294967097 4294967206 0 "list super regions of databases visible to the current user" +4294967097 4294967207 0 "which entries of pg_catalog are implemented in this version of CockroachDB" +4294967097 4294967209 0 "node-level table listing all currently running range feeds" +4294967097 4294967210 0 "virtual table with default privileges" +4294967097 4294967211 0 "available regions for the cluster" +4294967097 4294967212 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" +4294967097 4294967213 0 "virtual table with table descriptors that still have data" +4294967097 4294967214 0 "virtual table with cross db references" +4294967097 4294967215 0 "virtual table with database privileges" +4294967097 4294967216 0 "virtual table to validate descriptors" +4294967097 4294967217 0 "decoded zone configurations from system.zones (KV scan)" +4294967097 4294967222 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)." +4294967097 4294967223 0 "stats for all tables accessible by current user in current database as of 10s ago" +4294967097 4294967224 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" +4294967097 4294967225 0 "key spans per SQL object" +4294967097 4294967226 0 "indexes accessible by current user in current database (KV scan)" +4294967097 4294967227 0 "details for all columns accessible by current user in current database (KV scan)" +4294967097 4294967232 0 "session variables (RAM)" +4294967097 4294967233 0 "session trace accumulated so far (RAM)" +4294967097 4294967234 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" +4294967097 4294967235 0 "server parameters, useful to construct connection URLs (RAM, local node only)" +4294967097 4294967236 0 "ranges is a view which queries ranges_no_leases for system ranges" +4294967097 4294967237 0 "range metadata without leaseholder details (KV join; expensive!)" +4294967097 4294967238 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" +4294967097 4294967239 0 "per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours)" +4294967097 4294967240 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)." +4294967097 4294967241 0 "node-level table listing all currently active memory monitors" +4294967097 4294967242 0 "current values for metrics (RAM; local node only)" +4294967097 4294967243 0 "running sessions visible by current user (RAM; local node only)" +4294967097 4294967244 0 "running user transactions visible by the current user (RAM; local node only)" +4294967097 4294967245 0 "running queries visible by current user (RAM; local node only)" +4294967097 4294967246 0 "Node statement execution insights" +4294967097 4294967247 0 "DistSQL remote flows information (RAM; local node only)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on this node. The local flows (those that are\nrunning on the same node as the query originated on) are not included." +4294967097 4294967248 0 "contention information (RAM; local node only)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." +4294967097 4294967249 0 "acquired table leases (RAM; local node only)" +4294967097 4294967250 0 "store details and status (cluster RPC; expensive!)" +4294967097 4294967251 0 "node details across the entire cluster (cluster RPC; expensive!)" +4294967097 4294967252 0 "wrapper over system.jobs with row access control (KV scan)" +4294967097 4294967253 0 "decoded job metadata from crdb_internal.system_jobs (KV scan)" +4294967097 4294967254 0 "in-flight spans (RAM; local node only)" +4294967097 4294967255 0 "cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout." +4294967097 4294967256 0 "key spans per table index" +4294967097 4294967257 0 "index columns for all indexes accessible by current user in current database (KV scan)" +4294967097 4294967258 0 "cluster-wide transaction contention events. Querying this table is an\n\t\texpensive operation since it creates a cluster-wide RPC-fanout." +4294967097 4294967259 0 "locally known edges in the gossip network (RAM; local node only)" +4294967097 4294967260 0 "locally known gossiped node liveness (RAM; local node only)" +4294967097 4294967261 0 "locally known gossiped health alerts (RAM; local node only)" +4294967097 4294967262 0 "locally known gossiped node details (RAM; local node only)" +4294967097 4294967263 0 "node liveness status, as seen by kv" +4294967097 4294967264 0 "forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)" +4294967097 4294967265 0 "telemetry counters (RAM; local node only)" +4294967097 4294967266 0 "databases accessible by the current user (KV scan)" +4294967097 4294967267 0 "CREATE statements for all user defined types accessible by the current user in current database (KV scan)" +4294967097 4294967268 0 "CREATE and ALTER statements for all tables accessible by current user in current database (KV scan)" +4294967097 4294967269 0 "CREATE statements for all user defined schemas accessible by the current user in current database (KV scan)" +4294967097 4294967270 0 "CREATE statements for all user-defined procedures" +4294967097 4294967271 0 "CREATE statements for all user-defined functions" +4294967097 4294967272 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." +4294967097 4294967273 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." +4294967097 4294967274 0 "cluster settings (RAM)" +4294967097 4294967275 0 "running sessions visible to current user (cluster RPC; expensive!)" +4294967097 4294967276 0 "running user transactions visible by the current user (cluster RPC; expensive!)" +4294967097 4294967277 0 "running queries visible by current user (cluster RPC; expensive!)" +4294967097 4294967278 0 "cluster-wide locks held in lock tables. Querying this table is an\n\t\texpensive operation since it creates a cluster-wide RPC-fanout." +4294967097 4294967279 0 "Node transaction execution insights" +4294967097 4294967280 0 "Cluster transaction execution insights" +4294967097 4294967281 0 "Cluster-wide statement execution insights" +4294967097 4294967282 0 "DistSQL remote flows information (cluster RPC; expensive!)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on any node in the cluster. The local flows (those that are\nrunning on the same node as the query originated on) are not included." +4294967097 4294967283 0 "contention information (cluster RPC; expensive!)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." +4294967097 4294967287 0 "like system.zones but overlaid with in-txn in-memory changes" +4294967097 4294967288 0 "like system.namespace but overlaid with in-txn in-memory changes" +4294967097 4294967289 0 "like system.descriptor but overlaid with in-txn in-memory changes and including virtual objects" +4294967097 4294967290 0 "like system.comments but overlaid with in-txn in-memory changes and including virtual objects" +4294967097 4294967291 0 "built-in functions (RAM/static)" +4294967097 4294967292 0 "built-in functions (RAM/static)" +4294967097 4294967293 0 "detailed identification strings (RAM, local node only)" +4294967097 4294967294 0 "backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan)" # This subtest covers the expected in-transaction behavior of the declarative # schema changer: the system tables should remain untouched until the @@ -935,7 +937,7 @@ SELECT * FROM crdb_internal.kv_catalog_namespace WHERE id = $kv_id query OOIT SELECT * FROM crdb_internal.kv_catalog_comments WHERE objoid = $kv_id ORDER BY 1,2,3,4 ---- -4294967098 111 0 this is a table +4294967097 111 0 this is a table statement ok DROP TABLE db.kv CASCADE; @@ -965,7 +967,7 @@ SELECT * FROM system.namespace WHERE id = $kv_id query OOIT SELECT * FROM crdb_internal.kv_catalog_comments WHERE objoid = $kv_id ORDER BY 1,2,3,4 ---- -4294967098 111 0 this is a table +4294967097 111 0 this is a table query TIIT SELECT * FROM system.comments WHERE object_id = $kv_id diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 194d80172f71..1e291a6a5210 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -47,6 +47,7 @@ test crdb_internal cluster_transaction_statistics publi test crdb_internal cluster_transactions public SELECT false test crdb_internal cluster_txn_execution_insights public SELECT false test crdb_internal create_function_statements public SELECT false +test crdb_internal create_procedure_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 @@ -76,8 +77,8 @@ test crdb_internal kv_flow_token_deductions publi test crdb_internal kv_inherited_role_members public SELECT false test crdb_internal kv_node_liveness public SELECT false test crdb_internal kv_node_status public SELECT false -test crdb_internal kv_repairable_catalog_corruptions public SELECT false test crdb_internal kv_protected_ts_records public SELECT false +test crdb_internal kv_repairable_catalog_corruptions public SELECT false test crdb_internal kv_store_status public SELECT false test crdb_internal kv_system_privileges public SELECT false test crdb_internal leases public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 0b75145b3d1c..544a62160a68 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -412,6 +412,7 @@ crdb_internal cluster_transaction_statistics crdb_internal cluster_transactions crdb_internal cluster_txn_execution_insights crdb_internal create_function_statements +crdb_internal create_procedure_statements crdb_internal create_schema_statements crdb_internal create_statements crdb_internal create_type_statements @@ -441,8 +442,8 @@ crdb_internal kv_flow_token_deductions crdb_internal kv_inherited_role_members crdb_internal kv_node_liveness crdb_internal kv_node_status -crdb_internal kv_repairable_catalog_corruptions crdb_internal kv_protected_ts_records +crdb_internal kv_repairable_catalog_corruptions crdb_internal kv_store_status crdb_internal kv_system_privileges crdb_internal leases @@ -763,6 +764,7 @@ cluster_transaction_statistics cluster_transactions cluster_txn_execution_insights create_function_statements +create_procedure_statements create_schema_statements create_statements create_type_statements @@ -792,8 +794,8 @@ kv_flow_token_deductions kv_inherited_role_members kv_node_liveness kv_node_status -kv_repairable_catalog_corruptions kv_protected_ts_records +kv_repairable_catalog_corruptions kv_store_status kv_system_privileges leases @@ -1177,6 +1179,7 @@ system public comments BASE system information_schema constraint_column_usage SYSTEM VIEW NO 1 system information_schema constraint_table_usage SYSTEM VIEW NO 1 system crdb_internal create_function_statements SYSTEM VIEW NO 1 +system crdb_internal create_procedure_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 @@ -3163,6 +3166,7 @@ NULL public system crdb_internal cluster_transaction_statist NULL public system crdb_internal cluster_transactions SELECT NO YES NULL public system crdb_internal cluster_txn_execution_insights SELECT NO YES NULL public system crdb_internal create_function_statements SELECT NO YES +NULL public system crdb_internal create_procedure_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 @@ -3192,8 +3196,8 @@ NULL public system crdb_internal kv_flow_token_deductions NULL public system crdb_internal kv_inherited_role_members SELECT NO YES NULL public system crdb_internal kv_node_liveness SELECT NO YES NULL public system crdb_internal kv_node_status SELECT NO YES -NULL public system crdb_internal kv_repairable_catalog_corruptions SELECT NO YES NULL public system crdb_internal kv_protected_ts_records SELECT NO YES +NULL public system crdb_internal kv_repairable_catalog_corruptions SELECT NO YES NULL public system crdb_internal kv_store_status SELECT NO YES NULL public system crdb_internal kv_system_privileges SELECT NO YES NULL public system crdb_internal leases SELECT NO YES @@ -3833,6 +3837,7 @@ NULL public system crdb_internal cluster_transaction_statist NULL public system crdb_internal cluster_transactions SELECT NO YES NULL public system crdb_internal cluster_txn_execution_insights SELECT NO YES NULL public system crdb_internal create_function_statements SELECT NO YES +NULL public system crdb_internal create_procedure_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 @@ -3862,8 +3867,8 @@ NULL public system crdb_internal kv_flow_token_deductions NULL public system crdb_internal kv_inherited_role_members SELECT NO YES NULL public system crdb_internal kv_node_liveness SELECT NO YES NULL public system crdb_internal kv_node_status SELECT NO YES -NULL public system crdb_internal kv_repairable_catalog_corruptions SELECT NO YES NULL public system crdb_internal kv_protected_ts_records SELECT NO YES +NULL public system crdb_internal kv_repairable_catalog_corruptions SELECT NO YES NULL public system crdb_internal kv_store_status SELECT NO YES NULL public system crdb_internal kv_system_privileges SELECT NO YES NULL public system crdb_internal leases SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 082269104585..f369af4f6444 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -212,42 +212,42 @@ is_updatable b 123 2 28 is_updatable c 123 3 28 false is_updatable_view a 124 1 0 false is_updatable_view b 124 2 0 false -pg_class oid 4294967098 1 0 false -pg_class relname 4294967098 2 0 false -pg_class relnamespace 4294967098 3 0 false -pg_class reltype 4294967098 4 0 false -pg_class reloftype 4294967098 5 0 false -pg_class relowner 4294967098 6 0 false -pg_class relam 4294967098 7 0 false -pg_class relfilenode 4294967098 8 0 false -pg_class reltablespace 4294967098 9 0 false -pg_class relpages 4294967098 10 0 false -pg_class reltuples 4294967098 11 0 false -pg_class relallvisible 4294967098 12 0 false -pg_class reltoastrelid 4294967098 13 0 false -pg_class relhasindex 4294967098 14 0 false -pg_class relisshared 4294967098 15 0 false -pg_class relpersistence 4294967098 16 0 false -pg_class relistemp 4294967098 17 0 false -pg_class relkind 4294967098 18 0 false -pg_class relnatts 4294967098 19 0 false -pg_class relchecks 4294967098 20 0 false -pg_class relhasoids 4294967098 21 0 false -pg_class relhaspkey 4294967098 22 0 false -pg_class relhasrules 4294967098 23 0 false -pg_class relhastriggers 4294967098 24 0 false -pg_class relhassubclass 4294967098 25 0 false -pg_class relfrozenxid 4294967098 26 0 false -pg_class relacl 4294967098 27 0 false -pg_class reloptions 4294967098 28 0 false -pg_class relforcerowsecurity 4294967098 29 0 false -pg_class relispartition 4294967098 30 0 false -pg_class relispopulated 4294967098 31 0 false -pg_class relreplident 4294967098 32 0 false -pg_class relrewrite 4294967098 33 0 false -pg_class relrowsecurity 4294967098 34 0 false -pg_class relpartbound 4294967098 35 0 false -pg_class relminmxid 4294967098 36 0 false +pg_class oid 4294967097 1 0 false +pg_class relname 4294967097 2 0 false +pg_class relnamespace 4294967097 3 0 false +pg_class reltype 4294967097 4 0 false +pg_class reloftype 4294967097 5 0 false +pg_class relowner 4294967097 6 0 false +pg_class relam 4294967097 7 0 false +pg_class relfilenode 4294967097 8 0 false +pg_class reltablespace 4294967097 9 0 false +pg_class relpages 4294967097 10 0 false +pg_class reltuples 4294967097 11 0 false +pg_class relallvisible 4294967097 12 0 false +pg_class reltoastrelid 4294967097 13 0 false +pg_class relhasindex 4294967097 14 0 false +pg_class relisshared 4294967097 15 0 false +pg_class relpersistence 4294967097 16 0 false +pg_class relistemp 4294967097 17 0 false +pg_class relkind 4294967097 18 0 false +pg_class relnatts 4294967097 19 0 false +pg_class relchecks 4294967097 20 0 false +pg_class relhasoids 4294967097 21 0 false +pg_class relhaspkey 4294967097 22 0 false +pg_class relhasrules 4294967097 23 0 false +pg_class relhastriggers 4294967097 24 0 false +pg_class relhassubclass 4294967097 25 0 false +pg_class relfrozenxid 4294967097 26 0 false +pg_class relacl 4294967097 27 0 false +pg_class reloptions 4294967097 28 0 false +pg_class relforcerowsecurity 4294967097 29 0 false +pg_class relispartition 4294967097 30 0 false +pg_class relispopulated 4294967097 31 0 false +pg_class relreplident 4294967097 32 0 false +pg_class relrewrite 4294967097 33 0 false +pg_class relrowsecurity 4294967097 34 0 false +pg_class relpartbound 4294967097 35 0 false +pg_class relminmxid 4294967097 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 960cc8768c2d..c634e727bf98 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -428,9 +428,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal 3233629770 NULL -4294967197 information_schema 3233629770 NULL -4294967110 pg_catalog 3233629770 NULL -4294966980 pg_extension 3233629770 NULL +4294967196 information_schema 3233629770 NULL +4294967109 pg_catalog 3233629770 NULL +4294966979 pg_extension 3233629770 NULL 105 public 2310524507 NULL # Verify that we can still see the schemas even if we don't have any privilege @@ -447,9 +447,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal 3233629770 NULL -4294967197 information_schema 3233629770 NULL -4294967110 pg_catalog 3233629770 NULL -4294966980 pg_extension 3233629770 NULL +4294967196 information_schema 3233629770 NULL +4294967109 pg_catalog 3233629770 NULL +4294966979 pg_extension 3233629770 NULL 105 public 2310524507 NULL user root @@ -1455,7 +1455,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967110 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967109 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1654,16 +1654,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967098 111 0 4294967098 110 14 a -4294967098 112 0 4294967098 110 15 a -4294967052 842401391 0 4294967098 110 1 n -4294967052 842401391 0 4294967098 110 2 n -4294967052 842401391 0 4294967098 110 3 n -4294967052 842401391 0 4294967098 110 4 n -4294967095 1179276562 0 4294967098 3687884464 0 n -4294967095 3935750373 0 4294967098 3687884465 0 n -4294967095 4072017905 0 4294967098 0 0 n -4294967095 4170826110 0 4294967098 0 0 n +4294967097 111 0 4294967097 110 14 a +4294967097 112 0 4294967097 110 15 a +4294967051 842401391 0 4294967097 110 1 n +4294967051 842401391 0 4294967097 110 2 n +4294967051 842401391 0 4294967097 110 3 n +4294967051 842401391 0 4294967097 110 4 n +4294967094 1179276562 0 4294967097 3687884464 0 n +4294967094 3935750373 0 4294967097 3687884465 0 n +4294967094 4072017905 0 4294967097 0 0 n +4294967094 4170826110 0 4294967097 0 0 n statement ok CREATE TABLE t_with_pk_seq (a INT GENERATED ALWAYS AS IDENTITY PRIMARY KEY, b INT); @@ -1701,9 +1701,9 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967052 4294967098 pg_rewrite pg_class -4294967098 4294967098 pg_class pg_class -4294967095 4294967098 pg_constraint pg_class +4294967051 4294967097 pg_rewrite pg_class +4294967097 4294967097 pg_class pg_class +4294967094 4294967097 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 @@ -1803,90 +1803,90 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967110 NULL 1 true b -17 bytea 4294967110 NULL -1 false b -18 char 4294967110 NULL 1 true b -19 name 4294967110 NULL -1 false b -20 int8 4294967110 NULL 8 true b -21 int2 4294967110 NULL 2 true b -22 int2vector 4294967110 NULL -1 false b -23 int4 4294967110 NULL 4 true b -24 regproc 4294967110 NULL 4 true b -25 text 4294967110 NULL -1 false b -26 oid 4294967110 NULL 4 true b -30 oidvector 4294967110 NULL -1 false b -700 float4 4294967110 NULL 4 true b -701 float8 4294967110 NULL 8 true b -705 unknown 4294967110 NULL 0 true b -869 inet 4294967110 NULL 24 true b -1000 _bool 4294967110 NULL -1 false b -1001 _bytea 4294967110 NULL -1 false b -1002 _char 4294967110 NULL -1 false b -1003 _name 4294967110 NULL -1 false b -1005 _int2 4294967110 NULL -1 false b -1006 _int2vector 4294967110 NULL -1 false b -1007 _int4 4294967110 NULL -1 false b -1008 _regproc 4294967110 NULL -1 false b -1009 _text 4294967110 NULL -1 false b -1013 _oidvector 4294967110 NULL -1 false b -1014 _bpchar 4294967110 NULL -1 false b -1015 _varchar 4294967110 NULL -1 false b -1016 _int8 4294967110 NULL -1 false b -1021 _float4 4294967110 NULL -1 false b -1022 _float8 4294967110 NULL -1 false b -1028 _oid 4294967110 NULL -1 false b -1041 _inet 4294967110 NULL -1 false b -1042 bpchar 4294967110 NULL -1 false b -1043 varchar 4294967110 NULL -1 false b -1082 date 4294967110 NULL 4 true b -1083 time 4294967110 NULL 8 true b -1114 timestamp 4294967110 NULL 8 true b -1115 _timestamp 4294967110 NULL -1 false b -1182 _date 4294967110 NULL -1 false b -1183 _time 4294967110 NULL -1 false b -1184 timestamptz 4294967110 NULL 8 true b -1185 _timestamptz 4294967110 NULL -1 false b -1186 interval 4294967110 NULL 24 true b -1187 _interval 4294967110 NULL -1 false b -1231 _numeric 4294967110 NULL -1 false b -1266 timetz 4294967110 NULL 12 true b -1270 _timetz 4294967110 NULL -1 false b -1560 bit 4294967110 NULL -1 false b -1561 _bit 4294967110 NULL -1 false b -1562 varbit 4294967110 NULL -1 false b -1563 _varbit 4294967110 NULL -1 false b -1700 numeric 4294967110 NULL -1 false b -2202 regprocedure 4294967110 NULL 4 true b -2205 regclass 4294967110 NULL 4 true b -2206 regtype 4294967110 NULL 4 true b -2207 _regprocedure 4294967110 NULL -1 false b -2210 _regclass 4294967110 NULL -1 false b -2211 _regtype 4294967110 NULL -1 false b -2249 record 4294967110 NULL 0 true p -2277 anyarray 4294967110 NULL -1 false p -2278 void 4294967110 NULL 0 true p -2283 anyelement 4294967110 NULL -1 false p -2287 _record 4294967110 NULL -1 false b -2950 uuid 4294967110 NULL 16 true b -2951 _uuid 4294967110 NULL -1 false b -3220 pg_lsn 4294967110 NULL 8 true b -3221 _pg_lsn 4294967110 NULL -1 false b -3614 tsvector 4294967110 NULL -1 false b -3615 tsquery 4294967110 NULL -1 false b -3643 _tsvector 4294967110 NULL -1 false b -3645 _tsquery 4294967110 NULL -1 false b -3802 jsonb 4294967110 NULL -1 false b -3807 _jsonb 4294967110 NULL -1 false b -4089 regnamespace 4294967110 NULL 4 true b -4090 _regnamespace 4294967110 NULL -1 false b -4096 regrole 4294967110 NULL 4 true b -4097 _regrole 4294967110 NULL -1 false b -90000 geometry 4294967110 NULL -1 false b -90001 _geometry 4294967110 NULL -1 false b -90002 geography 4294967110 NULL -1 false b -90003 _geography 4294967110 NULL -1 false b -90004 box2d 4294967110 NULL 32 true b -90005 _box2d 4294967110 NULL -1 false b +16 bool 4294967109 NULL 1 true b +17 bytea 4294967109 NULL -1 false b +18 char 4294967109 NULL 1 true b +19 name 4294967109 NULL -1 false b +20 int8 4294967109 NULL 8 true b +21 int2 4294967109 NULL 2 true b +22 int2vector 4294967109 NULL -1 false b +23 int4 4294967109 NULL 4 true b +24 regproc 4294967109 NULL 4 true b +25 text 4294967109 NULL -1 false b +26 oid 4294967109 NULL 4 true b +30 oidvector 4294967109 NULL -1 false b +700 float4 4294967109 NULL 4 true b +701 float8 4294967109 NULL 8 true b +705 unknown 4294967109 NULL 0 true b +869 inet 4294967109 NULL 24 true b +1000 _bool 4294967109 NULL -1 false b +1001 _bytea 4294967109 NULL -1 false b +1002 _char 4294967109 NULL -1 false b +1003 _name 4294967109 NULL -1 false b +1005 _int2 4294967109 NULL -1 false b +1006 _int2vector 4294967109 NULL -1 false b +1007 _int4 4294967109 NULL -1 false b +1008 _regproc 4294967109 NULL -1 false b +1009 _text 4294967109 NULL -1 false b +1013 _oidvector 4294967109 NULL -1 false b +1014 _bpchar 4294967109 NULL -1 false b +1015 _varchar 4294967109 NULL -1 false b +1016 _int8 4294967109 NULL -1 false b +1021 _float4 4294967109 NULL -1 false b +1022 _float8 4294967109 NULL -1 false b +1028 _oid 4294967109 NULL -1 false b +1041 _inet 4294967109 NULL -1 false b +1042 bpchar 4294967109 NULL -1 false b +1043 varchar 4294967109 NULL -1 false b +1082 date 4294967109 NULL 4 true b +1083 time 4294967109 NULL 8 true b +1114 timestamp 4294967109 NULL 8 true b +1115 _timestamp 4294967109 NULL -1 false b +1182 _date 4294967109 NULL -1 false b +1183 _time 4294967109 NULL -1 false b +1184 timestamptz 4294967109 NULL 8 true b +1185 _timestamptz 4294967109 NULL -1 false b +1186 interval 4294967109 NULL 24 true b +1187 _interval 4294967109 NULL -1 false b +1231 _numeric 4294967109 NULL -1 false b +1266 timetz 4294967109 NULL 12 true b +1270 _timetz 4294967109 NULL -1 false b +1560 bit 4294967109 NULL -1 false b +1561 _bit 4294967109 NULL -1 false b +1562 varbit 4294967109 NULL -1 false b +1563 _varbit 4294967109 NULL -1 false b +1700 numeric 4294967109 NULL -1 false b +2202 regprocedure 4294967109 NULL 4 true b +2205 regclass 4294967109 NULL 4 true b +2206 regtype 4294967109 NULL 4 true b +2207 _regprocedure 4294967109 NULL -1 false b +2210 _regclass 4294967109 NULL -1 false b +2211 _regtype 4294967109 NULL -1 false b +2249 record 4294967109 NULL 0 true p +2277 anyarray 4294967109 NULL -1 false p +2278 void 4294967109 NULL 0 true p +2283 anyelement 4294967109 NULL -1 false p +2287 _record 4294967109 NULL -1 false b +2950 uuid 4294967109 NULL 16 true b +2951 _uuid 4294967109 NULL -1 false b +3220 pg_lsn 4294967109 NULL 8 true b +3221 _pg_lsn 4294967109 NULL -1 false b +3614 tsvector 4294967109 NULL -1 false b +3615 tsquery 4294967109 NULL -1 false b +3643 _tsvector 4294967109 NULL -1 false b +3645 _tsquery 4294967109 NULL -1 false b +3802 jsonb 4294967109 NULL -1 false b +3807 _jsonb 4294967109 NULL -1 false b +4089 regnamespace 4294967109 NULL 4 true b +4090 _regnamespace 4294967109 NULL -1 false b +4096 regrole 4294967109 NULL 4 true b +4097 _regrole 4294967109 NULL -1 false b +90000 geometry 4294967109 NULL -1 false b +90001 _geometry 4294967109 NULL -1 false b +90002 geography 4294967109 NULL -1 false b +90003 _geography 4294967109 NULL -1 false b +90004 box2d 4294967109 NULL 32 true b +90005 _box2d 4294967109 NULL -1 false b 100110 t1 109 1546506610 -1 false c 100111 t1_m_seq 109 1546506610 -1 false c 100112 t1_n_seq 109 1546506610 -1 false c @@ -2380,7 +2380,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967110 NULL -1 false b +1000 _bool 4294967109 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2438,7 +2438,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967060 pg_proc 4294967110 3233629770 -1 false c +4294967059 pg_proc 4294967109 3233629770 -1 false c ## pg_catalog.pg_proc @@ -2449,14 +2449,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 -substring 4294967110 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 +substring 4294967109 pg_catalog NULL 12 NULL NULL 0 query TTBBBB colnames,rowsort SELECT proname, protransform, proisagg, proiswindow, prosecdef, proleakproof @@ -2539,7 +2539,7 @@ WHERE proname='_pg_char_max_length' ORDER BY p.oid ---- proname prosrc pronamespace nspname prorettype proargtypes -_pg_char_max_length _pg_char_max_length 4294967197 information_schema 20 26 23 +_pg_char_max_length _pg_char_max_length 4294967196 information_schema 20 26 23 query TOIOTTB colnames SELECT proname, provariadic, pronargs, prorettype, proargtypes, proargmodes, proisstrict @@ -2660,14 +2660,14 @@ OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_na ORDER BY d.objoid, description ---- relname objoid classoid objsubid description -pg_class 137 4294967098 0 mycomment1 -pg_class 137 4294967098 1 mycomment2 -pg_namespace 138 4294967069 0 mycomment4 -pg_proc 738 4294967060 0 Calculates the absolute value of `val`. -pg_proc 739 4294967060 0 Calculates the absolute value of `val`. -pg_proc 740 4294967060 0 Calculates the absolute value of `val`. -pg_class 645202854 4294967098 0 mycomment3 -pg_class 4294966982 4294967098 0 database users +pg_class 137 4294967097 0 mycomment1 +pg_class 137 4294967097 1 mycomment2 +pg_namespace 138 4294967068 0 mycomment4 +pg_proc 738 4294967059 0 Calculates the absolute value of `val`. +pg_proc 739 4294967059 0 Calculates the absolute value of `val`. +pg_proc 740 4294967059 0 Calculates the absolute value of `val`. +pg_class 645202854 4294967097 0 mycomment3 +pg_class 4294966981 4294967097 0 database users ## pg_catalog.pg_shdescription @@ -2678,7 +2678,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967092 mydbcomment +100 4294967091 mydbcomment ## pg_catalog.pg_event_trigger @@ -3262,7 +3262,7 @@ query OTOOTBBOOOOOOOO colnames SELECT * FROM pg_catalog.pg_operator where oprname='+' and oprleft='float8'::regtype ---- oid oprname oprnamespace oprowner oprkind oprcanmerge oprcanhash oprleft oprright oprresult oprcom oprnegate oprcode oprrest oprjoin -74817020 + 4294967110 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967109 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -4126,7 +4126,7 @@ subtest end query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294966981 +test pg_views 4294966980 # 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 c6efd6348d26..52516902eee8 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -572,6 +572,7 @@ cluster_transaction_statistics NULL cluster_transactions NULL cluster_txn_execution_insights NULL create_function_statements NULL +create_procedure_statements NULL create_schema_statements NULL create_statements NULL create_type_statements NULL @@ -601,8 +602,8 @@ kv_flow_token_deductions NULL kv_inherited_role_members NULL kv_node_liveness NULL kv_node_status NULL -kv_repairable_catalog_corruptions NULL kv_protected_ts_records NULL +kv_repairable_catalog_corruptions NULL kv_store_status NULL kv_system_privileges NULL leases NULL diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 55da80f69ec4..02a717d13e74 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -447,7 +447,7 @@ vectorized: true │ │ └── • render │ │ │ │ │ └── • filter - │ │ │ filter: classoid = 4294967098 + │ │ │ filter: classoid = 4294967097 │ │ │ │ │ └── • virtual table │ │ table: kv_catalog_comments@primary diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index 05d4b728188c..97aaeb07df74 100644 --- a/pkg/sql/opt/xform/testdata/external/activerecord +++ b/pkg/sql/opt/xform/testdata/external/activerecord @@ -201,7 +201,7 @@ sort │ │ └── filters │ │ ├── column86:86 = object_id:82 [outer=(82,86), constraints=(/82: (/NULL - ]; /86: (/NULL - ]), fd=(82)==(86), (86)==(82)] │ │ ├── sub_id:83 = attnum:6 [outer=(6,83), constraints=(/6: (/NULL - ]; /83: (/NULL - ]), fd=(6)==(83), (83)==(6)] - │ │ └── attrelid:1 < 4294966977 [outer=(1), constraints=(/1: (/NULL - /4294966976]; tight)] + │ │ └── attrelid:1 < 4294966976 [outer=(1), constraints=(/1: (/NULL - /4294966975]; tight)] │ └── aggregations │ ├── const-agg [as=attname:2, outer=(2)] │ │ └── attname:2 diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index 0600292b92de..4fddab75a8af 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -206,7 +206,7 @@ project │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967092 [outer=(176), constraints=(/176: (/NULL - /4294967091] [/4294967093 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967091 [outer=(176), constraints=(/176: (/NULL - /4294967090] [/4294967092 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index d4c74d11fd56..8658db5cd541 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -210,7 +210,7 @@ sort │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967092 [outer=(176), constraints=(/176: (/NULL - /4294967091] [/4294967093 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967091 [outer=(176), constraints=(/176: (/NULL - /4294967090] [/4294967092 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index 7d1ac4964941..c6b5cfa40172 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -99,7 +99,7 @@ project │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:76!null crdb_internal.kv_catalog_comments.objoid:77!null crdb_internal.kv_catalog_comments.objsubid:78!null crdb_internal.kv_catalog_comments.description:79!null │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967092 [outer=(76), constraints=(/76: (/NULL - /4294967091] [/4294967093 - ]; tight)] + │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967091 [outer=(76), constraints=(/76: (/NULL - /4294967090] [/4294967092 - ]; tight)] │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:78::INT8 [as=objsubid:85, outer=(78), immutable] │ │ │ │ │ │ │ └── filters @@ -110,7 +110,7 @@ project │ │ │ │ │ │ ├── scan kv_builtin_function_comments │ │ │ │ │ │ │ └── columns: crdb_internal.kv_builtin_function_comments.oid:81!null crdb_internal.kv_builtin_function_comments.description:82!null │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── 4294967060 [as=classoid:83] + │ │ │ │ │ │ └── 4294967059 [as=classoid:83] │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── columns: c.oid:91!null relname:92!null relnamespace:93!null n.oid:128!null nspname:129!null │ │ │ │ │ │ ├── fd: ()-->(92,129), (93)==(128), (128)==(93) @@ -293,7 +293,7 @@ sort │ │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:108!null crdb_internal.kv_catalog_comments.objoid:109!null crdb_internal.kv_catalog_comments.objsubid:110!null crdb_internal.kv_catalog_comments.description:111!null │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:108 != 4294967092 [outer=(108), constraints=(/108: (/NULL - /4294967091] [/4294967093 - ]; tight)] + │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:108 != 4294967091 [outer=(108), constraints=(/108: (/NULL - /4294967090] [/4294967092 - ]; tight)] │ │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:110::INT8 [as=objsubid:117, outer=(110), immutable] │ │ │ │ │ │ │ │ └── filters diff --git a/pkg/sql/parser/testdata/create_procedure b/pkg/sql/parser/testdata/create_procedure index 50620fe16468..02f56f13d523 100644 --- a/pkg/sql/parser/testdata/create_procedure +++ b/pkg/sql/parser/testdata/create_procedure @@ -171,19 +171,15 @@ parse CREATE PROCEDURE f() VOLATILE LANGUAGE SQL AS 'SELECT 1' ---- CREATE PROCEDURE f() - VOLATILE LANGUAGE SQL AS $$SELECT 1$$ -- normalized! CREATE PROCEDURE f() - VOLATILE LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized CREATE PROCEDURE f() - VOLATILE LANGUAGE SQL AS $$_$$ -- literals removed CREATE PROCEDURE _() - VOLATILE LANGUAGE SQL AS $$_$$ -- identifiers removed @@ -193,19 +189,15 @@ parse CREATE PROCEDURE f() CALLED ON NULL INPUT LANGUAGE SQL AS 'SELECT 1' ---- CREATE PROCEDURE f() - CALLED ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- normalized! CREATE PROCEDURE f() - CALLED ON NULL INPUT LANGUAGE SQL AS $$SELECT 1$$ -- fully parenthesized CREATE PROCEDURE f() - CALLED ON NULL INPUT LANGUAGE SQL AS $$_$$ -- literals removed CREATE PROCEDURE _() - CALLED ON NULL INPUT LANGUAGE SQL AS $$_$$ -- identifiers removed diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index d408f6bf9ea0..8fba840d465a 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -372,11 +372,11 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( if err != nil { return nil, err } - if isFuncs && fnDesc.IsProcedure { + if isFuncs && fnDesc.IsProcedure() { return nil, pgerror.Newf(pgcode.WrongObjectType, "%q is not a %s", fnDesc.Name, "function") } - if !isFuncs && !fnDesc.IsProcedure { + if !isFuncs && !fnDesc.IsProcedure() { return nil, pgerror.Newf(pgcode.WrongObjectType, "%q is not a %s", fnDesc.Name, "procedure") } @@ -473,7 +473,7 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( if err != nil { return err } - if isProcs != fn.IsProcedure { + if isProcs != fn.IsProcedure() { // Skip functions if ALL PROCEDURES was specified, and // skip procedures if ALL FUNCTIONS was specified. return nil diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index 5c76e8139bf5..971ad7d69763 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -565,7 +565,7 @@ func (i *immediateVisitor) SetObjectParentID(ctx context.Context, op scop.SetObj ArgTypes: make([]*types.T, len(t.GetParams())), ReturnType: t.GetReturnType().Type, ReturnSet: t.GetReturnType().ReturnSet, - IsProcedure: t.IsProcedure, + IsProcedure: t.IsProcedure(), } for i := range t.Params { ol.ArgTypes[i] = t.Params[i].Type diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index dd753cd4ab6d..53c86a2b5fbc 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -127,6 +127,7 @@ const ( CrdbInternalClusterStmtStatsTableID CrdbInternalClusterTxnStatsTableID CrdbInternalCreateFunctionStmtsTableID + CrdbInternalCreateProcedureStmtsTableID CrdbInternalCreateSchemaStmtsTableID CrdbInternalCreateStmtsTableID CrdbInternalCreateTypeStmtsTableID diff --git a/pkg/sql/sem/tree/create_routine.go b/pkg/sql/sem/tree/create_routine.go index 3fc797b9b0bc..c95c2f9fc74c 100644 --- a/pkg/sql/sem/tree/create_routine.go +++ b/pkg/sql/sem/tree/create_routine.go @@ -128,6 +128,10 @@ func (node *CreateRoutine) Format(ctx *FmtCtx) { case RoutineBodyStr: funcBody = t continue + case RoutineLeakproof, RoutineVolatility, RoutineNullInputBehavior: + if node.IsProcedure { + continue + } } ctx.FormatNode(option) ctx.WriteString("\n\t") From 47157fd62e183a85ca54d095f4b40925d65091bd Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Sun, 24 Sep 2023 12:18:58 -0400 Subject: [PATCH 11/12] sql: add SHOW CREATE PROCEDURE This commit adds support for the `SHOW CREATE PROCEDURE` statement. Release note: None --- pkg/sql/delegate/show_function.go | 22 +++-- .../logictest/testdata/logic_test/show_create | 89 +++++++++++++++++++ pkg/sql/logictest/testdata/logic_test/udf | 59 ------------ pkg/sql/parser/sql.y | 10 +++ pkg/sql/sem/tree/show.go | 9 +- pkg/sql/sem/tree/stmt.go | 7 +- 6 files changed, 128 insertions(+), 68 deletions(-) diff --git a/pkg/sql/delegate/show_function.go b/pkg/sql/delegate/show_function.go index e061abe3b134..55a97ec1e42c 100644 --- a/pkg/sql/delegate/show_function.go +++ b/pkg/sql/delegate/show_function.go @@ -22,10 +22,10 @@ func (d *delegator) delegateShowCreateFunction(n *tree.ShowCreateRoutine) (tree. // 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 +SELECT %[1]s, create_statement +FROM crdb_internal.%[2]s +WHERE schema_name = %[3]s +AND %[1]s = %[4]s ` resolvableFunctionReference := &n.Name un, ok := resolvableFunctionReference.FunctionReference.(*tree.UnresolvedName) @@ -45,9 +45,18 @@ AND function_name = %[2]s return nil, err } + routineType := tree.UDFRoutine + tab := "create_function_statements" + nameCol := "function_name" + if n.Procedure { + routineType = tree.ProcedureRoutine + tab = "create_procedure_statements" + nameCol = "procedure_name" + } + var udfSchema string for _, o := range fn.Overloads { - if o.Type == tree.UDFRoutine { + if o.Type == routineType { udfSchema = o.Schema break } @@ -69,6 +78,7 @@ AND function_name = %[2]s } } - fullQuery := fmt.Sprintf(query, lexbase.EscapeSQLString(udfSchema), lexbase.EscapeSQLString(un.Parts[0])) + fullQuery := fmt.Sprintf(query, + nameCol, tab, lexbase.EscapeSQLString(udfSchema), lexbase.EscapeSQLString(un.Parts[0])) return d.parse(fullQuery) } diff --git a/pkg/sql/logictest/testdata/logic_test/show_create b/pkg/sql/logictest/testdata/logic_test/show_create index c0f994c42745..6194cdf6a9a6 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_create +++ b/pkg/sql/logictest/testdata/logic_test/show_create @@ -198,3 +198,92 @@ SHOW CREATE INDEXES FROM nonexistent statement error relation "nonexistent" does not exist SHOW CREATE SECONDARY INDEXES FROM nonexistent + +statement ok +CREATE FUNCTION r1() RETURNS INT LANGUAGE SQL AS 'SELECT 1' + +statement ok +CREATE FUNCTION r1(i INT) RETURNS INT LANGUAGE SQL AS 'SELECT 1' + +statement ok +CREATE PROCEDURE r1(s STRING) LANGUAGE SQL AS 'SELECT 1' + +statement ok +CREATE PROCEDURE r1(s STRING, i INT) LANGUAGE SQL AS 'SELECT 1' + +query TT +SELECT * FROM [SHOW CREATE FUNCTION r1] ORDER BY 2 +---- +r1 CREATE FUNCTION public.r1() + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +r1 CREATE FUNCTION public.r1(IN i INT8) + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ + +query TT +SELECT * FROM [SHOW CREATE PROCEDURE r1] ORDER BY 2 +---- +r1 CREATE PROCEDURE public.r1(IN s STRING) + LANGUAGE SQL + AS $$ + SELECT 1; + $$ +r1 CREATE PROCEDURE public.r1(IN s STRING, IN i INT8) + LANGUAGE SQL + AS $$ + SELECT 1; + $$ + +statement ok +CREATE SCHEMA sc + +statement ok +CREATE FUNCTION sc.r2() RETURNS INT LANGUAGE SQL AS 'SELECT 1' + +statement ok +CREATE PROCEDURE sc.r2(s STRING) LANGUAGE SQL AS 'SELECT 1' + +statement error pgcode 42883 pq: unknown function: r2() +SHOW CREATE FUNCTION r2; + +# TODO(mgartner): Fix the error message here. +statement error pgcode 42883 pq: unknown function: r2() +SHOW CREATE PROCEDURE r2; + +statement ok +SET search_path = sc; + +query TT +SHOW CREATE FUNCTION r2 +---- +r2 CREATE FUNCTION sc.r2() + RETURNS INT8 + VOLATILE + NOT LEAKPROOF + CALLED ON NULL INPUT + LANGUAGE SQL + AS $$ + SELECT 1; + $$ + +query TT +SHOW CREATE PROCEDURE r2 +---- +r2 CREATE PROCEDURE sc.r2(IN s STRING) + LANGUAGE SQL + AS $$ + SELECT 1; + $$ diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index a867250b71f0..73e6940d87b5 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -167,65 +167,6 @@ statement ok USE test; subtest end -subtest show_create_function - -query T -SELECT create_statement FROM [SHOW CREATE FUNCTION proc_f] ORDER BY 1 ----- -CREATE FUNCTION public.proc_f(IN INT8) - RETURNS INT8 - VOLATILE - NOT LEAKPROOF - CALLED ON NULL INPUT - LANGUAGE SQL - AS $$ - SELECT 1; -$$ -CREATE FUNCTION public.proc_f(IN STRING, IN b INT8) - RETURNS STRING - IMMUTABLE - LEAKPROOF - STRICT - LANGUAGE SQL - AS $$ - SELECT 'hello'; -$$ - -statement error pgcode 42883 pq: unknown function: proc_f_2() -SHOW CREATE FUNCTION proc_f_2; - -query T -SELECT create_statement FROM [SHOW CREATE FUNCTION sc.proc_f_2]; ----- -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 T -SELECT create_statement FROM [SHOW CREATE FUNCTION proc_f_2]; ----- -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 = public; -subtest end subtest udf_regproc diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 009513813e87..6fe77e91e5d5 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -8568,6 +8568,16 @@ show_create_stmt: }, } } +| SHOW CREATE PROCEDURE db_object_name + { + /* SKIP DOC */ + $$.val = &tree.ShowCreateRoutine{ + Name: tree.ResolvableFunctionReference{ + FunctionReference: $4.unresolvedObjectName().ToUnresolvedName(), + }, + Procedure: true, + } + } | SHOW CREATE ALL SCHEMAS { $$.val = &tree.ShowCreateAllSchemas{} diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 013362fd22d3..a482cc01067e 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -1397,12 +1397,17 @@ var _ Statement = &ShowCompletions{} // ShowCreateRoutine represents a SHOW CREATE FUNCTION or SHOW CREATE PROCEDURE // statement. type ShowCreateRoutine struct { - Name ResolvableFunctionReference + Name ResolvableFunctionReference + Procedure bool } // Format implements the NodeFormatter interface. func (node *ShowCreateRoutine) Format(ctx *FmtCtx) { - ctx.WriteString("SHOW CREATE FUNCTION ") + if node.Procedure { + ctx.WriteString("SHOW CREATE PROCEDURE ") + } else { + ctx.WriteString("SHOW CREATE FUNCTION ") + } ctx.FormatNode(&node.Name) } diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 36a07b4d9609..383ed294e0f4 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -2013,7 +2013,12 @@ func (*ShowCreateRoutine) StatementReturnType() StatementReturnType { return Row func (*ShowCreateRoutine) StatementType() StatementType { return TypeDML } // StatementTag returns a short string identifying the type of statement. -func (*ShowCreateRoutine) StatementTag() string { return "SHOW CREATE FUNCTION" } +func (n *ShowCreateRoutine) StatementTag() string { + if n.Procedure { + return "SHOW CREATE PROCEDURE" + } + return "SHOW CREATE FUNCTION" +} // StatementReturnType implements the Statement interface. func (*ShowCreateExternalConnections) StatementReturnType() StatementReturnType { return Rows } From d975845aa225a8b2f8cc16cf8900d025132736cd Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Wed, 4 Oct 2023 11:27:54 -0500 Subject: [PATCH 12/12] .gitignore: ignore `embedded.go` files that are no longer generated Epic: none Release note: None --- .gitignore | 2 -- 1 file changed, 2 deletions(-) diff --git a/.gitignore b/.gitignore index ed039649f514..22c5e1e26ed9 100644 --- a/.gitignore +++ b/.gitignore @@ -53,8 +53,6 @@ build/Railroad.jar # Generated code. *.pb.go *.pb.gw.go -pkg/roachprod/vm/aws/embedded.go -pkg/security/securitytest/embedded.go pkg/ccl/kvccl/kvtenantccl/upgradeinterlockccl/generated_test.go pkg/ccl/backupccl/data_driven_generated_test.go pkg/ccl/backupccl/restore_memory_monitoring_generated_test.go