diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index c78144c9f853..d39192f4b180 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -74,6 +74,7 @@ crdb_internal node_contention_events table admin NULL NULL crdb_internal node_distsql_flows table admin NULL NULL crdb_internal node_execution_insights table admin NULL NULL crdb_internal node_inflight_trace_spans table admin NULL NULL +crdb_internal node_memory_monitors table admin NULL NULL crdb_internal node_metrics table admin NULL NULL crdb_internal node_queries table admin NULL NULL crdb_internal node_runtime_info table admin NULL NULL diff --git a/pkg/ccl/serverccl/admin_test.go b/pkg/ccl/serverccl/admin_test.go index e77ffd274241..a90e20a71eb4 100644 --- a/pkg/ccl/serverccl/admin_test.go +++ b/pkg/ccl/serverccl/admin_test.go @@ -30,13 +30,13 @@ import ( var adminPrefix = "/_admin/v1/" func getAdminJSONProto( - ts serverutils.TestServerInterface, path string, response protoutil.Message, + ts serverutils.TestTenantInterface, path string, response protoutil.Message, ) error { return getAdminJSONProtoWithAdminOption(ts, path, response, true) } func getAdminJSONProtoWithAdminOption( - ts serverutils.TestServerInterface, path string, response protoutil.Message, isAdmin bool, + ts serverutils.TestTenantInterface, path string, response protoutil.Message, isAdmin bool, ) error { return serverutils.GetJSONProtoWithAdminOption(ts, adminPrefix+path, response, isAdmin) } @@ -195,3 +195,40 @@ func TestListTenants(t *testing.T) { } require.True(t, appTenantFound, "test tenant not found") } + +func TestTableAndDatabaseDetailsAndStats(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + st, db := serverutils.StartTenant(t, s, base.TestTenantArgs{ + TenantID: serverutils.TestTenantID(), + }) + _, err := db.Exec("CREATE TABLE test (id int)") + require.NoError(t, err) + _, err = db.Exec("INSERT INTO test VALUES (1)") + require.NoError(t, err) + + // DatabaseDetails + dbResp := &serverpb.DatabaseDetailsResponse{} + err = getAdminJSONProto(st, "databases/defaultdb", dbResp) + require.NoError(t, err) + + require.Equal(t, dbResp.TableNames[0], "public.test") + + // TableStats + tableStatsResp := &serverpb.TableStatsResponse{} + err = getAdminJSONProto(st, "databases/defaultdb/tables/public.test/stats", tableStatsResp) + require.NoError(t, err) + + require.Greater(t, tableStatsResp.Stats.LiveBytes, int64(0)) + + // TableDetails + tableDetailsResp := &serverpb.TableDetailsResponse{} + err = getAdminJSONProto(st, "databases/defaultdb/tables/public.test", tableDetailsResp) + require.NoError(t, err) + + require.Greater(t, tableDetailsResp.DataLiveBytes, int64(0)) +} diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index dff9079851df..d81862265176 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -87,6 +87,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 1] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/1/crdb_internal.node_distsql_flows.txt... done [node 1] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/1/crdb_internal.node_execution_insights.txt... done [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/1/crdb_internal.node_inflight_trace_spans.txt... done +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/1/crdb_internal.node_memory_monitors.txt... done [node 1] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/1/crdb_internal.node_metrics.txt... done [node 1] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/1/crdb_internal.node_queries.txt... done [node 1] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/1/crdb_internal.node_runtime_info.txt... done @@ -206,6 +207,9 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/2/crdb_internal.node_inflight_trace_spans.txt... [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans: last request failed: dial tcp ... [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans: creating error output: debug/nodes/2/crdb_internal.node_inflight_trace_spans.txt.err.txt... done +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/2/crdb_internal.node_memory_monitors.txt... +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors: last request failed: dial tcp ... +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors: creating error output: debug/nodes/2/crdb_internal.node_memory_monitors.txt.err.txt... done [node 2] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/2/crdb_internal.node_metrics.txt... [node 2] retrieving SQL data for crdb_internal.node_metrics: last request failed: dial tcp ... [node 2] retrieving SQL data for crdb_internal.node_metrics: creating error output: debug/nodes/2/crdb_internal.node_metrics.txt.err.txt... done @@ -276,6 +280,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/3/crdb_internal.node_distsql_flows.txt... done [node 3] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/3/crdb_internal.node_execution_insights.txt... done [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/3/crdb_internal.node_inflight_trace_spans.txt... done +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/3/crdb_internal.node_memory_monitors.txt... done [node 3] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/3/crdb_internal.node_metrics.txt... done [node 3] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/3/crdb_internal.node_queries.txt... done [node 3] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/3/crdb_internal.node_runtime_info.txt... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 625675f21454..d02b20299bb1 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -87,6 +87,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 1] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/1/crdb_internal.node_distsql_flows.txt... done [node 1] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/1/crdb_internal.node_execution_insights.txt... done [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/1/crdb_internal.node_inflight_trace_spans.txt... done +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/1/crdb_internal.node_memory_monitors.txt... done [node 1] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/1/crdb_internal.node_metrics.txt... done [node 1] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/1/crdb_internal.node_queries.txt... done [node 1] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/1/crdb_internal.node_runtime_info.txt... done @@ -185,6 +186,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/3/crdb_internal.node_distsql_flows.txt... done [node 3] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/3/crdb_internal.node_execution_insights.txt... done [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/3/crdb_internal.node_inflight_trace_spans.txt... done +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/3/crdb_internal.node_memory_monitors.txt... done [node 3] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/3/crdb_internal.node_metrics.txt... done [node 3] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/3/crdb_internal.node_queries.txt... done [node 3] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/3/crdb_internal.node_runtime_info.txt... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index 15529501e020..f6fa8727818c 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -87,6 +87,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 1] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/1/crdb_internal.node_distsql_flows.txt... done [node 1] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/1/crdb_internal.node_execution_insights.txt... done [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/1/crdb_internal.node_inflight_trace_spans.txt... done +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/1/crdb_internal.node_memory_monitors.txt... done [node 1] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/1/crdb_internal.node_metrics.txt... done [node 1] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/1/crdb_internal.node_queries.txt... done [node 1] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/1/crdb_internal.node_runtime_info.txt... done @@ -184,6 +185,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/3/crdb_internal.node_distsql_flows.txt... done [node 3] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/3/crdb_internal.node_execution_insights.txt... done [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/3/crdb_internal.node_inflight_trace_spans.txt... done +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/3/crdb_internal.node_memory_monitors.txt... done [node 3] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/3/crdb_internal.node_metrics.txt... done [node 3] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/3/crdb_internal.node_queries.txt... done [node 3] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/3/crdb_internal.node_runtime_info.txt... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 0d9f6c2d4caf..feaff2b710d0 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -90,6 +90,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] retrieving SQL data for crdb_internal.node_distsql_flows... writing output: debug/nodes/1/crdb_internal.node_distsql_flows.txt... done [node 1] retrieving SQL data for crdb_internal.node_execution_insights... writing output: debug/nodes/1/crdb_internal.node_execution_insights.txt... done [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans... writing output: debug/nodes/1/crdb_internal.node_inflight_trace_spans.txt... done +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors... writing output: debug/nodes/1/crdb_internal.node_memory_monitors.txt... done [node 1] retrieving SQL data for crdb_internal.node_metrics... writing output: debug/nodes/1/crdb_internal.node_metrics.txt... done [node 1] retrieving SQL data for crdb_internal.node_queries... writing output: debug/nodes/1/crdb_internal.node_queries.txt... done [node 1] retrieving SQL data for crdb_internal.node_runtime_info... writing output: debug/nodes/1/crdb_internal.node_runtime_info.txt... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index 286e7b83ee8c..6c7072037639 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -319,6 +319,9 @@ zip [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans... [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans: done [node 1] retrieving SQL data for crdb_internal.node_inflight_trace_spans: writing output: debug/nodes/1/crdb_internal.node_inflight_trace_spans.txt... +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors... +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors: done +[node 1] retrieving SQL data for crdb_internal.node_memory_monitors: writing output: debug/nodes/1/crdb_internal.node_memory_monitors.txt... [node 1] retrieving SQL data for crdb_internal.node_metrics... [node 1] retrieving SQL data for crdb_internal.node_metrics: done [node 1] retrieving SQL data for crdb_internal.node_metrics: writing output: debug/nodes/1/crdb_internal.node_metrics.txt... @@ -665,6 +668,9 @@ zip [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans... [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans: done [node 2] retrieving SQL data for crdb_internal.node_inflight_trace_spans: writing output: debug/nodes/2/crdb_internal.node_inflight_trace_spans.txt... +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors... +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors: done +[node 2] retrieving SQL data for crdb_internal.node_memory_monitors: writing output: debug/nodes/2/crdb_internal.node_memory_monitors.txt... [node 2] retrieving SQL data for crdb_internal.node_metrics... [node 2] retrieving SQL data for crdb_internal.node_metrics: done [node 2] retrieving SQL data for crdb_internal.node_metrics: writing output: debug/nodes/2/crdb_internal.node_metrics.txt... @@ -1011,6 +1017,9 @@ zip [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans... [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans: done [node 3] retrieving SQL data for crdb_internal.node_inflight_trace_spans: writing output: debug/nodes/3/crdb_internal.node_inflight_trace_spans.txt... +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors... +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors: done +[node 3] retrieving SQL data for crdb_internal.node_memory_monitors: writing output: debug/nodes/3/crdb_internal.node_memory_monitors.txt... [node 3] retrieving SQL data for crdb_internal.node_metrics... [node 3] retrieving SQL data for crdb_internal.node_metrics: done [node 3] retrieving SQL data for crdb_internal.node_metrics: writing output: debug/nodes/3/crdb_internal.node_metrics.txt... diff --git a/pkg/cli/zip_table_registry.go b/pkg/cli/zip_table_registry.go index 5141fe21b43a..3f2e7b0146c2 100644 --- a/pkg/cli/zip_table_registry.go +++ b/pkg/cli/zip_table_registry.go @@ -659,6 +659,17 @@ var zipInternalTablesPerNode = DebugZipTableRegistry{ ) SELECT trace_id, parent_span_id, span_id, goroutine_id, finished, start_time, duration, operation, payload_type FROM spans, LATERAL crdb_internal.payloads_for_span(span_id)`, }, + "crdb_internal.node_memory_monitors": { + nonSensitiveCols: NonSensitiveColumns{ + "level", + "name", + "id", + "parent_id", + "used", + "reserved_used", + "reserved_reserved", + }, + }, "crdb_internal.node_metrics": { nonSensitiveCols: NonSensitiveColumns{ "store_id", diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 4aefb916d095..9f6b5c9a8a74 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -523,6 +523,7 @@ func IsEndTxnTriggeringRetryError( } const lockResolutionBatchSize = 500 +const lockResolutionBatchByteSize = 4 << 20 // 4 MB. // resolveLocalLocks synchronously resolves any locks that are local to this // range in the same batch and returns those lock spans. The remainder are @@ -541,17 +542,19 @@ func resolveLocalLocks( evalCtx EvalContext, ) (resolvedLocks []roachpb.LockUpdate, externalLocks []roachpb.Span, _ error) { var resolveAllowance int64 = lockResolutionBatchSize + var targetBytes int64 = lockResolutionBatchByteSize if args.InternalCommitTrigger != nil { // If this is a system transaction (such as a split or merge), don't // enforce the resolve allowance. These transactions rely on having // their locks resolved synchronously. resolveAllowance = 0 + targetBytes = 0 } - return resolveLocalLocksWithPagination(ctx, desc, readWriter, ms, args, txn, evalCtx, resolveAllowance) + return resolveLocalLocksWithPagination(ctx, desc, readWriter, ms, args, txn, evalCtx, resolveAllowance, targetBytes) } -// resolveLocalLocksWithPagination is resolveLocalLocks but with a max key -// limit. +// resolveLocalLocksWithPagination is resolveLocalLocks but with a max key and +// target bytes limit. func resolveLocalLocksWithPagination( ctx context.Context, desc *roachpb.RangeDescriptor, @@ -561,6 +564,7 @@ func resolveLocalLocksWithPagination( txn *roachpb.Transaction, evalCtx EvalContext, maxKeys int64, + targetBytes int64, ) (resolvedLocks []roachpb.LockUpdate, externalLocks []roachpb.Span, _ error) { if mergeTrigger := args.InternalCommitTrigger.GetMergeTrigger(); mergeTrigger != nil { // If this is a merge, then use the post-merge descriptor to determine @@ -592,26 +596,33 @@ func resolveLocalLocksWithPagination( // // Note that the underlying pebbleIterator will still be reused // since readWriter is a pebbleBatch in the typical case. - ok, _, _, err := storage.MVCCResolveWriteIntent(ctx, readWriter, ms, update, - storage.MVCCResolveWriteIntentOptions{}) + ok, numBytes, resumeSpan, err := storage.MVCCResolveWriteIntent(ctx, readWriter, ms, update, + storage.MVCCResolveWriteIntentOptions{TargetBytes: targetBytes}) if err != nil { return 0, 0, 0, errors.Wrapf(err, "resolving write intent at %s on end transaction [%s]", span, txn.Status) } if ok { numKeys = 1 } - resolvedLocks = append(resolvedLocks, update) - // If requested, replace point tombstones with range tombstones. - if ok && evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { - if err := storage.ReplacePointTombstonesWithRangeTombstones( - ctx, spanset.DisableReadWriterAssertions(readWriter), - ms, update.Key, update.EndKey); err != nil { - return 0, 0, 0, errors.Wrapf(err, - "replacing point tombstones with range tombstones for write intent at %s on end transaction [%s]", - span, txn.Status) + if resumeSpan != nil { + externalLocks = append(externalLocks, *resumeSpan) + resumeReason = kvpb.RESUME_BYTE_LIMIT + } else { + // !ok && resumeSpan == nil is a valid condition that means + // that no intent was found. + resolvedLocks = append(resolvedLocks, update) + // If requested, replace point tombstones with range tombstones. + if ok && evalCtx.EvalKnobs().UseRangeTombstonesForPointDeletes { + if err := storage.ReplacePointTombstonesWithRangeTombstones( + ctx, spanset.DisableReadWriterAssertions(readWriter), + ms, update.Key, update.EndKey); err != nil { + return 0, 0, 0, errors.Wrapf(err, + "replacing point tombstones with range tombstones for write intent at %s on end transaction [%s]", + span, txn.Status) + } } } - return numKeys, 0, 0, nil + return numKeys, numBytes, resumeReason, nil } // For update ranges, cut into parts inside and outside our key // range. Resolve locally inside, delegate the rest. In particular, @@ -620,8 +631,8 @@ func resolveLocalLocksWithPagination( externalLocks = append(externalLocks, outSpans...) if inSpan != nil { update.Span = *inSpan - numKeys, _, resumeSpan, resumeReason, err := storage.MVCCResolveWriteIntentRange(ctx, readWriter, ms, update, - storage.MVCCResolveWriteIntentRangeOptions{MaxKeys: maxKeys}) + numKeys, numBytes, resumeSpan, resumeReason, err := storage.MVCCResolveWriteIntentRange(ctx, readWriter, ms, update, + storage.MVCCResolveWriteIntentRangeOptions{MaxKeys: maxKeys, TargetBytes: targetBytes}) if err != nil { return 0, 0, 0, errors.Wrapf(err, "resolving write intent range at %s on end transaction [%s]", span, txn.Status) } @@ -643,12 +654,12 @@ func resolveLocalLocksWithPagination( span, txn.Status) } } - return numKeys, 0, resumeReason, nil + return numKeys, numBytes, resumeReason, nil } return 0, 0, 0, nil } - numKeys, _, _, err := storage.MVCCPaginate(ctx, maxKeys, 0 /* targetBytes */, false /* allowEmpty */, f) + numKeys, _, _, err := storage.MVCCPaginate(ctx, maxKeys, targetBytes, false /* allowEmpty */, f) if err != nil { return nil, nil, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 51d7e2a4d307..df6db245f948 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -12,6 +12,7 @@ package batcheval import ( "context" + "fmt" "regexp" "testing" @@ -1390,3 +1391,208 @@ func TestComputeSplitRangeKeyStatsDelta(t *testing.T) { }) } } + +// TestResolveLocalLocks tests resolveLocalLocks for point and ranged intents +// as well as under a max key or max byte limit, ensuring the returned +// resolvedLocks, externalLocks, and numBytes are as expected. +func TestResolveLocalLocks(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + intToKey := func(i int) roachpb.Key { + return roachpb.Key(fmt.Sprintf("%01000d", i)) + } + ceil := func(i int, j int) int { + return (i + j - 1) / j + } + + const ( + numKeys = 20 + keysPerRangedLock = 4 + maxKeys = 11 // not divisible by keysPerRangedLock + targetBytes = 11900 + keysFromTargetBytes = 12 // divisible by keysPerRangedLock + ) + + pointLocks := make([]roachpb.Span, numKeys) + for i := range pointLocks { + pointLocks[i].Key = intToKey(i) + } + rangedLocks := make([]roachpb.Span, numKeys/keysPerRangedLock) + for i := range rangedLocks { + rangedLocks[i].Key = intToKey(i * keysPerRangedLock) + rangedLocks[i].EndKey = intToKey((i + 1) * keysPerRangedLock) + } + + expectedResolvedLocksPointMaxKeys := make([]roachpb.Span, maxKeys) + for i := range expectedResolvedLocksPointMaxKeys { + expectedResolvedLocksPointMaxKeys[i].Key = intToKey(i) + } + expectedExternalLocksPointMaxKeys := make([]roachpb.Span, numKeys-maxKeys) + for i := range expectedExternalLocksPointMaxKeys { + expectedExternalLocksPointMaxKeys[i].Key = intToKey(i + maxKeys) + } + + expectedResolvedLocksRangedMaxKeys := make([]roachpb.Span, ceil(maxKeys, keysPerRangedLock)) + for i := range expectedResolvedLocksRangedMaxKeys { + expectedResolvedLocksRangedMaxKeys[i].Key = intToKey(i * keysPerRangedLock) + if i == len(expectedResolvedLocksRangedMaxKeys)-1 { + expectedResolvedLocksRangedMaxKeys[i].EndKey = intToKey(maxKeys - 1).Next() + } else { + expectedResolvedLocksRangedMaxKeys[i].EndKey = intToKey((i + 1) * keysPerRangedLock) + } + } + expectedExternalLocksRangedMaxKeys := make([]roachpb.Span, ceil(numKeys, keysPerRangedLock)-ceil(maxKeys, keysPerRangedLock)+1) + for i := range expectedExternalLocksRangedMaxKeys { + offset := maxKeys / keysPerRangedLock + if i == 0 { + expectedExternalLocksRangedMaxKeys[i].Key = intToKey(maxKeys - 1).Next() + } else { + expectedExternalLocksRangedMaxKeys[i].Key = intToKey((i + offset) * keysPerRangedLock) + } + expectedExternalLocksRangedMaxKeys[i].EndKey = intToKey((i + offset + 1) * keysPerRangedLock) + } + + expectedResolvedLocksPointTargetBytes := make([]roachpb.Span, keysFromTargetBytes) + for i := range expectedResolvedLocksPointTargetBytes { + expectedResolvedLocksPointTargetBytes[i].Key = intToKey(i) + } + expectedExternalLocksPointTargetBytes := make([]roachpb.Span, numKeys-keysFromTargetBytes) + for i := range expectedExternalLocksPointTargetBytes { + expectedExternalLocksPointTargetBytes[i].Key = intToKey(i + keysFromTargetBytes) + } + + expectedResolvedLocksRangedTargetBytes := make([]roachpb.Span, keysFromTargetBytes/keysPerRangedLock) + for i := range expectedResolvedLocksRangedTargetBytes { + expectedResolvedLocksRangedTargetBytes[i].Key = intToKey(i * keysPerRangedLock) + expectedResolvedLocksRangedTargetBytes[i].EndKey = intToKey((i + 1) * keysPerRangedLock) + } + expectedExternalLocksRangedTargetBytes := make([]roachpb.Span, ceil(numKeys, keysPerRangedLock)-keysFromTargetBytes/keysPerRangedLock) + for i := range expectedExternalLocksRangedTargetBytes { + offset := keysFromTargetBytes / keysPerRangedLock + expectedExternalLocksRangedTargetBytes[i].Key = intToKey((i + offset) * keysPerRangedLock) + expectedExternalLocksRangedTargetBytes[i].EndKey = intToKey((i + offset + 1) * keysPerRangedLock) + } + + expectedResolvedLocksNoLimit := make([]roachpb.Span, numKeys) + for i := range expectedResolvedLocksNoLimit { + expectedResolvedLocksNoLimit[i].Key = intToKey(i) + } + expectedExternalLocksNoLimit := make([]roachpb.Span, 0) + + testCases := []struct { + desc string + lockSpans []roachpb.Span + resolveAllowance int64 + targetBytes int64 + expectedResolvedLocks []roachpb.Span + expectedExternalLocks []roachpb.Span + }{ + // Point intent resolution with a max keys limit. 20 point intents, 11 + // become resolved locks and 9 become external locks. + { + desc: "Point locks with max keys", + lockSpans: pointLocks, + resolveAllowance: maxKeys, + targetBytes: 0, + expectedResolvedLocks: expectedResolvedLocksPointMaxKeys, + expectedExternalLocks: expectedExternalLocksPointMaxKeys, + }, + // Ranged intent resolution with a max keys limit. 5 ranged locks (each + // containing 4 keys), 3 become resolved locks (containing the first 2 + // locks and part of the 3rd lock) and 3 become external locks (containing + // the remaining part of the 3rd lock and the last 2 locks). Note that the + // max key limit splits in between the 3rd lock, so the resolved locks will + // contain the first part of the 3rd lock span and the external locks will + // contain the remaining part of the 3rd lock span. + { + desc: "Ranged locks with max keys", + lockSpans: rangedLocks, + resolveAllowance: maxKeys, + targetBytes: 0, + expectedResolvedLocks: expectedResolvedLocksRangedMaxKeys, + expectedExternalLocks: expectedExternalLocksRangedMaxKeys, + }, + // Point intent resolution with a target bytes limit. 20 point intents, 12 + // become resolved locks and 8 become external locks. + { + desc: "Point span with target bytes", + lockSpans: pointLocks, + resolveAllowance: 0, + targetBytes: targetBytes, + expectedResolvedLocks: expectedResolvedLocksPointTargetBytes, + expectedExternalLocks: expectedExternalLocksPointTargetBytes, + }, + // Ranged intent resolution with a target bytes limit. 5 ranged locks (each + // containing 4 keys), 3 become resolved locks (containing the first 3 + // locks) and 2 become external locks (containing the last 2 locks). Note + // that the target byte limit does not split in between any locks, so the + // resolved and external locks do not contain part of a lock span for any + // lock. + { + desc: "Ranged span with target bytes", + lockSpans: rangedLocks, + resolveAllowance: 0, + targetBytes: targetBytes, + expectedResolvedLocks: expectedResolvedLocksRangedTargetBytes, + expectedExternalLocks: expectedExternalLocksRangedTargetBytes, + }, + // Point intent resolution without any limit. 20 point intents, 20 become + // resolved locks and 0 become external locks. + { + desc: "No key or byte limit", + lockSpans: pointLocks, + resolveAllowance: 0, + targetBytes: 0, + expectedResolvedLocks: expectedResolvedLocksNoLimit, + expectedExternalLocks: expectedExternalLocksNoLimit, + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + db := storage.NewDefaultInMemForTesting() + defer db.Close() + batch := db.NewBatch() + defer batch.Close() + + ts := hlc.Timestamp{WallTime: 1} + txn := roachpb.MakeTransaction("test", roachpb.Key("a"), 0, ts, 0, 1) + txn.Status = roachpb.COMMITTED + + for i := 0; i < numKeys; i++ { + err := storage.MVCCPut(ctx, batch, nil, intToKey(i), ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("a"), &txn) + require.NoError(t, err) + } + resolvedLocks, externalLocks, err := resolveLocalLocksWithPagination( + ctx, + &roachpb.RangeDescriptor{ + StartKey: roachpb.RKeyMin, + EndKey: roachpb.RKeyMax, + }, + batch, + nil, + &kvpb.EndTxnRequest{ + LockSpans: tc.lockSpans, + InternalCommitTrigger: &roachpb.InternalCommitTrigger{}, + }, + &txn, + (&MockEvalCtx{}).EvalContext(), + tc.resolveAllowance, + tc.targetBytes, + ) + require.NoError(t, err) + require.Equal(t, len(tc.expectedResolvedLocks), len(resolvedLocks)) + for i, lock := range resolvedLocks { + require.Equal(t, tc.expectedResolvedLocks[i].Key, lock.Key) + require.Equal(t, tc.expectedResolvedLocks[i].EndKey, lock.EndKey) + } + require.Equal(t, len(tc.expectedExternalLocks), len(externalLocks)) + for i, lock := range externalLocks { + require.Equal(t, tc.expectedExternalLocks[i].Key, lock.Key) + require.Equal(t, tc.expectedExternalLocks[i].EndKey, lock.EndKey) + } + }) + } +} diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_integration_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_integration_test.go index 54cd47f579c5..ceddf1dec869 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_integration_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_integration_test.go @@ -162,6 +162,101 @@ func TestAsyncIntentResolutionByteSizePagination(t *testing.T) { }) } +// TestEndTxnByteSizePagination tests that EndTxn has byte size pagination. +// This is done by creating a transaction where the total bytes of the write +// values exceeds the max raft command size and updating the transaction +// timestamp to ensure the key values are written to the raft command during +// intent resolution. EndTxn will synchronously resolve the intents and the +// write batch size from intent resolution will exceed the max raft command +// size resulting in an error and no intents will be resolved, unless byte size +// pagination is implemented. +func TestEndTxnByteSizePagination(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + // Start test cluster. + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + IntentResolverKnobs: kvserverbase.IntentResolverTestingKnobs{ + DisableAsyncIntentResolution: true, + }, + }, + }, + }, + } + tc := testcluster.StartTestCluster(t, 1, clusterArgs) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + const numIntents = 7 + + { + _, err := db.Exec("CREATE TABLE t (i INT PRIMARY KEY, j STRING)") + require.NoError(t, err) + } + + // Set the max raft command size to 5MB. + st := tc.Servers[0].ClusterSettings() + st.Manual.Store(true) + kvserverbase.MaxCommandSize.Override(ctx, &st.SV, 5<<20) + + { + // Insert kv pairs whose values exceed max raft command size = 5MB in + // total. + tx, err := db.Begin() + require.NoError(t, err) + for i := 0; i < numIntents-1; i++ { + _, err = tx.Exec(fmt.Sprintf("INSERT INTO t (i, j) VALUES (%d, '%01000000d')", i, i)) + require.NoError(t, err) + } + + // Create a later transaction that writes to key numIntents-1. + { + tx2, err := db.Begin() + require.NoError(t, err) + _, err = tx2.Exec(fmt.Sprintf("INSERT INTO t (i, j) VALUES (%d, '0')", numIntents-1)) + require.NoError(t, err) + err = tx2.Commit() + require.NoError(t, err) + } + + // Have the first transaction write to key numIntents-1, which will force + // the transaction to update its timestamp. + _, err = tx.Exec(fmt.Sprintf("UPDATE t SET j = '1' WHERE i = %d", numIntents-1)) + require.NoError(t, err) + + // Commit, which will call EndTxn and synchronously resolve the intents, + // and the write batch size from intent resolution will exceed the max raft + // command size resulting in an error and no intents will be resolved, + // unless byte size pagination is implemented. Below, we check that at + // least 1 intent has been resolved. + err = tx.Commit() + require.NoError(t, err) + } + + // Get the store, start key, and end key of the range containing table t. + startKey, endKey, store := getRangeInfoForTable(ctx, t, db, tc.Servers, "t") + + // Check that at least 1 intent has been resolved to ensure synchronous + // intent resolution did not exceed the max raft command size, which can only + // happen if byte size pagination was implemented. + testutils.SucceedsSoon(t, func() error { + result, err := storage.MVCCScan(ctx, store.TODOEngine(), startKey, endKey, + hlc.MaxTimestamp, storage.MVCCScanOptions{Inconsistent: true}) + if err != nil { + return err + } + if intentCount := len(result.Intents); intentCount == numIntents { + return errors.Errorf("Expected fewer than %d unresolved intents, got %d", numIntents, intentCount) + } + return nil + }) +} + // TestIntentResolutionUnavailableRange tests that InFlightBackpressureLimit // resolve intent batches for an unavailable range does not stall indefinitely // and times out, allowing other resolve intent requests and queries for diff --git a/pkg/server/admin.go b/pkg/server/admin.go index a0ebe8f15e45..e6c4a4f03464 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -670,7 +670,7 @@ func (s *adminServer) getDatabaseTableSpans( if err != nil { return nil, err } - tableSpans[tableName] = generateTableSpan(tableID) + tableSpans[tableName] = generateTableSpan(tableID, s.sqlServer.execCfg.Codec) } return tableSpans, nil } @@ -1161,7 +1161,7 @@ func (s *adminServer) tableDetailsHelper( // Get the number of ranges in the table. We get the key span for the table // data. Then, we count the number of ranges that make up that key span. { - tableSpan := generateTableSpan(tableID) + tableSpan := generateTableSpan(tableID, s.sqlServer.execCfg.Codec) tableRSpan, err := keys.SpanAddr(tableSpan) if err != nil { return nil, err @@ -1190,8 +1190,8 @@ func (s *adminServer) tableDetailsHelper( // // NOTE: this doesn't make sense for interleaved (children) table. As of // 03/2018, callers around here use it anyway. -func generateTableSpan(tableID descpb.ID) roachpb.Span { - tableStartKey := keys.TODOSQLCodec.TablePrefix(uint32(tableID)) +func generateTableSpan(tableID descpb.ID, codec keys.SQLCodec) roachpb.Span { + tableStartKey := codec.TablePrefix(uint32(tableID)) tableEndKey := tableStartKey.PrefixEnd() return roachpb.Span{Key: tableStartKey, EndKey: tableEndKey} } @@ -1223,7 +1223,7 @@ func (s *adminServer) TableStats( if err != nil { return nil, serverError(ctx, err) } - tableSpan := generateTableSpan(tableID) + tableSpan := generateTableSpan(tableID, s.sqlServer.execCfg.Codec) r, err := s.statsForSpan(ctx, tableSpan) if err != nil { diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index 15bc475ed4c4..18f62cb736be 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -3707,7 +3707,7 @@ func TestTransactionContentionEvents(t *testing.T) { WHERE length(contending_key) > 0`, ) if tc.testName == "nopermission" { - require.Contains(t, err.Error(), "requires VIEWACTIVITY") + require.Contains(t, err.Error(), "does not have VIEWACTIVITY") } else { require.NoError(t, err) visibleContendingKeysCount := tree.MustBeDInt(row[0]) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index d8ee2dd1e440..e5f5fb9b71ef 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -166,6 +166,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalLocalSessionsTableID: crdbInternalLocalSessionsTable, catconstants.CrdbInternalLocalMetricsTableID: crdbInternalLocalMetricsTable, catconstants.CrdbInternalNodeExecutionInsightsTableID: crdbInternalNodeExecutionInsightsTable, + catconstants.CrdbInternalNodeMemoryMonitorsTableID: crdbInternalNodeMemoryMonitors, catconstants.CrdbInternalNodeStmtStatsTableID: crdbInternalNodeStmtStatsTable, catconstants.CrdbInternalNodeTxnExecutionInsightsTableID: crdbInternalNodeTxnExecutionInsightsTable, catconstants.CrdbInternalNodeTxnStatsTableID: crdbInternalNodeTxnStatsTable, @@ -1417,8 +1418,7 @@ CREATE TABLE crdb_internal.node_statement_statistics ( return err } if !hasViewActivityOrViewActivityRedacted { - return pgerror.Newf(pgcode.InsufficientPrivilege, - "user %s does not have %s or %s privilege", p.User(), roleoption.VIEWACTIVITY, roleoption.VIEWACTIVITYREDACTED) + return noViewActivityOrViewActivityRedactedRoleError(p.User()) } sqlStats, err := getSQLStats(p, "crdb_internal.node_statement_statistics") @@ -1643,8 +1643,7 @@ CREATE TABLE crdb_internal.node_transaction_statistics ( return err } if !hasViewActivityOrhasViewActivityRedacted { - return pgerror.Newf(pgcode.InsufficientPrivilege, - "user %s does not have %s or %s privilege", p.User(), roleoption.VIEWACTIVITY, roleoption.VIEWACTIVITYREDACTED) + return noViewActivityOrViewActivityRedactedRoleError(p.User()) } sqlStats, err := getSQLStats(p, "crdb_internal.node_transaction_statistics") @@ -6596,8 +6595,7 @@ CREATE TABLE crdb_internal.transaction_contention_events ( return nil, nil, err } if !hasPermission { - return nil, nil, errors.New("crdb_internal.transaction_contention_events " + - "requires VIEWACTIVITY or VIEWACTIVITYREDACTED role option") + return nil, nil, noViewActivityOrViewActivityRedactedRoleError(p.User()) } // If a user has VIEWACTIVITYREDACTED role option but the user does not @@ -6881,8 +6879,7 @@ func genClusterLocksGenerator( return nil, nil, err } if !hasViewActivityOrViewActivityRedacted { - return nil, nil, pgerror.Newf(pgcode.InsufficientPrivilege, - "user %s does not have %s or %s privilege", p.User(), roleoption.VIEWACTIVITY, roleoption.VIEWACTIVITYREDACTED) + return nil, nil, noViewActivityOrViewActivityRedactedRoleError(p.User()) } shouldRedactKeys := false if !hasAdmin { @@ -7197,13 +7194,7 @@ func populateTxnExecutionInsights( return err } if !hasRoleOption { - return pgerror.Newf( - pgcode.InsufficientPrivilege, - "user %s does not have %s or %s privilege", - p.User(), - roleoption.VIEWACTIVITY, - roleoption.VIEWACTIVITYREDACTED, - ) + return noViewActivityOrViewActivityRedactedRoleError(p.User()) } response, err := p.extendedEvalCtx.SQLStatusServer.ListExecutionInsights(ctx, request) @@ -7353,6 +7344,16 @@ var crdbInternalNodeExecutionInsightsTable = virtualSchemaTable{ }, } +func noViewActivityOrViewActivityRedactedRoleError(user username.SQLUsername) error { + return pgerror.Newf( + pgcode.InsufficientPrivilege, + "user %s does not have %s or %s privilege", + user, + roleoption.VIEWACTIVITY, + roleoption.VIEWACTIVITYREDACTED, + ) +} + func populateStmtInsights( ctx context.Context, p *planner, @@ -7364,13 +7365,7 @@ func populateStmtInsights( return err } if !hasRoleOption { - return pgerror.Newf( - pgcode.InsufficientPrivilege, - "user %s does not have %s or %s privilege", - p.User(), - roleoption.VIEWACTIVITY, - roleoption.VIEWACTIVITYREDACTED, - ) + return noViewActivityOrViewActivityRedactedRoleError(p.User()) } response, err := p.extendedEvalCtx.SQLStatusServer.ListExecutionInsights(ctx, request) @@ -7501,3 +7496,42 @@ func getContentionEventInfo( return schemaDesc.GetName(), dbDesc.GetName(), tableDesc.GetName(), idxName, nil } + +var crdbInternalNodeMemoryMonitors = virtualSchemaTable{ + comment: `node-level table listing all currently active memory monitors`, + schema: ` +CREATE TABLE crdb_internal.node_memory_monitors ( + level INT8, + name STRING, + id INT8, + parent_id INT8, + used INT8, + reserved_used INT8, + reserved_reserved INT8 +);`, + populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + // The memory monitors' names can expose some information about the + // activity on the node, so we require VIEWACTIVITY or + // VIEWACTIVITYREDACTED permissions. + hasRoleOption, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return err + } + if !hasRoleOption { + return noViewActivityOrViewActivityRedactedRoleError(p.User()) + } + + monitorStateCb := func(monitor mon.MonitorState) error { + return addRow( + tree.NewDInt(tree.DInt(monitor.Level)), + tree.NewDString(monitor.Name), + tree.NewDInt(tree.DInt(monitor.ID)), + tree.NewDInt(tree.DInt(monitor.ParentID)), + tree.NewDInt(tree.DInt(monitor.Used)), + tree.NewDInt(tree.DInt(monitor.ReservedUsed)), + tree.NewDInt(tree.DInt(monitor.ReservedReserved)), + ) + } + return p.extendedEvalCtx.ExecCfg.RootMemoryMonitor.TraverseTree(monitorStateCb) + }, +} diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 305528eb34af..657f1e2c2cad 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -268,7 +268,7 @@ func (ds *ServerImpl) setupFlow( } monitor = mon.NewMonitor( - "flow", + "flow "+redact.RedactableString(req.Flow.FlowID.Short()), mon.MemoryResource, ds.Metrics.CurBytesCount, ds.Metrics.MaxBytesHist, diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index 07366eef4a77..46ffbe553412 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -3732,9 +3732,9 @@ SELECT crdb_internal.hide_sql_constants('select _, _, _') SELECT _, _, _ query T -SELECT crdb_internal.hide_sql_constants(ARRAY('select 1', NULL, 'select ''hello''', '')) +SELECT crdb_internal.hide_sql_constants(ARRAY('select 1', NULL, 'select ''hello''', '', 'not a sql stmt')) ---- -{"SELECT _",NULL,"SELECT '_'",""} +{"SELECT _",NULL,"SELECT '_'","",""} query T SELECT crdb_internal.hide_sql_constants('SELECT ''yes'' IN (''no'', ''maybe'', ''yes'')') diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 8b08085f5cd0..60c9ed50ffa8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -66,6 +66,7 @@ crdb_internal node_contention_events table admin NULL NULL crdb_internal node_distsql_flows table admin NULL NULL crdb_internal node_execution_insights table admin NULL NULL crdb_internal node_inflight_trace_spans table admin NULL NULL +crdb_internal node_memory_monitors table admin NULL NULL crdb_internal node_metrics table admin NULL NULL crdb_internal node_queries table admin NULL NULL crdb_internal node_runtime_info table admin NULL NULL @@ -1211,3 +1212,40 @@ user root statement ok REVOKE SYSTEM MODIFYCLUSTERSETTING FROM testuser + +query TT +SELECT crdb_internal.humanize_bytes(NULL), crdb_internal.humanize_bytes(102400) +---- +NULL 100 KiB + +# Sanity checks of the crdb_internal.node_memory_monitors table. + +# The root monitor has a standalone budget, so its 'used' field is zero. +query BB +SELECT used = 0, reserved_used > 0 FROM crdb_internal.node_memory_monitors WHERE name = 'root' +---- +true true + +query B +SELECT used > 0 FROM crdb_internal.node_memory_monitors WHERE name = 'sql' +---- +true + +# Run a query on one connection and observe it from another. +user testuser + +statement async sleepQuery count 1 +SELECT pg_sleep(3) + +user root + +query B +SELECT count(*) > 0 FROM crdb_internal.node_memory_monitors WHERE name LIKE '%flow%' +---- +true + +user testuser + +awaitstatement sleepQuery + +user root diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index d03f5961fd40..964015cb806a 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -165,257 +165,258 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 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": "9", "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', 'families']), ARRAY['table', 'nextFamilyId']), ARRAY['table', 'indexes', '0', 'createdAtNanos']), ARRAY['table', 'indexes', '1', 'createdAtNanos']), ARRAY['table', 'indexes', '2', 'createdAtNanos']), ARRAY['table', 'primaryIndex', 'createdAtNanos']), ARRAY['table', 'createAsOfTime']), ARRAY['table', 'modificationTime']), ARRAY['function', 'modificationTime']), ARRAY['type', 'modificationTime']), ARRAY['schema', 'modificationTime']), ARRAY['database', 'modificationTime']);", "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": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966993 {"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": 4294966993, "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": 4294966996, "version": "1"}} -4294966994 {"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": 4294966994, "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": 4294966996, "version": "1"}} -4294966995 {"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": 4294966995, "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": 4294966996, "version": "1"}} -4294966996 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966996, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294966997 {"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": 4294966997, "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": 4294967126, "version": "1"}} -4294966998 {"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": 4294966998, "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": 4294967126, "version": "1"}} -4294966999 {"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": 4294966999, "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": 4294967126, "version": "1"}} -4294967000 {"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": 4294967000, "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": 4294967126, "version": "1"}} -4294967001 {"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": 4294967001, "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": 4294967126, "version": "1"}} -4294967002 {"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": 4294967002, "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": 4294967126, "version": "1"}} -4294967003 {"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": 4294967003, "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": 4294967126, "version": "1"}} -4294967004 {"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": 4294967004, "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": 4294967126, "version": "1"}} -4294967005 {"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": 4294967005, "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": 4294967126, "version": "1"}} -4294967006 {"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": 4294967006, "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": 4294967126, "version": "1"}} -4294967007 {"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": 4294967007, "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": 4294967126, "version": "1"}} -4294967008 {"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": 4294967008, "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": 4294967126, "version": "1"}} -4294967009 {"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": 4294967009, "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": 4294967126, "version": "1"}} -4294967010 {"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": 4294967010, "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": 4294967126, "version": "1"}} -4294967011 {"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": 4294967011, "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": 4294967126, "version": "1"}} -4294967012 {"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": 4294967012, "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": 4294967126, "version": "1"}} -4294967013 {"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": 4294967013, "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": 4294967126, "version": "1"}} -4294967014 {"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": 4294967014, "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": 4294967126, "version": "1"}} -4294967015 {"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": 4294967015, "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": 4294967126, "version": "1"}} -4294967016 {"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": 4294967016, "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": 4294967126, "version": "1"}} -4294967017 {"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": 4294967017, "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": 4294967126, "version": "1"}} -4294967018 {"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": 4294967018, "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": 4294967126, "version": "1"}} -4294967019 {"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": 4294967019, "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": 4294967126, "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": "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": 4294967020, "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": 4294967126, "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": "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": 4294967021, "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": 4294967126, "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_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": 4294967022, "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": 4294967126, "version": "1"}} -4294967023 {"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": 4294967023, "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": 4294967126, "version": "1"}} -4294967024 {"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": 4294967024, "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": 4294967126, "version": "1"}} -4294967025 {"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": 4294967025, "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": 4294967126, "version": "1"}} -4294967026 {"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": 4294967026, "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": 4294967126, "version": "1"}} -4294967027 {"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": 4294967027, "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": 4294967126, "version": "1"}} -4294967028 {"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": 4294967028, "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": 4294967126, "version": "1"}} -4294967029 {"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": 4294967029, "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": 4294967126, "version": "1"}} -4294967030 {"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": 4294967030, "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": 4294967126, "version": "1"}} -4294967031 {"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": 4294967031, "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": 4294967126, "version": "1"}} -4294967032 {"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": 4294967032, "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": 4294967126, "version": "1"}} -4294967033 {"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": 4294967033, "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": 4294967126, "version": "1"}} -4294967034 {"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": 4294967034, "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": 4294967126, "version": "1"}} -4294967035 {"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": 4294967035, "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": 4294967126, "version": "1"}} -4294967036 {"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": 4294967036, "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": 4294967126, "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_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": 4294967126, "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_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": 4294967126, "version": "1"}} -4294967039 {"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": 4294967039, "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": 4294967126, "version": "1"}} -4294967040 {"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": 4294967040, "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": 4294967126, "version": "1"}} -4294967041 {"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": 4294967041, "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": 4294967126, "version": "1"}} -4294967042 {"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": 4294967042, "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": 4294967126, "version": "1"}} -4294967043 {"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": 4294967043, "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": 4294967126, "version": "1"}} -4294967044 {"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": 4294967044, "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": 4294967126, "version": "1"}} -4294967045 {"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": 4294967045, "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": 4294967126, "version": "1"}} -4294967046 {"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": 4294967046, "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": 4294967126, "version": "1"}} -4294967047 {"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": 4294967047, "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": 4294967126, "version": "1"}} -4294967048 {"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": 4294967048, "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": 4294967126, "version": "1"}} -4294967049 {"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": 4294967049, "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": 4294967126, "version": "1"}} -4294967050 {"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": 4294967050, "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": 4294967126, "version": "1"}} -4294967051 {"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": 4294967051, "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": 4294967126, "version": "1"}} -4294967052 {"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": 4294967052, "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": 4294967126, "version": "1"}} -4294967053 {"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": 4294967053, "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": 4294967126, "version": "1"}} -4294967054 {"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": 4294967054, "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": 4294967126, "version": "1"}} -4294967055 {"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": 4294967055, "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": 4294967126, "version": "1"}} -4294967056 {"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": 4294967056, "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": 4294967126, "version": "1"}} -4294967057 {"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": 4294967057, "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": 4294967126, "version": "1"}} -4294967058 {"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": 4294967058, "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": 4294967126, "version": "1"}} -4294967059 {"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": 4294967059, "name": "pg_shdescription", "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": 4294967126, "version": "1"}} -4294967060 {"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": 4294967060, "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": 4294967126, "version": "1"}} -4294967061 {"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": 4294967061, "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": 4294967126, "version": "1"}} -4294967062 {"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": 4294967062, "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": 4294967126, "version": "1"}} -4294967063 {"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": 4294967063, "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": 4294967126, "version": "1"}} -4294967064 {"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": 4294967064, "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": 4294967126, "version": "1"}} -4294967065 {"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": 4294967065, "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": 4294967126, "version": "1"}} -4294967066 {"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": 4294967066, "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": 4294967126, "version": "1"}} -4294967067 {"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": 4294967067, "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": 4294967126, "version": "1"}} -4294967068 {"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": 4294967068, "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": 4294967126, "version": "1"}} -4294967069 {"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": 4294967069, "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": 4294967126, "version": "1"}} -4294967070 {"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": 4294967070, "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": 4294967126, "version": "1"}} -4294967071 {"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": 4294967071, "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": 4294967126, "version": "1"}} -4294967072 {"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": 4294967072, "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": 4294967126, "version": "1"}} -4294967073 {"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": 4294967073, "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": 4294967126, "version": "1"}} -4294967074 {"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": 4294967074, "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": 4294967126, "version": "1"}} -4294967075 {"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": 4294967075, "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": 4294967126, "version": "1"}} -4294967076 {"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": 4294967076, "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": 4294967126, "version": "1"}} -4294967077 {"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": 4294967077, "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": 4294967126, "version": "1"}} -4294967078 {"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": 4294967078, "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": 4294967126, "version": "1"}} -4294967079 {"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": 4294967079, "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": 4294967126, "version": "1"}} -4294967080 {"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": 4294967080, "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": 4294967126, "version": "1"}} -4294967081 {"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": 4294967081, "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": 4294967126, "version": "1"}} -4294967082 {"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": 4294967082, "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": 4294967126, "version": "1"}} -4294967083 {"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": 4294967083, "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": 4294967126, "version": "1"}} -4294967084 {"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": 4294967084, "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": 4294967126, "version": "1"}} -4294967085 {"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": 4294967085, "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": 4294967126, "version": "1"}} -4294967086 {"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": 4294967086, "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": 4294967126, "version": "1"}} -4294967087 {"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": 4294967087, "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": 4294967126, "version": "1"}} -4294967088 {"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": 4294967088, "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": 4294967126, "version": "1"}} -4294967089 {"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": 4294967089, "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": 4294967126, "version": "1"}} -4294967090 {"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": 4294967090, "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": 4294967126, "version": "1"}} -4294967091 {"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": 4294967091, "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": 4294967126, "version": "1"}} -4294967092 {"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": 4294967092, "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": 4294967126, "version": "1"}} -4294967093 {"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": 4294967093, "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": 4294967126, "version": "1"}} -4294967094 {"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": 4294967094, "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": 4294967126, "version": "1"}} -4294967095 {"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": 4294967095, "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": 4294967126, "version": "1"}} -4294967096 {"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": 4294967096, "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": 4294967126, "version": "1"}} -4294967097 {"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": 4294967097, "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": 4294967126, "version": "1"}} -4294967098 {"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": 4294967098, "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": 4294967126, "version": "1"}} -4294967099 {"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": 4294967099, "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": 4294967126, "version": "1"}} -4294967100 {"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": 4294967100, "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": 4294967126, "version": "1"}} -4294967101 {"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": 4294967101, "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": 4294967126, "version": "1"}} -4294967102 {"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": 4294967102, "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": 4294967126, "version": "1"}} -4294967103 {"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": 4294967103, "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": 4294967126, "version": "1"}} -4294967104 {"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": 4294967104, "name": "pg_description", "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": 4294967126, "version": "1"}} -4294967105 {"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": 4294967105, "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": 4294967126, "version": "1"}} -4294967106 {"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": 4294967106, "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": 4294967126, "version": "1"}} -4294967107 {"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": 4294967107, "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": 4294967126, "version": "1"}} -4294967108 {"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": 4294967108, "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": 4294967126, "version": "1"}} -4294967109 {"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": 4294967109, "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": 4294967126, "version": "1"}} -4294967110 {"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": 4294967110, "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": 4294967126, "version": "1"}} -4294967111 {"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": 4294967111, "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": 4294967126, "version": "1"}} -4294967112 {"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": 4294967112, "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": 4294967126, "version": "1"}} -4294967113 {"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": 4294967113, "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": 4294967126, "version": "1"}} -4294967114 {"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": 4294967114, "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": 4294967126, "version": "1"}} -4294967115 {"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": 4294967115, "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": 4294967126, "version": "1"}} -4294967116 {"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": 4294967116, "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": 4294967126, "version": "1"}} -4294967117 {"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": 4294967117, "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": 4294967126, "version": "1"}} -4294967118 {"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": 4294967118, "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": 4294967126, "version": "1"}} -4294967119 {"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": 4294967119, "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": 4294967126, "version": "1"}} -4294967120 {"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": 4294967120, "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": 4294967126, "version": "1"}} -4294967121 {"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": 4294967121, "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": 4294967126, "version": "1"}} -4294967122 {"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": 4294967122, "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": 4294967126, "version": "1"}} -4294967123 {"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": 4294967123, "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": 4294967126, "version": "1"}} -4294967124 {"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": 4294967124, "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": 4294967126, "version": "1"}} -4294967125 {"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": 4294967125, "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": 4294967126, "version": "1"}} -4294967126 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967126, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "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": "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": 4294967127, "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": 4294967213, "version": "1"}} -4294967128 {"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": 4294967128, "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": 4294967213, "version": "1"}} -4294967129 {"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": 4294967129, "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": 4294967213, "version": "1"}} -4294967130 {"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": 4294967130, "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": 4294967213, "version": "1"}} -4294967131 {"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": 4294967131, "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": 4294967213, "version": "1"}} -4294967132 {"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": 4294967132, "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": 4294967213, "version": "1"}} -4294967133 {"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": 4294967133, "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": 4294967213, "version": "1"}} -4294967134 {"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": 4294967134, "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": 4294967213, "version": "1"}} -4294967135 {"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": 4294967135, "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": 4294967213, "version": "1"}} -4294967136 {"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": 4294967136, "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": 4294967213, "version": "1"}} -4294967137 {"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": 4294967137, "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": 4294967213, "version": "1"}} -4294967138 {"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": 4294967138, "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": 4294967213, "version": "1"}} -4294967139 {"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": 4294967139, "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": 4294967213, "version": "1"}} -4294967140 {"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": 4294967140, "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": 4294967213, "version": "1"}} -4294967141 {"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": 4294967141, "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": 4294967213, "version": "1"}} -4294967142 {"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": 4294967142, "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": 4294967213, "version": "1"}} -4294967143 {"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": 4294967143, "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": 4294967213, "version": "1"}} -4294967144 {"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": 4294967144, "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": 4294967213, "version": "1"}} -4294967145 {"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": 4294967145, "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": 4294967213, "version": "1"}} -4294967146 {"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": 4294967146, "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": 4294967213, "version": "1"}} -4294967147 {"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": 4294967147, "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": 4294967213, "version": "1"}} -4294967148 {"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": 4294967148, "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": 4294967213, "version": "1"}} -4294967149 {"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}}], "formatVersion": 3, "id": 4294967149, "name": "statistics", "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": 4294967213, "version": "1"}} -4294967150 {"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": 4294967150, "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": 4294967213, "version": "1"}} -4294967151 {"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": 4294967151, "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": 4294967213, "version": "1"}} -4294967152 {"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": 4294967152, "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": 4294967213, "version": "1"}} -4294967153 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967153, "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": 4294967213, "version": "1"}} -4294967154 {"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": 4294967154, "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": 4294967213, "version": "1"}} -4294967155 {"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": 4294967155, "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": 4294967213, "version": "1"}} -4294967156 {"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": 4294967156, "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": 4294967213, "version": "1"}} -4294967157 {"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": 4294967157, "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": 4294967213, "version": "1"}} -4294967158 {"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": 4294967158, "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": 4294967213, "version": "1"}} -4294967159 {"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": 4294967159, "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": 4294967213, "version": "1"}} -4294967160 {"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": 4294967160, "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": 4294967213, "version": "1"}} -4294967161 {"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": 4294967161, "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": 4294967213, "version": "1"}} -4294967162 {"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_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967162, "name": "routines", "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": 4294967213, "version": "1"}} -4294967163 {"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": 4294967163, "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": 4294967213, "version": "1"}} -4294967164 {"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": 4294967164, "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": 4294967213, "version": "1"}} -4294967165 {"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": 4294967165, "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": 4294967213, "version": "1"}} -4294967166 {"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": 4294967166, "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": 4294967213, "version": "1"}} -4294967167 {"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": 4294967167, "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": 4294967213, "version": "1"}} -4294967168 {"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": 4294967168, "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": 4294967213, "version": "1"}} -4294967169 {"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": 4294967169, "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": 4294967213, "version": "1"}} -4294967170 {"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": 4294967170, "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": 4294967213, "version": "1"}} -4294967171 {"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": 4294967171, "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": 4294967213, "version": "1"}} -4294967172 {"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": 4294967172, "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": 4294967213, "version": "1"}} -4294967173 {"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": 4294967173, "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": 4294967213, "version": "1"}} -4294967174 {"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": 4294967174, "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": 4294967213, "version": "1"}} -4294967175 {"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": 4294967175, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967213, "version": "1"}} -4294967176 {"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": 4294967176, "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": 4294967213, "version": "1"}} -4294967177 {"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": 4294967177, "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": 4294967213, "version": "1"}} -4294967178 {"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": 4294967178, "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": 4294967213, "version": "1"}} -4294967179 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "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": 4294967213, "version": "1"}} -4294967180 {"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": 4294967180, "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": 4294967213, "version": "1"}} -4294967181 {"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": 4294967181, "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": 4294967213, "version": "1"}} -4294967182 {"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": 4294967182, "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": 4294967213, "version": "1"}} -4294967183 {"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": 4294967183, "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": 4294967213, "version": "1"}} -4294967184 {"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": 4294967184, "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": 4294967213, "version": "1"}} -4294967185 {"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": 4294967185, "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": 4294967213, "version": "1"}} -4294967186 {"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": 4294967186, "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": 4294967213, "version": "1"}} -4294967187 {"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": 4294967187, "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": 4294967213, "version": "1"}} -4294967188 {"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": 4294967188, "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": 4294967213, "version": "1"}} -4294967189 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967189, "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": 4294967213, "version": "1"}} -4294967190 {"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": 4294967190, "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": 4294967213, "version": "1"}} -4294967191 {"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": 4294967191, "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": 4294967213, "version": "1"}} -4294967192 {"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": 4294967192, "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": 4294967213, "version": "1"}} -4294967193 {"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": 4294967193, "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": 4294967213, "version": "1"}} -4294967194 {"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": 4294967194, "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": 4294967213, "version": "1"}} -4294967195 {"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": 4294967195, "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": 4294967213, "version": "1"}} -4294967196 {"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": 4294967196, "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": 4294967213, "version": "1"}} -4294967197 {"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": 4294967197, "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": 4294967213, "version": "1"}} -4294967198 {"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": 4294967198, "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": 4294967213, "version": "1"}} -4294967199 {"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": 4294967199, "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": 4294967213, "version": "1"}} -4294967200 {"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": 4294967200, "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": 4294967213, "version": "1"}} -4294967201 {"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": 4294967201, "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": 4294967213, "version": "1"}} -4294967202 {"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": 4294967202, "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": 4294967213, "version": "1"}} -4294967203 {"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": 4294967203, "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": 4294967213, "version": "1"}} -4294967204 {"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": 4294967204, "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": 4294967213, "version": "1"}} -4294967205 {"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": 4294967205, "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": 4294967213, "version": "1"}} -4294967206 {"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": 4294967206, "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": 4294967213, "version": "1"}} -4294967207 {"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": 4294967207, "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": 4294967213, "version": "1"}} -4294967208 {"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": 4294967208, "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": 4294967213, "version": "1"}} -4294967209 {"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": 4294967209, "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": 4294967213, "version": "1"}} -4294967210 {"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": 4294967210, "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": 4294967213, "version": "1"}} -4294967211 {"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": 4294967211, "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": 4294967213, "version": "1"}} -4294967212 {"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": 4294967212, "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": 4294967213, "version": "1"}} -4294967213 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967213, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294967214 {"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": 4294967214, "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"}} -4294967215 {"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": 4294967215, "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"}} -4294967216 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967216, "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"}} -4294967217 {"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}}], "formatVersion": 3, "id": 4294967217, "name": "tenant_usage_details", "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 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->>'externalIOEgressBytes')::INT8 AS total_external_io_egress_bytes FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} -4294967218 {"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": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967218, "name": "active_range_feeds", "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"}} -4294967219 {"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": 4294967219, "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"}} -4294967220 {"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": 4294967220, "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"}} -4294967221 {"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": 4294967221, "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"}} -4294967222 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967222, "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"}} -4294967223 {"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": 4294967223, "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"}} -4294967224 {"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": 4294967224, "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"}} -4294967225 {"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}}], "formatVersion": 3, "id": 4294967225, "name": "invalid_objects", "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"}} -4294967226 {"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": 4294967226, "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"}} -4294967227 {"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": 4294967227, "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"}} -4294967228 {"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": 4294967228, "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"}} -4294967229 {"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": 4294967229, "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"}} -4294967230 {"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": 4294967230, "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"}} -4294967231 {"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": 4294967231, "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"}} -4294967232 {"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": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967232, "name": "table_indexes", "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": 4294967295, "version": "1"}} -4294967233 {"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": 4294967233, "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"}} -4294967234 {"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": 4294967234, "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_agg(index_rec), 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) LEFT JOIN LATERAL ROWS FROM (unnest(index_recommendations)) AS index_rec ON true GROUP BY aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, aggregation_interval"}} -4294967235 {"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": 4294967235, "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"}} -4294967236 {"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": 4294967236, "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"}} -4294967237 {"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": 4294967237, "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"}} -4294967238 {"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": 4294967238, "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"}} -4294967239 {"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": 4294967239, "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"}} -4294967240 {"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": 4294967240, "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"}} -4294967241 {"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": 4294967241, "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"}} -4294967242 {"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": 4294967242, "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"}} -4294967243 {"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": 4294967243, "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"}} +4294966992 {"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": 4294966992, "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": 4294966995, "version": "1"}} +4294966993 {"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": 4294966993, "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": 4294966995, "version": "1"}} +4294966994 {"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": 4294966994, "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": 4294966995, "version": "1"}} +4294966995 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966995, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294966996 {"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": 4294966996, "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": 4294967125, "version": "1"}} +4294966997 {"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": 4294966997, "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": 4294967125, "version": "1"}} +4294966998 {"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": 4294966998, "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": 4294967125, "version": "1"}} +4294966999 {"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": 4294966999, "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": 4294967125, "version": "1"}} +4294967000 {"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": 4294967000, "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": 4294967125, "version": "1"}} +4294967001 {"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": 4294967001, "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": 4294967125, "version": "1"}} +4294967002 {"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": 4294967002, "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": 4294967125, "version": "1"}} +4294967003 {"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": 4294967003, "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": 4294967125, "version": "1"}} +4294967004 {"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": 4294967004, "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": 4294967125, "version": "1"}} +4294967005 {"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": 4294967005, "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": 4294967125, "version": "1"}} +4294967006 {"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": 4294967006, "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": 4294967125, "version": "1"}} +4294967007 {"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": 4294967007, "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": 4294967125, "version": "1"}} +4294967008 {"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": 4294967008, "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": 4294967125, "version": "1"}} +4294967009 {"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": 4294967009, "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": 4294967125, "version": "1"}} +4294967010 {"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": 4294967010, "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": 4294967125, "version": "1"}} +4294967011 {"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": 4294967011, "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": 4294967125, "version": "1"}} +4294967012 {"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": 4294967012, "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": 4294967125, "version": "1"}} +4294967013 {"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": 4294967013, "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": 4294967125, "version": "1"}} +4294967014 {"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": 4294967014, "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": 4294967125, "version": "1"}} +4294967015 {"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": 4294967015, "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": 4294967125, "version": "1"}} +4294967016 {"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": 4294967016, "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": 4294967125, "version": "1"}} +4294967017 {"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": 4294967017, "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": 4294967125, "version": "1"}} +4294967018 {"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": 4294967018, "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": 4294967125, "version": "1"}} +4294967019 {"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": 4294967019, "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": 4294967125, "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": "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": 4294967020, "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": 4294967125, "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_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": 4294967021, "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": 4294967125, "version": "1"}} +4294967022 {"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": 4294967022, "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": 4294967125, "version": "1"}} +4294967023 {"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": 4294967023, "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": 4294967125, "version": "1"}} +4294967024 {"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": 4294967024, "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": 4294967125, "version": "1"}} +4294967025 {"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": 4294967025, "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": 4294967125, "version": "1"}} +4294967026 {"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": 4294967026, "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": 4294967125, "version": "1"}} +4294967027 {"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": 4294967027, "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": 4294967125, "version": "1"}} +4294967028 {"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": 4294967028, "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": 4294967125, "version": "1"}} +4294967029 {"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": 4294967029, "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": 4294967125, "version": "1"}} +4294967030 {"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": 4294967030, "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": 4294967125, "version": "1"}} +4294967031 {"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": 4294967031, "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": 4294967125, "version": "1"}} +4294967032 {"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": 4294967032, "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": 4294967125, "version": "1"}} +4294967033 {"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": 4294967033, "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": 4294967125, "version": "1"}} +4294967034 {"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": 4294967034, "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": 4294967125, "version": "1"}} +4294967035 {"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": 4294967035, "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": 4294967125, "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_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": 4294967125, "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_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": 4294967125, "version": "1"}} +4294967038 {"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": 4294967038, "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": 4294967125, "version": "1"}} +4294967039 {"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": 4294967039, "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": 4294967125, "version": "1"}} +4294967040 {"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": 4294967040, "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": 4294967125, "version": "1"}} +4294967041 {"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": 4294967041, "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": 4294967125, "version": "1"}} +4294967042 {"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": 4294967042, "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": 4294967125, "version": "1"}} +4294967043 {"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": 4294967043, "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": 4294967125, "version": "1"}} +4294967044 {"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": 4294967044, "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": 4294967125, "version": "1"}} +4294967045 {"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": 4294967045, "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": 4294967125, "version": "1"}} +4294967046 {"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": 4294967046, "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": 4294967125, "version": "1"}} +4294967047 {"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": 4294967047, "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": 4294967125, "version": "1"}} +4294967048 {"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": 4294967048, "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": 4294967125, "version": "1"}} +4294967049 {"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": 4294967049, "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": 4294967125, "version": "1"}} +4294967050 {"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": 4294967050, "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": 4294967125, "version": "1"}} +4294967051 {"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": 4294967051, "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": 4294967125, "version": "1"}} +4294967052 {"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": 4294967052, "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": 4294967125, "version": "1"}} +4294967053 {"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": 4294967053, "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": 4294967125, "version": "1"}} +4294967054 {"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": 4294967054, "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": 4294967125, "version": "1"}} +4294967055 {"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": 4294967055, "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": 4294967125, "version": "1"}} +4294967056 {"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": 4294967056, "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": 4294967125, "version": "1"}} +4294967057 {"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": 4294967057, "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": 4294967125, "version": "1"}} +4294967058 {"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": 4294967058, "name": "pg_shdescription", "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": 4294967125, "version": "1"}} +4294967059 {"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": 4294967059, "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": 4294967125, "version": "1"}} +4294967060 {"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": 4294967060, "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": 4294967125, "version": "1"}} +4294967061 {"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": 4294967061, "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": 4294967125, "version": "1"}} +4294967062 {"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": 4294967062, "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": 4294967125, "version": "1"}} +4294967063 {"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": 4294967063, "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": 4294967125, "version": "1"}} +4294967064 {"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": 4294967064, "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": 4294967125, "version": "1"}} +4294967065 {"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": 4294967065, "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": 4294967125, "version": "1"}} +4294967066 {"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": 4294967066, "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": 4294967125, "version": "1"}} +4294967067 {"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": 4294967067, "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": 4294967125, "version": "1"}} +4294967068 {"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": 4294967068, "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": 4294967125, "version": "1"}} +4294967069 {"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": 4294967069, "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": 4294967125, "version": "1"}} +4294967070 {"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": 4294967070, "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": 4294967125, "version": "1"}} +4294967071 {"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": 4294967071, "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": 4294967125, "version": "1"}} +4294967072 {"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": 4294967072, "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": 4294967125, "version": "1"}} +4294967073 {"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": 4294967073, "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": 4294967125, "version": "1"}} +4294967074 {"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": 4294967074, "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": 4294967125, "version": "1"}} +4294967075 {"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": 4294967075, "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": 4294967125, "version": "1"}} +4294967076 {"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": 4294967076, "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": 4294967125, "version": "1"}} +4294967077 {"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": 4294967077, "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": 4294967125, "version": "1"}} +4294967078 {"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": 4294967078, "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": 4294967125, "version": "1"}} +4294967079 {"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": 4294967079, "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": 4294967125, "version": "1"}} +4294967080 {"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": 4294967080, "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": 4294967125, "version": "1"}} +4294967081 {"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": 4294967081, "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": 4294967125, "version": "1"}} +4294967082 {"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": 4294967082, "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": 4294967125, "version": "1"}} +4294967083 {"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": 4294967083, "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": 4294967125, "version": "1"}} +4294967084 {"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": 4294967084, "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": 4294967125, "version": "1"}} +4294967085 {"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": 4294967085, "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": 4294967125, "version": "1"}} +4294967086 {"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": 4294967086, "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": 4294967125, "version": "1"}} +4294967087 {"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": 4294967087, "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": 4294967125, "version": "1"}} +4294967088 {"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": 4294967088, "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": 4294967125, "version": "1"}} +4294967089 {"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": 4294967089, "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": 4294967125, "version": "1"}} +4294967090 {"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": 4294967090, "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": 4294967125, "version": "1"}} +4294967091 {"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": 4294967091, "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": 4294967125, "version": "1"}} +4294967092 {"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": 4294967092, "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": 4294967125, "version": "1"}} +4294967093 {"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": 4294967093, "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": 4294967125, "version": "1"}} +4294967094 {"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": 4294967094, "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": 4294967125, "version": "1"}} +4294967095 {"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": 4294967095, "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": 4294967125, "version": "1"}} +4294967096 {"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": 4294967096, "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": 4294967125, "version": "1"}} +4294967097 {"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": 4294967097, "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": 4294967125, "version": "1"}} +4294967098 {"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": 4294967098, "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": 4294967125, "version": "1"}} +4294967099 {"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": 4294967099, "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": 4294967125, "version": "1"}} +4294967100 {"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": 4294967100, "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": 4294967125, "version": "1"}} +4294967101 {"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": 4294967101, "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": 4294967125, "version": "1"}} +4294967102 {"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": 4294967102, "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": 4294967125, "version": "1"}} +4294967103 {"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": 4294967103, "name": "pg_description", "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": 4294967125, "version": "1"}} +4294967104 {"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": 4294967104, "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": 4294967125, "version": "1"}} +4294967105 {"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": 4294967105, "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": 4294967125, "version": "1"}} +4294967106 {"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": 4294967106, "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": 4294967125, "version": "1"}} +4294967107 {"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": 4294967107, "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": 4294967125, "version": "1"}} +4294967108 {"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": 4294967108, "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": 4294967125, "version": "1"}} +4294967109 {"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": 4294967109, "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": 4294967125, "version": "1"}} +4294967110 {"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": 4294967110, "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": 4294967125, "version": "1"}} +4294967111 {"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": 4294967111, "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": 4294967125, "version": "1"}} +4294967112 {"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": 4294967112, "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": 4294967125, "version": "1"}} +4294967113 {"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": 4294967113, "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": 4294967125, "version": "1"}} +4294967114 {"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": 4294967114, "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": 4294967125, "version": "1"}} +4294967115 {"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": 4294967115, "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": 4294967125, "version": "1"}} +4294967116 {"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": 4294967116, "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": 4294967125, "version": "1"}} +4294967117 {"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": 4294967117, "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": 4294967125, "version": "1"}} +4294967118 {"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": 4294967118, "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": 4294967125, "version": "1"}} +4294967119 {"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": 4294967119, "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": 4294967125, "version": "1"}} +4294967120 {"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": 4294967120, "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": 4294967125, "version": "1"}} +4294967121 {"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": 4294967121, "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": 4294967125, "version": "1"}} +4294967122 {"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": 4294967122, "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": 4294967125, "version": "1"}} +4294967123 {"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": 4294967123, "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": 4294967125, "version": "1"}} +4294967124 {"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": 4294967124, "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": 4294967125, "version": "1"}} +4294967125 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967125, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967126 {"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": 4294967126, "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": 4294967212, "version": "1"}} +4294967127 {"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": 4294967127, "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": 4294967212, "version": "1"}} +4294967128 {"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": 4294967128, "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": 4294967212, "version": "1"}} +4294967129 {"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": 4294967129, "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": 4294967212, "version": "1"}} +4294967130 {"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": 4294967130, "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": 4294967212, "version": "1"}} +4294967131 {"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": 4294967131, "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": 4294967212, "version": "1"}} +4294967132 {"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": 4294967132, "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": 4294967212, "version": "1"}} +4294967133 {"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": 4294967133, "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": 4294967212, "version": "1"}} +4294967134 {"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": 4294967134, "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": 4294967212, "version": "1"}} +4294967135 {"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": 4294967135, "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": 4294967212, "version": "1"}} +4294967136 {"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": 4294967136, "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": 4294967212, "version": "1"}} +4294967137 {"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": 4294967137, "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": 4294967212, "version": "1"}} +4294967138 {"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": 4294967138, "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": 4294967212, "version": "1"}} +4294967139 {"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": 4294967139, "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": 4294967212, "version": "1"}} +4294967140 {"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": 4294967140, "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": 4294967212, "version": "1"}} +4294967141 {"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": 4294967141, "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": 4294967212, "version": "1"}} +4294967142 {"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": 4294967142, "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": 4294967212, "version": "1"}} +4294967143 {"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": 4294967143, "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": 4294967212, "version": "1"}} +4294967144 {"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": 4294967144, "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": 4294967212, "version": "1"}} +4294967145 {"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": 4294967145, "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": 4294967212, "version": "1"}} +4294967146 {"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": 4294967146, "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": 4294967212, "version": "1"}} +4294967147 {"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": 4294967147, "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": 4294967212, "version": "1"}} +4294967148 {"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}}], "formatVersion": 3, "id": 4294967148, "name": "statistics", "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": 4294967212, "version": "1"}} +4294967149 {"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": 4294967149, "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": 4294967212, "version": "1"}} +4294967150 {"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": 4294967150, "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": 4294967212, "version": "1"}} +4294967151 {"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": 4294967151, "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": 4294967212, "version": "1"}} +4294967152 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "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": 4294967212, "version": "1"}} +4294967153 {"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": 4294967153, "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": 4294967212, "version": "1"}} +4294967154 {"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": 4294967154, "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": 4294967212, "version": "1"}} +4294967155 {"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": 4294967155, "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": 4294967212, "version": "1"}} +4294967156 {"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": 4294967156, "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": 4294967212, "version": "1"}} +4294967157 {"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": 4294967157, "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": 4294967212, "version": "1"}} +4294967158 {"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": 4294967158, "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": 4294967212, "version": "1"}} +4294967159 {"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": 4294967159, "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": 4294967212, "version": "1"}} +4294967160 {"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": 4294967160, "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": 4294967212, "version": "1"}} +4294967161 {"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_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967161, "name": "routines", "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": 4294967212, "version": "1"}} +4294967162 {"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": 4294967162, "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": 4294967212, "version": "1"}} +4294967163 {"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": 4294967163, "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": 4294967212, "version": "1"}} +4294967164 {"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": 4294967164, "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": 4294967212, "version": "1"}} +4294967165 {"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": 4294967165, "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": 4294967212, "version": "1"}} +4294967166 {"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": 4294967166, "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": 4294967212, "version": "1"}} +4294967167 {"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": 4294967167, "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": 4294967212, "version": "1"}} +4294967168 {"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": 4294967168, "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": 4294967212, "version": "1"}} +4294967169 {"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": 4294967169, "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": 4294967212, "version": "1"}} +4294967170 {"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": 4294967170, "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": 4294967212, "version": "1"}} +4294967171 {"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": 4294967171, "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": 4294967212, "version": "1"}} +4294967172 {"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": 4294967172, "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": 4294967212, "version": "1"}} +4294967173 {"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": 4294967173, "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": 4294967212, "version": "1"}} +4294967174 {"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": 4294967174, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967212, "version": "1"}} +4294967175 {"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": 4294967175, "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": 4294967212, "version": "1"}} +4294967176 {"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": 4294967176, "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": 4294967212, "version": "1"}} +4294967177 {"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": 4294967177, "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": 4294967212, "version": "1"}} +4294967178 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967178, "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": 4294967212, "version": "1"}} +4294967179 {"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": 4294967179, "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": 4294967212, "version": "1"}} +4294967180 {"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": 4294967180, "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": 4294967212, "version": "1"}} +4294967181 {"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": 4294967181, "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": 4294967212, "version": "1"}} +4294967182 {"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": 4294967182, "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": 4294967212, "version": "1"}} +4294967183 {"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": 4294967183, "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": 4294967212, "version": "1"}} +4294967184 {"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": 4294967184, "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": 4294967212, "version": "1"}} +4294967185 {"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": 4294967185, "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": 4294967212, "version": "1"}} +4294967186 {"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": 4294967186, "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": 4294967212, "version": "1"}} +4294967187 {"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": 4294967187, "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": 4294967212, "version": "1"}} +4294967188 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967188, "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": 4294967212, "version": "1"}} +4294967189 {"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": 4294967189, "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": 4294967212, "version": "1"}} +4294967190 {"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": 4294967190, "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": 4294967212, "version": "1"}} +4294967191 {"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": 4294967191, "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": 4294967212, "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": "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": 4294967192, "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": 4294967212, "version": "1"}} +4294967193 {"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": 4294967193, "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": 4294967212, "version": "1"}} +4294967194 {"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": 4294967194, "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": 4294967212, "version": "1"}} +4294967195 {"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": 4294967195, "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": 4294967212, "version": "1"}} +4294967196 {"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": 4294967196, "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": 4294967212, "version": "1"}} +4294967197 {"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": 4294967197, "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": 4294967212, "version": "1"}} +4294967198 {"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": 4294967198, "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": 4294967212, "version": "1"}} +4294967199 {"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": 4294967199, "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": 4294967212, "version": "1"}} +4294967200 {"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": 4294967200, "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": 4294967212, "version": "1"}} +4294967201 {"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": 4294967201, "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": 4294967212, "version": "1"}} +4294967202 {"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": 4294967202, "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": 4294967212, "version": "1"}} +4294967203 {"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": 4294967203, "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": 4294967212, "version": "1"}} +4294967204 {"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": 4294967204, "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": 4294967212, "version": "1"}} +4294967205 {"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": 4294967205, "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": 4294967212, "version": "1"}} +4294967206 {"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": 4294967206, "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": 4294967212, "version": "1"}} +4294967207 {"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": 4294967207, "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": 4294967212, "version": "1"}} +4294967208 {"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": 4294967208, "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": 4294967212, "version": "1"}} +4294967209 {"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": 4294967209, "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": 4294967212, "version": "1"}} +4294967210 {"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": 4294967210, "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": 4294967212, "version": "1"}} +4294967211 {"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": 4294967211, "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": 4294967212, "version": "1"}} +4294967212 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967212, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967213 {"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": 4294967213, "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"}} +4294967214 {"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": 4294967214, "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"}} +4294967215 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967215, "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"}} +4294967216 {"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}}], "formatVersion": 3, "id": 4294967216, "name": "tenant_usage_details", "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 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->>'externalIOEgressBytes')::INT8 AS total_external_io_egress_bytes FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} +4294967217 {"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": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967217, "name": "active_range_feeds", "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"}} +4294967218 {"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": 4294967218, "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"}} +4294967219 {"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": 4294967219, "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"}} +4294967220 {"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": 4294967220, "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"}} +4294967221 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967221, "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"}} +4294967222 {"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": 4294967222, "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"}} +4294967223 {"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": 4294967223, "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"}} +4294967224 {"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}}], "formatVersion": 3, "id": 4294967224, "name": "invalid_objects", "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"}} +4294967225 {"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": 4294967225, "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"}} +4294967226 {"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": 4294967226, "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"}} +4294967227 {"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": 4294967227, "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"}} +4294967228 {"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": 4294967228, "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"}} +4294967229 {"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": 4294967229, "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"}} +4294967230 {"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": 4294967230, "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"}} +4294967231 {"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": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967231, "name": "table_indexes", "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": 4294967295, "version": "1"}} +4294967232 {"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": 4294967232, "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"}} +4294967233 {"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": 4294967233, "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_agg(index_rec), 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) LEFT JOIN LATERAL ROWS FROM (unnest(index_recommendations)) AS index_rec ON true GROUP BY aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, aggregation_interval"}} +4294967234 {"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": 4294967234, "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"}} +4294967235 {"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": 4294967235, "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"}} +4294967236 {"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": 4294967236, "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"}} +4294967237 {"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": 4294967237, "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"}} +4294967238 {"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": 4294967238, "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"}} +4294967239 {"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": 4294967239, "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"}} +4294967240 {"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": 4294967240, "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"}} +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": "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": 4294967241, "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"}} +4294967242 {"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": 4294967242, "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"}} +4294967243 {"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": 4294967243, "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"}} 4294967244 {"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": 4294967244, "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"}} 4294967245 {"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": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "active_query_start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"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": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967245, "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"}} 4294967246 {"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}}], "formatVersion": 3, "id": 4294967246, "name": "node_transactions", "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"}} @@ -559,251 +560,252 @@ FROM "".crdb_internal.kv_catalog_comments ---- DatabaseCommentType 104 0 "this is the test database" TableCommentType 111 0 "this is a table" -TableCommentType 4294966993 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." -TableCommentType 4294966994 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." -TableCommentType 4294966995 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." -TableCommentType 4294966997 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" -TableCommentType 4294966998 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" -TableCommentType 4294966999 0 "pg_user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967000 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" -TableCommentType 4294967001 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" -TableCommentType 4294967002 0 "pg_ts_template was created for compatibility and is currently unimplemented" -TableCommentType 4294967003 0 "pg_ts_parser was created for compatibility and is currently unimplemented" -TableCommentType 4294967004 0 "pg_ts_dict was created for compatibility and is currently unimplemented" -TableCommentType 4294967005 0 "pg_ts_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967006 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" -TableCommentType 4294967007 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" -TableCommentType 4294967008 0 "pg_transform was created for compatibility and is currently unimplemented" -TableCommentType 4294967009 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" -TableCommentType 4294967010 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" -TableCommentType 4294967011 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" -TableCommentType 4294967012 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" -TableCommentType 4294967013 0 "pg_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967014 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967015 0 "pg_stats was created for compatibility and is currently unimplemented" -TableCommentType 4294967016 0 "pg_stats_ext was created for compatibility and is currently unimplemented" -TableCommentType 4294967017 0 "pg_statistic was created for compatibility and is currently unimplemented" -TableCommentType 4294967018 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" -TableCommentType 4294967019 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" -TableCommentType 4294967020 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967021 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967022 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967023 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967024 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967025 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967026 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967027 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967028 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967029 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967030 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967031 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967032 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967033 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967034 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967035 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967036 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967037 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967038 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967039 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967040 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" -TableCommentType 4294967041 0 "pg_stat_slru was created for compatibility and is currently unimplemented" -TableCommentType 4294967042 0 "pg_stat_replication was created for compatibility and is currently unimplemented" -TableCommentType 4294967043 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" -TableCommentType 4294967044 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" -TableCommentType 4294967045 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" -TableCommentType 4294967046 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" -TableCommentType 4294967047 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" -TableCommentType 4294967048 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" -TableCommentType 4294967049 0 "pg_stat_database was created for compatibility and is currently unimplemented" -TableCommentType 4294967050 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" -TableCommentType 4294967051 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" -TableCommentType 4294967052 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967053 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967054 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967055 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" -TableCommentType 4294967056 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" -TableCommentType 4294967057 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" -TableCommentType 4294967058 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" -TableCommentType 4294967059 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" -TableCommentType 4294967060 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" -TableCommentType 4294967061 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" -TableCommentType 4294967062 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" -TableCommentType 4294967063 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" -TableCommentType 4294967064 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" -TableCommentType 4294967065 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" -TableCommentType 4294967066 0 "pg_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967067 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" -TableCommentType 4294967068 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" -TableCommentType 4294967069 0 "pg_replication_slots was created for compatibility and is currently unimplemented" -TableCommentType 4294967070 0 "pg_replication_origin was created for compatibility and is currently unimplemented" -TableCommentType 4294967071 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" -TableCommentType 4294967072 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" -TableCommentType 4294967073 0 "pg_publication_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967074 0 "pg_publication was created for compatibility and is currently unimplemented" -TableCommentType 4294967075 0 "pg_publication_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967076 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" -TableCommentType 4294967077 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" -TableCommentType 4294967078 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" -TableCommentType 4294967079 0 "pg_policy was created for compatibility and is currently unimplemented" -TableCommentType 4294967080 0 "pg_policies was created for compatibility and is currently unimplemented" -TableCommentType 4294967081 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" -TableCommentType 4294967082 0 "pg_opfamily was created for compatibility and is currently unimplemented" -TableCommentType 4294967083 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" -TableCommentType 4294967084 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" -TableCommentType 4294967085 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" -TableCommentType 4294967086 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" -TableCommentType 4294967087 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" -TableCommentType 4294967088 0 "pg_largeobject was created for compatibility and is currently unimplemented" -TableCommentType 4294967089 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" -TableCommentType 4294967090 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" -TableCommentType 4294967091 0 "pg_init_privs was created for compatibility and is currently unimplemented" -TableCommentType 4294967092 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" -TableCommentType 4294967093 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" -TableCommentType 4294967094 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" -TableCommentType 4294967095 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967096 0 "pg_group was created for compatibility and is currently unimplemented" -TableCommentType 4294967097 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" -TableCommentType 4294967098 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" -TableCommentType 4294967099 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" -TableCommentType 4294967100 0 "pg_file_settings was created for compatibility and is currently unimplemented" -TableCommentType 4294967101 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" -TableCommentType 4294967102 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" -TableCommentType 4294967103 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" -TableCommentType 4294967104 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" -TableCommentType 4294967105 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" -TableCommentType 4294967106 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" -TableCommentType 4294967107 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" -TableCommentType 4294967108 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" -TableCommentType 4294967109 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" -TableCommentType 4294967110 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" -TableCommentType 4294967111 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" -TableCommentType 4294967112 0 "pg_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967113 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" -TableCommentType 4294967114 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" -TableCommentType 4294967115 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" -TableCommentType 4294967116 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" -TableCommentType 4294967117 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" -TableCommentType 4294967118 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" -TableCommentType 4294967119 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" -TableCommentType 4294967120 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" -TableCommentType 4294967121 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" -TableCommentType 4294967122 0 "pg_amproc was created for compatibility and is currently unimplemented" -TableCommentType 4294967123 0 "pg_amop was created for compatibility and is currently unimplemented" -TableCommentType 4294967124 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" -TableCommentType 4294967125 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" -TableCommentType 4294967127 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" -TableCommentType 4294967128 0 "view_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967129 0 "view_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967130 0 "view_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967131 0 "grantable privileges (incomplete)" -TableCommentType 4294967132 0 "user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967133 0 "user_mapping_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967134 0 "user_defined_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967135 0 "user_attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967136 0 "usage_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967137 0 "udt_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967138 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" -TableCommentType 4294967139 0 "triggers was created for compatibility and is currently unimplemented" -TableCommentType 4294967140 0 "triggered_update_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967141 0 "transforms was created for compatibility and is currently unimplemented" -TableCommentType 4294967142 0 "tablespaces was created for compatibility and is currently unimplemented" -TableCommentType 4294967143 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967144 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" -TableCommentType 4294967145 0 "tables_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967146 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" -TableCommentType 4294967147 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967148 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" -TableCommentType 4294967149 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" -TableCommentType 4294967150 0 "st_units_of_measure was created for compatibility and is currently unimplemented" -TableCommentType 4294967151 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" -TableCommentType 4294967152 0 "st_geometry_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967153 0 "exposes the session variables." -TableCommentType 4294967154 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" -TableCommentType 4294967155 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" -TableCommentType 4294967156 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" -TableCommentType 4294967157 0 "schemata_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967158 0 "sql_sizing was created for compatibility and is currently unimplemented" -TableCommentType 4294967159 0 "sql_parts was created for compatibility and is currently unimplemented" -TableCommentType 4294967160 0 "sql_implementation_info was created for compatibility and is currently unimplemented" -TableCommentType 4294967161 0 "sql_features was created for compatibility and is currently unimplemented" -TableCommentType 4294967162 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" -TableCommentType 4294967163 0 "routine_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967164 0 "role_usage_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967165 0 "role_udt_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967166 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" -TableCommentType 4294967167 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" -TableCommentType 4294967168 0 "role_column_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967169 0 "resource_groups was created for compatibility and is currently unimplemented" -TableCommentType 4294967170 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" -TableCommentType 4294967171 0 "profiling was created for compatibility and is currently unimplemented" -TableCommentType 4294967172 0 "processlist was created for compatibility and is currently unimplemented" -TableCommentType 4294967173 0 "plugins was created for compatibility and is currently unimplemented" -TableCommentType 4294967174 0 "partitions was created for compatibility and is currently unimplemented" -TableCommentType 4294967175 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" -TableCommentType 4294967176 0 "optimizer_trace was created for compatibility and is currently unimplemented" -TableCommentType 4294967177 0 "keywords was created for compatibility and is currently unimplemented" -TableCommentType 4294967178 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" -TableCommentType 4294967179 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" -TableCommentType 4294967180 0 "foreign_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967181 0 "foreign_table_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967182 0 "foreign_servers was created for compatibility and is currently unimplemented" -TableCommentType 4294967183 0 "foreign_server_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967184 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" -TableCommentType 4294967185 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967186 0 "files was created for compatibility and is currently unimplemented" -TableCommentType 4294967187 0 "events was created for compatibility and is currently unimplemented" -TableCommentType 4294967188 0 "engines was created for compatibility and is currently unimplemented" -TableCommentType 4294967189 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" -TableCommentType 4294967190 0 "element_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967191 0 "domains was created for compatibility and is currently unimplemented" -TableCommentType 4294967192 0 "domain_udt_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967193 0 "domain_constraints was created for compatibility and is currently unimplemented" -TableCommentType 4294967194 0 "data_type_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967195 0 "constraint_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967196 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" -TableCommentType 4294967197 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" -TableCommentType 4294967198 0 "columns_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967199 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" -TableCommentType 4294967200 0 "column_statistics was created for compatibility and is currently unimplemented" -TableCommentType 4294967201 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" -TableCommentType 4294967202 0 "column_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967203 0 "column_domain_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967204 0 "column_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967205 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" -TableCommentType 4294967206 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" -TableCommentType 4294967207 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" -TableCommentType 4294967208 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967209 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" -TableCommentType 4294967210 0 "attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967211 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" -TableCommentType 4294967212 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" -TableCommentType 4294967214 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" -TableCommentType 4294967215 0 "list super regions of databases visible to the current user" -TableCommentType 4294967216 0 "which entries of pg_catalog are implemented in this version of CockroachDB" -TableCommentType 4294967218 0 "node-level table listing all currently running range feeds" -TableCommentType 4294967219 0 "virtual table with default privileges" -TableCommentType 4294967220 0 "available regions for the cluster" -TableCommentType 4294967221 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" -TableCommentType 4294967222 0 "virtual table with table descriptors that still have data" -TableCommentType 4294967223 0 "virtual table with cross db references" -TableCommentType 4294967224 0 "virtual table with database privileges" -TableCommentType 4294967225 0 "virtual table to validate descriptors" -TableCommentType 4294967226 0 "decoded zone configurations from system.zones (KV scan)" -TableCommentType 4294967228 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)." -TableCommentType 4294967229 0 "stats for all tables accessible by current user in current database as of 10s ago" -TableCommentType 4294967230 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" -TableCommentType 4294967231 0 "key spans per SQL object" -TableCommentType 4294967232 0 "indexes accessible by current user in current database (KV scan)" -TableCommentType 4294967233 0 "details for all columns accessible by current user in current database (KV scan)" -TableCommentType 4294967235 0 "session variables (RAM)" -TableCommentType 4294967236 0 "session trace accumulated so far (RAM)" -TableCommentType 4294967237 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" -TableCommentType 4294967238 0 "server parameters, useful to construct connection URLs (RAM, local node only)" -TableCommentType 4294967239 0 "ranges is a view which queries ranges_no_leases for system ranges" -TableCommentType 4294967240 0 "range metadata without leaseholder details (KV join; expensive!)" -TableCommentType 4294967241 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" -TableCommentType 4294967242 0 "per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours)" -TableCommentType 4294967243 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)." +TableCommentType 4294966992 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." +TableCommentType 4294966993 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." +TableCommentType 4294966994 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." +TableCommentType 4294966996 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" +TableCommentType 4294966997 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" +TableCommentType 4294966998 0 "pg_user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294966999 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" +TableCommentType 4294967000 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" +TableCommentType 4294967001 0 "pg_ts_template was created for compatibility and is currently unimplemented" +TableCommentType 4294967002 0 "pg_ts_parser was created for compatibility and is currently unimplemented" +TableCommentType 4294967003 0 "pg_ts_dict was created for compatibility and is currently unimplemented" +TableCommentType 4294967004 0 "pg_ts_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967005 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" +TableCommentType 4294967006 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" +TableCommentType 4294967007 0 "pg_transform was created for compatibility and is currently unimplemented" +TableCommentType 4294967008 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" +TableCommentType 4294967009 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" +TableCommentType 4294967010 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" +TableCommentType 4294967011 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" +TableCommentType 4294967012 0 "pg_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967013 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967014 0 "pg_stats was created for compatibility and is currently unimplemented" +TableCommentType 4294967015 0 "pg_stats_ext was created for compatibility and is currently unimplemented" +TableCommentType 4294967016 0 "pg_statistic was created for compatibility and is currently unimplemented" +TableCommentType 4294967017 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" +TableCommentType 4294967018 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" +TableCommentType 4294967019 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967020 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967021 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967022 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967023 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967024 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967025 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967026 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967027 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967028 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967029 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967030 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967031 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967032 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967033 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967034 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967035 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967036 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967037 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967038 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967039 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" +TableCommentType 4294967040 0 "pg_stat_slru was created for compatibility and is currently unimplemented" +TableCommentType 4294967041 0 "pg_stat_replication was created for compatibility and is currently unimplemented" +TableCommentType 4294967042 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" +TableCommentType 4294967043 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" +TableCommentType 4294967044 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" +TableCommentType 4294967045 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" +TableCommentType 4294967046 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" +TableCommentType 4294967047 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" +TableCommentType 4294967048 0 "pg_stat_database was created for compatibility and is currently unimplemented" +TableCommentType 4294967049 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" +TableCommentType 4294967050 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" +TableCommentType 4294967051 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967052 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967053 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967054 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" +TableCommentType 4294967055 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" +TableCommentType 4294967056 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" +TableCommentType 4294967057 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" +TableCommentType 4294967058 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" +TableCommentType 4294967059 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" +TableCommentType 4294967060 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" +TableCommentType 4294967061 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" +TableCommentType 4294967062 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" +TableCommentType 4294967063 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" +TableCommentType 4294967064 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" +TableCommentType 4294967065 0 "pg_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967066 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" +TableCommentType 4294967067 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" +TableCommentType 4294967068 0 "pg_replication_slots was created for compatibility and is currently unimplemented" +TableCommentType 4294967069 0 "pg_replication_origin was created for compatibility and is currently unimplemented" +TableCommentType 4294967070 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" +TableCommentType 4294967071 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" +TableCommentType 4294967072 0 "pg_publication_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967073 0 "pg_publication was created for compatibility and is currently unimplemented" +TableCommentType 4294967074 0 "pg_publication_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967075 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" +TableCommentType 4294967076 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" +TableCommentType 4294967077 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" +TableCommentType 4294967078 0 "pg_policy was created for compatibility and is currently unimplemented" +TableCommentType 4294967079 0 "pg_policies was created for compatibility and is currently unimplemented" +TableCommentType 4294967080 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" +TableCommentType 4294967081 0 "pg_opfamily was created for compatibility and is currently unimplemented" +TableCommentType 4294967082 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" +TableCommentType 4294967083 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" +TableCommentType 4294967084 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" +TableCommentType 4294967085 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" +TableCommentType 4294967086 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" +TableCommentType 4294967087 0 "pg_largeobject was created for compatibility and is currently unimplemented" +TableCommentType 4294967088 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" +TableCommentType 4294967089 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" +TableCommentType 4294967090 0 "pg_init_privs was created for compatibility and is currently unimplemented" +TableCommentType 4294967091 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" +TableCommentType 4294967092 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" +TableCommentType 4294967093 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" +TableCommentType 4294967094 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967095 0 "pg_group was created for compatibility and is currently unimplemented" +TableCommentType 4294967096 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" +TableCommentType 4294967097 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" +TableCommentType 4294967098 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" +TableCommentType 4294967099 0 "pg_file_settings was created for compatibility and is currently unimplemented" +TableCommentType 4294967100 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" +TableCommentType 4294967101 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" +TableCommentType 4294967102 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" +TableCommentType 4294967103 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" +TableCommentType 4294967104 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" +TableCommentType 4294967105 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" +TableCommentType 4294967106 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" +TableCommentType 4294967107 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" +TableCommentType 4294967108 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" +TableCommentType 4294967109 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" +TableCommentType 4294967110 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" +TableCommentType 4294967111 0 "pg_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967112 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" +TableCommentType 4294967113 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" +TableCommentType 4294967114 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" +TableCommentType 4294967115 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" +TableCommentType 4294967116 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" +TableCommentType 4294967117 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" +TableCommentType 4294967118 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" +TableCommentType 4294967119 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" +TableCommentType 4294967120 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" +TableCommentType 4294967121 0 "pg_amproc was created for compatibility and is currently unimplemented" +TableCommentType 4294967122 0 "pg_amop was created for compatibility and is currently unimplemented" +TableCommentType 4294967123 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" +TableCommentType 4294967124 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" +TableCommentType 4294967126 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" +TableCommentType 4294967127 0 "view_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967128 0 "view_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967129 0 "view_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967130 0 "grantable privileges (incomplete)" +TableCommentType 4294967131 0 "user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294967132 0 "user_mapping_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967133 0 "user_defined_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967134 0 "user_attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967135 0 "usage_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967136 0 "udt_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967137 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" +TableCommentType 4294967138 0 "triggers was created for compatibility and is currently unimplemented" +TableCommentType 4294967139 0 "triggered_update_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967140 0 "transforms was created for compatibility and is currently unimplemented" +TableCommentType 4294967141 0 "tablespaces was created for compatibility and is currently unimplemented" +TableCommentType 4294967142 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967143 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" +TableCommentType 4294967144 0 "tables_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967145 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" +TableCommentType 4294967146 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967147 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" +TableCommentType 4294967148 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" +TableCommentType 4294967149 0 "st_units_of_measure was created for compatibility and is currently unimplemented" +TableCommentType 4294967150 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" +TableCommentType 4294967151 0 "st_geometry_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967152 0 "exposes the session variables." +TableCommentType 4294967153 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" +TableCommentType 4294967154 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" +TableCommentType 4294967155 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" +TableCommentType 4294967156 0 "schemata_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967157 0 "sql_sizing was created for compatibility and is currently unimplemented" +TableCommentType 4294967158 0 "sql_parts was created for compatibility and is currently unimplemented" +TableCommentType 4294967159 0 "sql_implementation_info was created for compatibility and is currently unimplemented" +TableCommentType 4294967160 0 "sql_features was created for compatibility and is currently unimplemented" +TableCommentType 4294967161 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" +TableCommentType 4294967162 0 "routine_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967163 0 "role_usage_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967164 0 "role_udt_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967165 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" +TableCommentType 4294967166 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" +TableCommentType 4294967167 0 "role_column_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967168 0 "resource_groups was created for compatibility and is currently unimplemented" +TableCommentType 4294967169 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" +TableCommentType 4294967170 0 "profiling was created for compatibility and is currently unimplemented" +TableCommentType 4294967171 0 "processlist was created for compatibility and is currently unimplemented" +TableCommentType 4294967172 0 "plugins was created for compatibility and is currently unimplemented" +TableCommentType 4294967173 0 "partitions was created for compatibility and is currently unimplemented" +TableCommentType 4294967174 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" +TableCommentType 4294967175 0 "optimizer_trace was created for compatibility and is currently unimplemented" +TableCommentType 4294967176 0 "keywords was created for compatibility and is currently unimplemented" +TableCommentType 4294967177 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" +TableCommentType 4294967178 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" +TableCommentType 4294967179 0 "foreign_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967180 0 "foreign_table_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967181 0 "foreign_servers was created for compatibility and is currently unimplemented" +TableCommentType 4294967182 0 "foreign_server_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967183 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" +TableCommentType 4294967184 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967185 0 "files was created for compatibility and is currently unimplemented" +TableCommentType 4294967186 0 "events was created for compatibility and is currently unimplemented" +TableCommentType 4294967187 0 "engines was created for compatibility and is currently unimplemented" +TableCommentType 4294967188 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" +TableCommentType 4294967189 0 "element_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967190 0 "domains was created for compatibility and is currently unimplemented" +TableCommentType 4294967191 0 "domain_udt_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967192 0 "domain_constraints was created for compatibility and is currently unimplemented" +TableCommentType 4294967193 0 "data_type_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967194 0 "constraint_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967195 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" +TableCommentType 4294967196 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" +TableCommentType 4294967197 0 "columns_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967198 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" +TableCommentType 4294967199 0 "column_statistics was created for compatibility and is currently unimplemented" +TableCommentType 4294967200 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" +TableCommentType 4294967201 0 "column_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967202 0 "column_domain_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967203 0 "column_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967204 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" +TableCommentType 4294967205 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" +TableCommentType 4294967206 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" +TableCommentType 4294967207 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967208 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" +TableCommentType 4294967209 0 "attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967210 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" +TableCommentType 4294967211 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" +TableCommentType 4294967213 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" +TableCommentType 4294967214 0 "list super regions of databases visible to the current user" +TableCommentType 4294967215 0 "which entries of pg_catalog are implemented in this version of CockroachDB" +TableCommentType 4294967217 0 "node-level table listing all currently running range feeds" +TableCommentType 4294967218 0 "virtual table with default privileges" +TableCommentType 4294967219 0 "available regions for the cluster" +TableCommentType 4294967220 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" +TableCommentType 4294967221 0 "virtual table with table descriptors that still have data" +TableCommentType 4294967222 0 "virtual table with cross db references" +TableCommentType 4294967223 0 "virtual table with database privileges" +TableCommentType 4294967224 0 "virtual table to validate descriptors" +TableCommentType 4294967225 0 "decoded zone configurations from system.zones (KV scan)" +TableCommentType 4294967227 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)." +TableCommentType 4294967228 0 "stats for all tables accessible by current user in current database as of 10s ago" +TableCommentType 4294967229 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" +TableCommentType 4294967230 0 "key spans per SQL object" +TableCommentType 4294967231 0 "indexes accessible by current user in current database (KV scan)" +TableCommentType 4294967232 0 "details for all columns accessible by current user in current database (KV scan)" +TableCommentType 4294967234 0 "session variables (RAM)" +TableCommentType 4294967235 0 "session trace accumulated so far (RAM)" +TableCommentType 4294967236 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" +TableCommentType 4294967237 0 "server parameters, useful to construct connection URLs (RAM, local node only)" +TableCommentType 4294967238 0 "ranges is a view which queries ranges_no_leases for system ranges" +TableCommentType 4294967239 0 "range metadata without leaseholder details (KV join; expensive!)" +TableCommentType 4294967240 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" +TableCommentType 4294967241 0 "per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours)" +TableCommentType 4294967242 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)." +TableCommentType 4294967243 0 "node-level table listing all currently active memory monitors" TableCommentType 4294967244 0 "current values for metrics (RAM; local node only)" TableCommentType 4294967245 0 "running sessions visible by current user (RAM; local node only)" TableCommentType 4294967246 0 "running user transactions visible by the current user (RAM; local node only)" diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 2ca4cfab4c7e..4d2aa1b352ac 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -79,6 +79,7 @@ test crdb_internal node_contention_events public test crdb_internal node_distsql_flows public SELECT false test crdb_internal node_execution_insights public SELECT false test crdb_internal node_inflight_trace_spans public SELECT false +test crdb_internal node_memory_monitors public SELECT false test crdb_internal node_metrics public SELECT false test crdb_internal node_queries public SELECT false test crdb_internal node_runtime_info public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index c6e70cfd7366..803dd7852f37 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -444,6 +444,7 @@ crdb_internal node_contention_events crdb_internal node_distsql_flows crdb_internal node_execution_insights crdb_internal node_inflight_trace_spans +crdb_internal node_memory_monitors crdb_internal node_metrics crdb_internal node_queries crdb_internal node_runtime_info @@ -779,6 +780,7 @@ node_contention_events node_distsql_flows node_execution_insights node_inflight_trace_spans +node_memory_monitors node_metrics node_queries node_runtime_info @@ -1154,6 +1156,7 @@ system crdb_internal node_contention_events SYSTEM system crdb_internal node_distsql_flows SYSTEM VIEW NO 1 system crdb_internal node_execution_insights SYSTEM VIEW NO 1 system crdb_internal node_inflight_trace_spans SYSTEM VIEW NO 1 +system crdb_internal node_memory_monitors SYSTEM VIEW NO 1 system crdb_internal node_metrics SYSTEM VIEW NO 1 system crdb_internal node_queries SYSTEM VIEW NO 1 system crdb_internal node_runtime_info SYSTEM VIEW NO 1 @@ -2928,6 +2931,7 @@ NULL public system crdb_internal node_contention_events NULL public system crdb_internal node_distsql_flows SELECT NO YES NULL public system crdb_internal node_execution_insights SELECT NO YES NULL public system crdb_internal node_inflight_trace_spans SELECT NO YES +NULL public system crdb_internal node_memory_monitors SELECT NO YES NULL public system crdb_internal node_metrics SELECT NO YES NULL public system crdb_internal node_queries SELECT NO YES NULL public system crdb_internal node_runtime_info SELECT NO YES @@ -3551,6 +3555,7 @@ NULL public system crdb_internal node_contention_events NULL public system crdb_internal node_distsql_flows SELECT NO YES NULL public system crdb_internal node_execution_insights SELECT NO YES NULL public system crdb_internal node_inflight_trace_spans SELECT NO YES +NULL public system crdb_internal node_memory_monitors SELECT NO YES NULL public system crdb_internal node_metrics SELECT NO YES NULL public system crdb_internal node_queries SELECT NO YES NULL public system crdb_internal node_runtime_info SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 325e188722db..4ee6a8880406 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -210,42 +210,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 4294967114 1 0 false -pg_class relname 4294967114 2 0 false -pg_class relnamespace 4294967114 3 0 false -pg_class reltype 4294967114 4 0 false -pg_class reloftype 4294967114 5 0 false -pg_class relowner 4294967114 6 0 false -pg_class relam 4294967114 7 0 false -pg_class relfilenode 4294967114 8 0 false -pg_class reltablespace 4294967114 9 0 false -pg_class relpages 4294967114 10 0 false -pg_class reltuples 4294967114 11 0 false -pg_class relallvisible 4294967114 12 0 false -pg_class reltoastrelid 4294967114 13 0 false -pg_class relhasindex 4294967114 14 0 false -pg_class relisshared 4294967114 15 0 false -pg_class relpersistence 4294967114 16 0 false -pg_class relistemp 4294967114 17 0 false -pg_class relkind 4294967114 18 0 false -pg_class relnatts 4294967114 19 0 false -pg_class relchecks 4294967114 20 0 false -pg_class relhasoids 4294967114 21 0 false -pg_class relhaspkey 4294967114 22 0 false -pg_class relhasrules 4294967114 23 0 false -pg_class relhastriggers 4294967114 24 0 false -pg_class relhassubclass 4294967114 25 0 false -pg_class relfrozenxid 4294967114 26 0 false -pg_class relacl 4294967114 27 0 false -pg_class reloptions 4294967114 28 0 false -pg_class relforcerowsecurity 4294967114 29 0 false -pg_class relispartition 4294967114 30 0 false -pg_class relispopulated 4294967114 31 0 false -pg_class relreplident 4294967114 32 0 false -pg_class relrewrite 4294967114 33 0 false -pg_class relrowsecurity 4294967114 34 0 false -pg_class relpartbound 4294967114 35 0 false -pg_class relminmxid 4294967114 36 0 false +pg_class oid 4294967113 1 0 false +pg_class relname 4294967113 2 0 false +pg_class relnamespace 4294967113 3 0 false +pg_class reltype 4294967113 4 0 false +pg_class reloftype 4294967113 5 0 false +pg_class relowner 4294967113 6 0 false +pg_class relam 4294967113 7 0 false +pg_class relfilenode 4294967113 8 0 false +pg_class reltablespace 4294967113 9 0 false +pg_class relpages 4294967113 10 0 false +pg_class reltuples 4294967113 11 0 false +pg_class relallvisible 4294967113 12 0 false +pg_class reltoastrelid 4294967113 13 0 false +pg_class relhasindex 4294967113 14 0 false +pg_class relisshared 4294967113 15 0 false +pg_class relpersistence 4294967113 16 0 false +pg_class relistemp 4294967113 17 0 false +pg_class relkind 4294967113 18 0 false +pg_class relnatts 4294967113 19 0 false +pg_class relchecks 4294967113 20 0 false +pg_class relhasoids 4294967113 21 0 false +pg_class relhaspkey 4294967113 22 0 false +pg_class relhasrules 4294967113 23 0 false +pg_class relhastriggers 4294967113 24 0 false +pg_class relhassubclass 4294967113 25 0 false +pg_class relfrozenxid 4294967113 26 0 false +pg_class relacl 4294967113 27 0 false +pg_class reloptions 4294967113 28 0 false +pg_class relforcerowsecurity 4294967113 29 0 false +pg_class relispartition 4294967113 30 0 false +pg_class relispopulated 4294967113 31 0 false +pg_class relreplident 4294967113 32 0 false +pg_class relrewrite 4294967113 33 0 false +pg_class relrowsecurity 4294967113 34 0 false +pg_class relpartbound 4294967113 35 0 false +pg_class relminmxid 4294967113 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 8335d26d1b97..dfa2825c2b19 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 NULL NULL -4294967213 information_schema NULL NULL -4294967126 pg_catalog NULL NULL -4294966996 pg_extension NULL NULL +4294967212 information_schema NULL NULL +4294967125 pg_catalog NULL NULL +4294966995 pg_extension NULL 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 NULL NULL -4294967213 information_schema NULL NULL -4294967126 pg_catalog NULL NULL -4294966996 pg_extension NULL NULL +4294967212 information_schema NULL NULL +4294967125 pg_catalog NULL NULL +4294966995 pg_extension NULL NULL 105 public 2310524507 NULL user root @@ -1321,7 +1321,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967126 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967125 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1520,16 +1520,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967111 111 0 4294967114 110 14 a -4294967111 112 0 4294967114 110 15 a -4294967068 842401391 0 4294967114 110 1 n -4294967068 842401391 0 4294967114 110 2 n -4294967068 842401391 0 4294967114 110 3 n -4294967068 842401391 0 4294967114 110 4 n -4294967111 1179276562 0 4294967114 3687884464 0 n -4294967111 3935750373 0 4294967114 3687884465 0 n -4294967111 4072017905 0 4294967114 0 0 n -4294967111 4170826110 0 4294967114 0 0 n +4294967110 111 0 4294967113 110 14 a +4294967110 112 0 4294967113 110 15 a +4294967067 842401391 0 4294967113 110 1 n +4294967067 842401391 0 4294967113 110 2 n +4294967067 842401391 0 4294967113 110 3 n +4294967067 842401391 0 4294967113 110 4 n +4294967110 1179276562 0 4294967113 3687884464 0 n +4294967110 3935750373 0 4294967113 3687884465 0 n +4294967110 4072017905 0 4294967113 0 0 n +4294967110 4170826110 0 4294967113 0 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1542,8 +1542,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967068 4294967114 pg_rewrite pg_class -4294967111 4294967114 pg_constraint pg_class +4294967067 4294967113 pg_rewrite pg_class +4294967110 4294967113 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 @@ -1642,88 +1642,88 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967126 NULL 1 true b -17 bytea 4294967126 NULL -1 false b -18 char 4294967126 NULL 1 true b -19 name 4294967126 NULL -1 false b -20 int8 4294967126 NULL 8 true b -21 int2 4294967126 NULL 2 true b -22 int2vector 4294967126 NULL -1 false b -23 int4 4294967126 NULL 4 true b -24 regproc 4294967126 NULL 4 true b -25 text 4294967126 NULL -1 false b -26 oid 4294967126 NULL 4 true b -30 oidvector 4294967126 NULL -1 false b -700 float4 4294967126 NULL 4 true b -701 float8 4294967126 NULL 8 true b -705 unknown 4294967126 NULL 0 true b -869 inet 4294967126 NULL 24 true b -1000 _bool 4294967126 NULL -1 false b -1001 _bytea 4294967126 NULL -1 false b -1002 _char 4294967126 NULL -1 false b -1003 _name 4294967126 NULL -1 false b -1005 _int2 4294967126 NULL -1 false b -1006 _int2vector 4294967126 NULL -1 false b -1007 _int4 4294967126 NULL -1 false b -1008 _regproc 4294967126 NULL -1 false b -1009 _text 4294967126 NULL -1 false b -1013 _oidvector 4294967126 NULL -1 false b -1014 _bpchar 4294967126 NULL -1 false b -1015 _varchar 4294967126 NULL -1 false b -1016 _int8 4294967126 NULL -1 false b -1021 _float4 4294967126 NULL -1 false b -1022 _float8 4294967126 NULL -1 false b -1028 _oid 4294967126 NULL -1 false b -1041 _inet 4294967126 NULL -1 false b -1042 bpchar 4294967126 NULL -1 false b -1043 varchar 4294967126 NULL -1 false b -1082 date 4294967126 NULL 4 true b -1083 time 4294967126 NULL 8 true b -1114 timestamp 4294967126 NULL 8 true b -1115 _timestamp 4294967126 NULL -1 false b -1182 _date 4294967126 NULL -1 false b -1183 _time 4294967126 NULL -1 false b -1184 timestamptz 4294967126 NULL 8 true b -1185 _timestamptz 4294967126 NULL -1 false b -1186 interval 4294967126 NULL 24 true b -1187 _interval 4294967126 NULL -1 false b -1231 _numeric 4294967126 NULL -1 false b -1266 timetz 4294967126 NULL 12 true b -1270 _timetz 4294967126 NULL -1 false b -1560 bit 4294967126 NULL -1 false b -1561 _bit 4294967126 NULL -1 false b -1562 varbit 4294967126 NULL -1 false b -1563 _varbit 4294967126 NULL -1 false b -1700 numeric 4294967126 NULL -1 false b -2202 regprocedure 4294967126 NULL 4 true b -2205 regclass 4294967126 NULL 4 true b -2206 regtype 4294967126 NULL 4 true b -2207 _regprocedure 4294967126 NULL -1 false b -2210 _regclass 4294967126 NULL -1 false b -2211 _regtype 4294967126 NULL -1 false b -2249 record 4294967126 NULL 0 true p -2277 anyarray 4294967126 NULL -1 false p -2278 void 4294967126 NULL 0 true p -2283 anyelement 4294967126 NULL -1 false p -2287 _record 4294967126 NULL -1 false b -2950 uuid 4294967126 NULL 16 true b -2951 _uuid 4294967126 NULL -1 false b -3614 tsvector 4294967126 NULL -1 false b -3615 tsquery 4294967126 NULL -1 false b -3643 _tsvector 4294967126 NULL -1 false b -3645 _tsquery 4294967126 NULL -1 false b -3802 jsonb 4294967126 NULL -1 false b -3807 _jsonb 4294967126 NULL -1 false b -4089 regnamespace 4294967126 NULL 4 true b -4090 _regnamespace 4294967126 NULL -1 false b -4096 regrole 4294967126 NULL 4 true b -4097 _regrole 4294967126 NULL -1 false b -90000 geometry 4294967126 NULL -1 false b -90001 _geometry 4294967126 NULL -1 false b -90002 geography 4294967126 NULL -1 false b -90003 _geography 4294967126 NULL -1 false b -90004 box2d 4294967126 NULL 32 true b -90005 _box2d 4294967126 NULL -1 false b +16 bool 4294967125 NULL 1 true b +17 bytea 4294967125 NULL -1 false b +18 char 4294967125 NULL 1 true b +19 name 4294967125 NULL -1 false b +20 int8 4294967125 NULL 8 true b +21 int2 4294967125 NULL 2 true b +22 int2vector 4294967125 NULL -1 false b +23 int4 4294967125 NULL 4 true b +24 regproc 4294967125 NULL 4 true b +25 text 4294967125 NULL -1 false b +26 oid 4294967125 NULL 4 true b +30 oidvector 4294967125 NULL -1 false b +700 float4 4294967125 NULL 4 true b +701 float8 4294967125 NULL 8 true b +705 unknown 4294967125 NULL 0 true b +869 inet 4294967125 NULL 24 true b +1000 _bool 4294967125 NULL -1 false b +1001 _bytea 4294967125 NULL -1 false b +1002 _char 4294967125 NULL -1 false b +1003 _name 4294967125 NULL -1 false b +1005 _int2 4294967125 NULL -1 false b +1006 _int2vector 4294967125 NULL -1 false b +1007 _int4 4294967125 NULL -1 false b +1008 _regproc 4294967125 NULL -1 false b +1009 _text 4294967125 NULL -1 false b +1013 _oidvector 4294967125 NULL -1 false b +1014 _bpchar 4294967125 NULL -1 false b +1015 _varchar 4294967125 NULL -1 false b +1016 _int8 4294967125 NULL -1 false b +1021 _float4 4294967125 NULL -1 false b +1022 _float8 4294967125 NULL -1 false b +1028 _oid 4294967125 NULL -1 false b +1041 _inet 4294967125 NULL -1 false b +1042 bpchar 4294967125 NULL -1 false b +1043 varchar 4294967125 NULL -1 false b +1082 date 4294967125 NULL 4 true b +1083 time 4294967125 NULL 8 true b +1114 timestamp 4294967125 NULL 8 true b +1115 _timestamp 4294967125 NULL -1 false b +1182 _date 4294967125 NULL -1 false b +1183 _time 4294967125 NULL -1 false b +1184 timestamptz 4294967125 NULL 8 true b +1185 _timestamptz 4294967125 NULL -1 false b +1186 interval 4294967125 NULL 24 true b +1187 _interval 4294967125 NULL -1 false b +1231 _numeric 4294967125 NULL -1 false b +1266 timetz 4294967125 NULL 12 true b +1270 _timetz 4294967125 NULL -1 false b +1560 bit 4294967125 NULL -1 false b +1561 _bit 4294967125 NULL -1 false b +1562 varbit 4294967125 NULL -1 false b +1563 _varbit 4294967125 NULL -1 false b +1700 numeric 4294967125 NULL -1 false b +2202 regprocedure 4294967125 NULL 4 true b +2205 regclass 4294967125 NULL 4 true b +2206 regtype 4294967125 NULL 4 true b +2207 _regprocedure 4294967125 NULL -1 false b +2210 _regclass 4294967125 NULL -1 false b +2211 _regtype 4294967125 NULL -1 false b +2249 record 4294967125 NULL 0 true p +2277 anyarray 4294967125 NULL -1 false p +2278 void 4294967125 NULL 0 true p +2283 anyelement 4294967125 NULL -1 false p +2287 _record 4294967125 NULL -1 false b +2950 uuid 4294967125 NULL 16 true b +2951 _uuid 4294967125 NULL -1 false b +3614 tsvector 4294967125 NULL -1 false b +3615 tsquery 4294967125 NULL -1 false b +3643 _tsvector 4294967125 NULL -1 false b +3645 _tsquery 4294967125 NULL -1 false b +3802 jsonb 4294967125 NULL -1 false b +3807 _jsonb 4294967125 NULL -1 false b +4089 regnamespace 4294967125 NULL 4 true b +4090 _regnamespace 4294967125 NULL -1 false b +4096 regrole 4294967125 NULL 4 true b +4097 _regrole 4294967125 NULL -1 false b +90000 geometry 4294967125 NULL -1 false b +90001 _geometry 4294967125 NULL -1 false b +90002 geography 4294967125 NULL -1 false b +90003 _geography 4294967125 NULL -1 false b +90004 box2d 4294967125 NULL 32 true b +90005 _box2d 4294967125 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 @@ -2199,7 +2199,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967126 NULL -1 false b +1000 _bool 4294967125 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2257,7 +2257,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967076 pg_proc 4294967126 2310524507 -1 false c +4294967075 pg_proc 4294967125 2310524507 -1 false c ## pg_catalog.pg_proc @@ -2268,14 +2268,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 -substring 4294967126 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 +substring 4294967125 pg_catalog NULL 0 NULL NULL 0 query TTBBBB colnames SELECT proname, protransform, proisagg, proiswindow, prosecdef, proleakproof @@ -2357,7 +2357,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 4294967213 information_schema 20 26 23 +_pg_char_max_length _pg_char_max_length 4294967212 information_schema 20 26 23 query TOIOTTB colnames SELECT proname, provariadic, pronargs, prorettype, proargtypes, proargmodes, proisstrict @@ -2474,14 +2474,14 @@ OR (c.relname = 'pg_class' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_class OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname = 'testschema')) ---- relname objoid classoid objsubid description -pg_class 135 4294967114 0 mycomment1 -pg_class 4294966998 4294967114 0 database users -pg_class 135 4294967114 1 mycomment2 -pg_class 125730312 4294967114 0 mycomment3 -pg_namespace 136 4294967085 0 mycomment4 -pg_proc 738 4294967076 0 Calculates the absolute value of `val`. -pg_proc 739 4294967076 0 Calculates the absolute value of `val`. -pg_proc 740 4294967076 0 Calculates the absolute value of `val`. +pg_class 135 4294967113 0 mycomment1 +pg_class 4294966997 4294967113 0 database users +pg_class 135 4294967113 1 mycomment2 +pg_class 125730312 4294967113 0 mycomment3 +pg_namespace 136 4294967084 0 mycomment4 +pg_proc 738 4294967075 0 Calculates the absolute value of `val`. +pg_proc 739 4294967075 0 Calculates the absolute value of `val`. +pg_proc 740 4294967075 0 Calculates the absolute value of `val`. ## pg_catalog.pg_shdescription @@ -2492,7 +2492,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967108 mydbcomment +100 4294967107 mydbcomment ## pg_catalog.pg_event_trigger @@ -3027,7 +3027,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 + 4294967126 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967125 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -3874,7 +3874,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294966997 +test pg_views 4294966996 # 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 8a404b10aa01..75285130477b 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -604,6 +604,7 @@ node_contention_events NULL node_distsql_flows NULL node_execution_insights NULL node_inflight_trace_spans NULL +node_memory_monitors NULL node_metrics NULL node_queries NULL node_runtime_info NULL diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 4c7d0dcdc423..05d47827c391 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -440,7 +440,7 @@ vectorized: true │ │ │ order: +objoid │ │ │ │ │ └── • filter - │ │ │ filter: (objsubid = 0) AND (classoid = 4294967114) + │ │ │ filter: (objsubid = 0) AND (classoid = 4294967113) │ │ │ │ │ └── • virtual table │ │ table: pg_description@primary diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index af051a26e8af..19a7a0a5234c 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 < 4294966993 [outer=(1), constraints=(/1: (/NULL - /4294966992]; tight)] + │ │ └── attrelid:1 < 4294966992 [outer=(1), constraints=(/1: (/NULL - /4294966991]; tight)] │ └── aggregations │ ├── const-agg [as=attname:2, outer=(2)] │ │ └── attname:2 diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index cf083f36475d..67fe1d9e235a 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -7742,11 +7742,10 @@ expires until the statement bundle is collected`, if len(sql) != 0 { parsed, err := parser.ParseOne(sql) - if err != nil { - return tree.NewDString(sqlNoConstants), nil //nolint:returnerrcheck + // Leave result as empty string on parsing error. + if err == nil { + sqlNoConstants = tree.AsStringWithFlags(parsed.AST, tree.FmtHideConstants) } - - sqlNoConstants = tree.AsStringWithFlags(parsed.AST, tree.FmtHideConstants) } if err := result.Append(tree.NewDString(sqlNoConstants)); err != nil { @@ -7761,6 +7760,24 @@ expires until the statement bundle is collected`, Volatility: volatility.Immutable, }, ), + "crdb_internal.humanize_bytes": makeBuiltin(tree.FunctionProperties{ + Category: builtinconstants.CategoryString, + Undocumented: true, + }, + tree.Overload{ + Types: tree.ParamTypes{{Name: "val", Typ: types.Int}}, + ReturnType: tree.FixedReturnType(types.String), + Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { + if args[0] == tree.DNull { + return tree.DNull, nil + } + b := tree.MustBeDInt(args[0]) + return tree.NewDString(string(humanizeutil.IBytes(int64(b)))), nil + }, + Info: "Converts integer size (in bytes) into the human-readable form.", + Volatility: volatility.Leakproof, + }, + ), } 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 42df35e3e48f..837f8c75b782 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -2050,6 +2050,7 @@ var builtinOidsArray = []string{ 2075: `phraseto_tsquery(config: string, text: string) -> tsquery`, 2076: `plainto_tsquery(config: string, text: string) -> tsquery`, 2077: `ts_parse(parser_name: string, document: string) -> tuple{int AS tokid, string AS token}`, + 2078: `crdb_internal.humanize_bytes(val: int) -> string`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index be6a519ecf33..6a688dbc7edf 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -148,6 +148,7 @@ const ( CrdbInternalLocalTransactionsTableID CrdbInternalLocalSessionsTableID CrdbInternalLocalMetricsTableID + CrdbInternalNodeMemoryMonitorsTableID CrdbInternalNodeStmtStatsTableID CrdbInternalNodeTxnStatsTableID CrdbInternalPartitionsTableID diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 2734ac59047b..b852c30e1af4 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -437,14 +437,14 @@ func TestTenantID3() roachpb.TenantID { // GetJSONProto uses the supplied client to GET the URL specified by the parameters // and unmarshals the result into response. -func GetJSONProto(ts TestServerInterface, path string, response protoutil.Message) error { +func GetJSONProto(ts TestTenantInterface, path string, response protoutil.Message) error { return GetJSONProtoWithAdminOption(ts, path, response, true) } // GetJSONProtoWithAdminOption is like GetJSONProto but the caller can customize // whether the request is performed with admin privilege func GetJSONProtoWithAdminOption( - ts TestServerInterface, path string, response protoutil.Message, isAdmin bool, + ts TestTenantInterface, path string, response protoutil.Message, isAdmin bool, ) error { httpClient, err := ts.GetAuthenticatedHTTPClient(isAdmin) if err != nil { @@ -455,7 +455,7 @@ func GetJSONProtoWithAdminOption( // PostJSONProto uses the supplied client to POST the URL specified by the parameters // and unmarshals the result into response. -func PostJSONProto(ts TestServerInterface, path string, request, response protoutil.Message) error { +func PostJSONProto(ts TestTenantInterface, path string, request, response protoutil.Message) error { return PostJSONProtoWithAdminOption(ts, path, request, response, true) } @@ -463,7 +463,7 @@ func PostJSONProto(ts TestServerInterface, path string, request, response protou // can customize whether the request is performed with admin // privilege. func PostJSONProtoWithAdminOption( - ts TestServerInterface, path string, request, response protoutil.Message, isAdmin bool, + ts TestTenantInterface, path string, request, response protoutil.Message, isAdmin bool, ) error { httpClient, err := ts.GetAuthenticatedHTTPClient(isAdmin) if err != nil { diff --git a/pkg/util/mon/BUILD.bazel b/pkg/util/mon/BUILD.bazel index 9b9b9e31e3b4..808ad0152d1d 100644 --- a/pkg/util/mon/BUILD.bazel +++ b/pkg/util/mon/BUILD.bazel @@ -38,6 +38,7 @@ go_test( "//pkg/util/log", "//pkg/util/metric", "//pkg/util/randutil", + "@com_github_cockroachdb_redact//:redact", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index c1ada5f75f10..a3a81ccda9e8 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -16,6 +16,7 @@ import ( "io" "math" "math/bits" + "unsafe" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" @@ -196,6 +197,31 @@ type BytesMonitor struct { // maxBytesHist is the metric object used to track the high watermark of bytes // allocated by the monitor during its lifetime. maxBytesHist metric.IHistogram + + // head is the head of the doubly-linked list of children of this + // monitor. + // NOTE: a child's mutex **cannot** be acquired while holding this + // monitor's lock since it could lead to deadlocks. The main allocation + // code path (reserveBytes() and releaseBytes()) might acquire the + // parent's lock, so only locking "upwards" is allowed while keeping the + // current monitor's lock. + head *BytesMonitor + + // numChildren is the number of children of this BytesMonitor (i.e. the + // number of nodes in the linked list). + numChildren int + + // stopped indicates whether this monitor has been stopped. + stopped bool + } + + // parentMu encompasses the fields that must be accessed while holding the + // mutex of the parent monitor. + parentMu struct { + // prevSibling and nextSibling are references to the previous and the + // next siblings of this monitor (i.e. the previous and the next nodes + // in the doubly-linked list of children of the parent monitor). + prevSibling, nextSibling *BytesMonitor } // name identifies this monitor in logging messages. @@ -237,6 +263,90 @@ type BytesMonitor struct { settings *cluster.Settings } +// MonitorState describes the current state of a single monitor. +type MonitorState struct { + // Level tracks how many "generations" away the current monitor is from the + // root. + Level int + // Name is the name of the monitor. + Name string + // ID is the "id" of the monitor (its address converted to int64). + ID int64 + // ParentID is the "id" of the parent monitor (parent's address converted to + // int64), or 0 if the monitor doesn't have a parent. + ParentID int64 + // Used is amount of bytes currently used by the monitor (as reported by + // curBudget.used). This doesn't include the usage registered with the + // reserved account. + Used int64 + // ReservedUsed is amount of bytes currently consumed from the reserved + // account, or 0 if no reserved account was provided in Start. + ReservedUsed int64 + // ReservedReserved is amount of bytes reserved in the reserved account, or + // 0 if no reserved account was provided in Start. + ReservedReserved int64 +} + +// TraverseTree traverses the tree of monitors rooted in the BytesMonitor. The +// passed-in callback is called for each non-stopped monitor. If the callback +// returns an error, the traversal stops immediately. +// +// Note that this state can be inconsistent since a parent's state is recorded +// before its children without synchronization with children being stopped. +// Namely, the parent's MonitorState might include the state of the monitors +// that don't get the callback called on their MonitorState. +func (mm *BytesMonitor) TraverseTree(monitorStateCb func(MonitorState) error) error { + return mm.traverseTree(0 /* level */, monitorStateCb) +} + +// traverseTree recursively traverses the tree of monitors rooted in the current +// monitor. If the monitor is stopped, then the tree is not traversed and the +// callback is not called. +func (mm *BytesMonitor) traverseTree(level int, monitorStateCb func(MonitorState) error) error { + mm.mu.Lock() + if mm.mu.stopped { + // The monitor has been stopped, so it should be ignored. + mm.mu.Unlock() + return nil + } + var reservedUsed, reservedReserved int64 + if mm.reserved != nil { + reservedUsed = mm.reserved.used + reservedReserved = mm.reserved.reserved + } + id := uintptr(unsafe.Pointer(mm)) + var parentID uintptr + if parent := mm.mu.curBudget.mon; parent != nil { + parentID = uintptr(unsafe.Pointer(parent)) + } + monitorState := MonitorState{ + Level: level, + Name: string(mm.name), + ID: int64(id), + ParentID: int64(parentID), + Used: mm.mu.curBudget.used, + ReservedUsed: reservedUsed, + ReservedReserved: reservedReserved, + } + // Note that we cannot call traverseTree on the children while holding mm's + // lock since it could lead to deadlocks. Instead, we store all children as + // of right now, and then export them after unlocking ourselves. + children := make([]*BytesMonitor, 0, mm.mu.numChildren) + for c := mm.mu.head; c != nil; c = c.parentMu.nextSibling { + children = append(children, c) + } + mm.mu.Unlock() + if err := monitorStateCb(monitorState); err != nil { + return err + } + for _, c := range children { + if err := c.traverseTree(level+1, monitorStateCb); err != nil { + return err + } + } + return nil +} + // maxAllocatedButUnusedBlocks determines the maximum difference between the // amount of bytes used by a monitor and the amount of bytes reserved at the // upstream pool before the monitor relinquishes the bytes back to the pool. @@ -366,6 +476,7 @@ func (mm *BytesMonitor) Start(ctx context.Context, pool *BytesMonitor, reserved mm.mu.curAllocated = 0 mm.mu.maxAllocated = 0 mm.mu.curBudget = pool.MakeBoundAccount() + mm.mu.stopped = false mm.reserved = reserved if log.V(2) { poolname := redact.RedactableString("(none)") @@ -377,6 +488,18 @@ func (mm *BytesMonitor) Start(ctx context.Context, pool *BytesMonitor, reserved humanizeutil.IBytes(mm.reserved.used), poolname) } + if pool != nil { + // If we have a "parent" monitor, then register mm as its child by + // making it the head of the doubly-linked list. + pool.mu.Lock() + if s := pool.mu.head; s != nil { + s.parentMu.prevSibling = mm + mm.parentMu.nextSibling = s + } + pool.mu.head = mm + pool.mu.numChildren++ + pool.mu.Unlock() + } } // NewUnlimitedMonitor creates a new monitor and starts the monitor in @@ -433,8 +556,10 @@ func (mm *BytesMonitor) Limit() int64 { const bytesMaxUsageLoggingThreshold = 100 * 1024 func (mm *BytesMonitor) doStop(ctx context.Context, check bool) { - // NB: No need to lock mm.mu here, when StopMonitor() is called the - // monitor is not shared any more. + mm.mu.Lock() + defer mm.mu.Unlock() + mm.mu.stopped = true + if log.V(1) && mm.mu.maxAllocated >= bytesMaxUsageLoggingThreshold { log.InfofDepth(ctx, 1, "%s, bytes usage max %s", mm.name, @@ -459,6 +584,24 @@ func (mm *BytesMonitor) doStop(ctx context.Context, check bool) { mm.mu.maxBytesHist.RecordValue(val) } + if parent := mm.mu.curBudget.mon; parent != nil { + // If we have a "parent" monitor, then unregister mm from the list of + // the parent's children. + parent.mu.Lock() + prev, next := mm.parentMu.prevSibling, mm.parentMu.nextSibling + if parent.mu.head == mm { + parent.mu.head = next + } + if prev != nil { + prev.parentMu.nextSibling = next + } + if next != nil { + next.parentMu.prevSibling = prev + } + parent.mu.numChildren-- + parent.mu.Unlock() + } + // Disable the pool for further allocations, so that further // uses outside of monitor control get errors. mm.mu.curBudget.mon = nil diff --git a/pkg/util/mon/bytes_usage_test.go b/pkg/util/mon/bytes_usage_test.go index 260a525dd38b..3d3e4acd4531 100644 --- a/pkg/util/mon/bytes_usage_test.go +++ b/pkg/util/mon/bytes_usage_test.go @@ -15,13 +15,17 @@ import ( "fmt" "math" "math/rand" + "strings" + "sync" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/redact" "github.com/stretchr/testify/require" ) @@ -420,6 +424,149 @@ func TestReservedAccountCleared(t *testing.T) { require.Equal(t, int64(0), root.mu.curBudget.used) } +func getMonitor( + ctx context.Context, st *cluster.Settings, name string, parent *BytesMonitor, +) *BytesMonitor { + m := NewMonitor(redact.RedactableString(name), MemoryResource, nil, nil, 1, math.MaxInt64, st) + var reserved *BoundAccount + if parent == nil { + reserved = NewStandaloneBudget(math.MaxInt64) + } else { + reserved = &noReserved + } + m.Start(ctx, parent, reserved) + return m +} + +// TestBytesMonitorTree is a sanity check that the tree structure of related +// monitors is maintained and traversed as expected. +func TestBytesMonitorTree(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + export := func(m *BytesMonitor) string { + var monitors []MonitorState + _ = m.TraverseTree(func(monitor MonitorState) error { + monitors = append(monitors, monitor) + return nil + }) + var sb strings.Builder + for _, e := range monitors { + for i := 0; i < e.Level; i++ { + sb.WriteString("-") + } + sb.WriteString(e.Name + "\n") + } + return sb.String() + } + + parent := getMonitor(ctx, st, "parent", nil /* parent */) + child1 := getMonitor(ctx, st, "child1", parent) + child2 := getMonitor(ctx, st, "child2", parent) + require.Equal(t, "parent\n-child2\n-child1\n", export(parent)) + require.Equal(t, "child1\n", export(child1)) + require.Equal(t, "child2\n", export(child2)) + + grandchild1 := getMonitor(ctx, st, "grandchild1", child1) + grandchild2 := getMonitor(ctx, st, "grandchild2", child2) + require.Equal(t, "parent\n-child2\n--grandchild2\n-child1\n--grandchild1\n", export(parent)) + require.Equal(t, "child1\n-grandchild1\n", export(child1)) + require.Equal(t, "child2\n-grandchild2\n", export(child2)) + + grandchild2.Stop(ctx) + child2.Stop(ctx) + + require.Equal(t, "parent\n-child1\n--grandchild1\n", export(parent)) + require.Equal(t, "child1\n-grandchild1\n", export(child1)) + + grandchild1.Stop(ctx) + child1.Stop(ctx) + + require.Equal(t, "parent\n", export(parent)) + parent.Stop(ctx) +} + +// TestBytesMonitorNoDeadlocks ensures that no deadlocks can occur when monitors +// are started and stopped concurrently with the monitor tree traversal. +func TestBytesMonitorNoDeadlocks(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + + root := getMonitor(ctx, st, "root", nil /* parent */) + defer root.Stop(ctx) + + // Spin up 10 goroutines that repeatedly start and stop child monitors while + // also making reservations against them. + var wg sync.WaitGroup + const numGoroutines = 10 + // done will be closed when the concurrent goroutines should exit. + done := make(chan struct{}) + for i := 0; i < numGoroutines; i++ { + wg.Add(1) + go func(i int) { + defer wg.Done() + rng, _ := randutil.NewTestRand() + for { + select { + case <-done: + return + default: + func() { + m := getMonitor(ctx, st, fmt.Sprintf("m%d", i), root) + defer m.Stop(ctx) + numOps := rng.Intn(10 + 1) + var reserved int64 + defer func() { + m.releaseBytes(ctx, reserved) + }() + for op := 0; op < numOps; op++ { + if reserved > 0 && rng.Float64() < 0.5 { + toRelease := int64(rng.Intn(int(reserved))) + 1 + m.releaseBytes(ctx, toRelease) + reserved -= toRelease + } else { + toReserve := int64(rng.Intn(1000) + 1) + // We shouldn't hit any errors since we have an + // unlimited root budget. + _ = m.reserveBytes(ctx, toReserve) + reserved += toReserve + } + // Sleep up to 1ms in-between operations. + time.Sleep(time.Duration(rng.Intn(1000)) * time.Microsecond) + } + }() + // Sleep up to 2ms after having stopped the monitor. + time.Sleep(time.Duration(rng.Intn(2000)) * time.Microsecond) + } + } + }(i) + } + + // In the main goroutine, perform the tree traversal several times with + // sleeps in-between. + rng, _ := randutil.NewTestRand() + for i := 0; i < 1000; i++ { + // We mainly want to ensure that no deadlocks nor data races are + // occurring, but also we do a sanity check that each "row" in the + // output of TraverseTree() is a non-empty MonitorState. + var monitors []MonitorState + _ = root.TraverseTree(func(monitor MonitorState) error { + monitors = append(monitors, monitor) + return nil + }) + for _, m := range monitors { + require.NotEqual(t, MonitorState{}, m) + } + // Sleep up to 3ms. + time.Sleep(time.Duration(rng.Intn(3000)) * time.Microsecond) + } + close(done) + wg.Wait() +} + func BenchmarkBoundAccountGrow(b *testing.B) { ctx := context.Background() m := NewMonitor("test", MemoryResource, @@ -432,3 +579,48 @@ func BenchmarkBoundAccountGrow(b *testing.B) { _ = a.Grow(ctx, 1) } } + +func BenchmarkTraverseTree(b *testing.B) { + makeMonitorTree := func(numLevels int, numChildrenPerMonitor int) (root *BytesMonitor, cleanup func()) { + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + allMonitors := make([][]*BytesMonitor, numLevels) + allMonitors[0] = []*BytesMonitor{getMonitor(ctx, st, "root", nil /* parent */)} + for level := 1; level < numLevels; level++ { + allMonitors[level] = make([]*BytesMonitor, 0, len(allMonitors[level-1])*numChildrenPerMonitor) + for parent, parentMon := range allMonitors[level-1] { + for child := 0; child < numChildrenPerMonitor; child++ { + name := fmt.Sprintf("child%d_parent%d", child, parent) + allMonitors[level] = append(allMonitors[level], getMonitor(ctx, st, name, parentMon)) + } + } + } + cleanup = func() { + // Simulate the production setting where we stop the children before + // their parent (this is not strictly necessary since we don't + // reserve budget from the monitors below). + for i := len(allMonitors) - 1; i >= 0; i-- { + for _, m := range allMonitors[i] { + m.Stop(ctx) + } + } + } + return allMonitors[0][0], cleanup + } + for _, numLevels := range []int{2, 4, 8} { + for _, numChildrenPerMonitor := range []int{2, 4, 8} { + b.Run(fmt.Sprintf("levels=%d/children=%d", numLevels, numChildrenPerMonitor), func(b *testing.B) { + root, cleanup := makeMonitorTree(numLevels, numChildrenPerMonitor) + defer cleanup() + b.ResetTimer() + for i := 0; i < b.N; i++ { + var numMonitors int + _ = root.TraverseTree(func(MonitorState) error { + numMonitors++ + return nil + }) + } + }) + } + } +}