From 5786a71ac594c8a4ce39c2db692c461442cbb4b6 Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Mon, 19 Dec 2022 15:33:20 -0500 Subject: [PATCH 1/4] server: refactor database index recommendations for DatabaseDetails API Resolves: https://github.com/cockroachdb/cockroach/issues/93909 Previously, the DatabaseDetails API would fetch the index recommendations of the database by executing an expensive query for **each table of the database**, then coalesce the results of each table to get the database-level result. This was needlessly expensive and impractical, particularly so for large schemas. This change ensures that only a single query is executed **per database** to fetch its index recommendations. Release note (performance improvement): Refactored the query logic when fetching database index recommendations for the DatabaseDetails API endpoint, greatly reducing the query time and cost, particularly for large schemas. --- pkg/server/admin.go | 23 +--- pkg/server/index_usage_stats.go | 100 +++++++++++++++--- pkg/sql/idxusage/BUILD.bazel | 1 + pkg/sql/idxusage/index_usage_stats_rec.go | 27 +++-- .../idxusage/index_usage_stats_rec_test.go | 96 ++++++----------- 5 files changed, 144 insertions(+), 103 deletions(-) diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 655933c1050d..34fb31c99c19 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -642,12 +642,12 @@ func (s *adminServer) databaseDetailsHelper( if err != nil { return nil, err } - resp.Stats.NumIndexRecommendations, err = s.getNumDatabaseIndexRecommendations(ctx, req.Database, resp.TableNames) + dbIndexRecommendations, err := getDatabaseIndexRecommendations(ctx, req.Database, s.ie, s.st, s.sqlServer.execCfg) if err != nil { return nil, err } + resp.Stats.NumIndexRecommendations = int32(len(dbIndexRecommendations)) } - return &resp, nil } @@ -760,25 +760,6 @@ func (s *adminServer) getDatabaseStats( return &stats, nil } -func (s *adminServer) getNumDatabaseIndexRecommendations( - ctx context.Context, databaseName string, tableNames []string, -) (int32, error) { - var numDatabaseIndexRecommendations int - idxUsageStatsProvider := s.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics() - for _, tableName := range tableNames { - tableIndexStatsRequest := &serverpb.TableIndexStatsRequest{ - Database: databaseName, - Table: tableName, - } - tableIndexStatsResponse, err := getTableIndexUsageStats(ctx, tableIndexStatsRequest, idxUsageStatsProvider, s.ie, s.st, s.sqlServer.execCfg) - if err != nil { - return 0, err - } - numDatabaseIndexRecommendations += len(tableIndexStatsResponse.IndexRecommendations) - } - return int32(numDatabaseIndexRecommendations), nil -} - // getFullyQualifiedTableName, given a database name and a tableName that either // is a unqualified name or a schema-qualified name, returns a maximally // qualified name: either database.table if the input wasn't schema qualified, diff --git a/pkg/server/index_usage_stats.go b/pkg/server/index_usage_stats.go index ce2f97c902a3..757e9b039693 100644 --- a/pkg/server/index_usage_stats.go +++ b/pkg/server/index_usage_stats.go @@ -12,6 +12,7 @@ package server import ( "context" + "fmt" "time" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -20,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/idxusage" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -256,9 +258,6 @@ func getTableIndexUsageStats( WHERE ti.descriptor_id = $::REGCLASS`, tableID, ) - - const expectedNumDatums = 7 - it, err := ie.QueryIteratorEx(ctx, "index-usage-stats", nil, sessiondata.InternalExecutorOverride{ User: userName, @@ -278,14 +277,7 @@ func getTableIndexUsageStats( defer func() { err = errors.CombineErrors(err, it.Close()) }() for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - var row tree.Datums - if row = it.Cur(); row == nil { - return nil, errors.New("unexpected null row") - } - - if row.Len() != expectedNumDatums { - return nil, errors.Newf("expected %d columns, received %d", expectedNumDatums, row.Len()) - } + row := it.Cur() indexID := tree.MustBeDInt(row[0]) indexName := tree.MustBeDString(row[1]) @@ -324,7 +316,11 @@ func getTableIndexUsageStats( } statsRow := idxusage.IndexStatsRow{ - Row: idxStatsRow, + TableID: idxStatsRow.Statistics.Key.TableID, + IndexID: idxStatsRow.Statistics.Key.IndexID, + CreatedAt: idxStatsRow.CreatedAt, + LastRead: idxStatsRow.Statistics.Stats.LastRead, + IndexType: idxStatsRow.IndexType, UnusedIndexKnobs: execConfig.UnusedIndexRecommendationsKnobs, } recommendations := statsRow.GetRecommendationsFromIndexStats(req.Database, st) @@ -373,3 +369,83 @@ func getTableIDFromDatabaseAndTableName( tableID := tree.MustBeDOid(row[0]).Oid return int(tableID), nil } + +func getDatabaseIndexRecommendations( + ctx context.Context, + dbName string, + ie *sql.InternalExecutor, + st *cluster.Settings, + execConfig *sql.ExecutorConfig, +) ([]*serverpb.IndexRecommendation, error) { + + // Omit fetching index recommendations for the 'system' database. + if dbName == catconstants.SystemDatabaseName { + return []*serverpb.IndexRecommendation{}, nil + } + + userName, err := userFromContext(ctx) + if err != nil { + return []*serverpb.IndexRecommendation{}, err + } + + query := fmt.Sprintf(` + SELECT + ti.descriptor_id as table_id, + ti.index_id, + ti.index_type, + last_read, + ti.created_at + FROM %[1]s.crdb_internal.index_usage_statistics AS us + JOIN %[1]s.crdb_internal.table_indexes AS ti ON (us.index_id = ti.index_id AND us.table_id = ti.descriptor_id AND index_type = 'secondary') + JOIN %[1]s.crdb_internal.tables AS t ON (ti.descriptor_id = t.table_id AND t.database_name != 'system');`, dbName) + + it, err := ie.QueryIteratorEx(ctx, "db-index-recommendations", nil, + sessiondata.InternalExecutorOverride{ + User: userName, + Database: dbName, + }, query) + + if err != nil { + return []*serverpb.IndexRecommendation{}, err + } + + // We have to make sure to close the iterator since we might return from the + // for loop early (before Next() returns false). + defer func() { err = errors.CombineErrors(err, it.Close()) }() + + var ok bool + var idxRecommendations []*serverpb.IndexRecommendation + + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() + + tableID := tree.MustBeDInt(row[0]) + indexID := tree.MustBeDInt(row[1]) + indexType := tree.MustBeDString(row[2]) + lastRead := time.Time{} + if row[3] != tree.DNull { + lastRead = tree.MustBeDTimestampTZ(row[3]).Time + } + var createdAt *time.Time + if row[4] != tree.DNull { + ts := tree.MustBeDTimestamp(row[4]) + createdAt = &ts.Time + } + + if err != nil { + return []*serverpb.IndexRecommendation{}, err + } + + statsRow := idxusage.IndexStatsRow{ + TableID: roachpb.TableID(tableID), + IndexID: roachpb.IndexID(indexID), + CreatedAt: createdAt, + LastRead: lastRead, + IndexType: string(indexType), + UnusedIndexKnobs: execConfig.UnusedIndexRecommendationsKnobs, + } + recommendations := statsRow.GetRecommendationsFromIndexStats(dbName, st) + idxRecommendations = append(idxRecommendations, recommendations...) + } + return idxRecommendations, nil +} diff --git a/pkg/sql/idxusage/BUILD.bazel b/pkg/sql/idxusage/BUILD.bazel index 88fa9de2ebfd..3d2a3868f649 100644 --- a/pkg/sql/idxusage/BUILD.bazel +++ b/pkg/sql/idxusage/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//pkg/server/serverpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/sem/catconstants", "//pkg/util/syncutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/idxusage/index_usage_stats_rec.go b/pkg/sql/idxusage/index_usage_stats_rec.go index edaf5615fda1..84579efb4475 100644 --- a/pkg/sql/idxusage/index_usage_stats_rec.go +++ b/pkg/sql/idxusage/index_usage_stats_rec.go @@ -15,9 +15,11 @@ import ( "math" "time" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -26,7 +28,11 @@ import ( // implements additional methods to support unused index recommendations and // hold testing knobs. type IndexStatsRow struct { - Row *serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics + TableID roachpb.TableID + IndexID roachpb.IndexID + CreatedAt *time.Time + LastRead time.Time + IndexType string UnusedIndexKnobs *UnusedIndexRecommendationTestingKnobs } @@ -65,7 +71,8 @@ func (i IndexStatsRow) GetRecommendationsFromIndexStats( dbName string, st *cluster.Settings, ) []*serverpb.IndexRecommendation { var recommendations = []*serverpb.IndexRecommendation{} - if dbName == "system" || i.Row.IndexType == "primary" { + // Omit fetching index recommendations for the 'system' database. + if dbName == catconstants.SystemDatabaseName { return recommendations } rec := i.maybeAddUnusedIndexRecommendation(DropUnusedIndexDuration.Get(&st.SV)) @@ -78,11 +85,15 @@ func (i IndexStatsRow) GetRecommendationsFromIndexStats( func (i IndexStatsRow) maybeAddUnusedIndexRecommendation( unusedIndexDuration time.Duration, ) *serverpb.IndexRecommendation { + if i.IndexType == "primary" { + return nil + } + var rec *serverpb.IndexRecommendation if i.UnusedIndexKnobs == nil { - rec = i.recommendDropUnusedIndex(timeutil.Now(), i.Row.CreatedAt, - i.Row.Statistics.Stats.LastRead, unusedIndexDuration) + rec = i.recommendDropUnusedIndex(timeutil.Now(), i.CreatedAt, + i.LastRead, unusedIndexDuration) } else { rec = i.recommendDropUnusedIndex(i.UnusedIndexKnobs.GetCurrentTime(), i.UnusedIndexKnobs.GetCreatedAt(), i.UnusedIndexKnobs.GetLastRead(), unusedIndexDuration) @@ -106,8 +117,8 @@ func (i IndexStatsRow) recommendDropUnusedIndex( // dropping with a "never used" reason. if lastActive.Equal(time.Time{}) { return &serverpb.IndexRecommendation{ - TableID: i.Row.Statistics.Key.TableID, - IndexID: i.Row.Statistics.Key.IndexID, + TableID: i.TableID, + IndexID: i.IndexID, Type: serverpb.IndexRecommendation_DROP_UNUSED, Reason: indexNeverUsedReason, } @@ -115,8 +126,8 @@ func (i IndexStatsRow) recommendDropUnusedIndex( // Last usage of the index exceeds the unused index duration. if currentTime.Sub(lastActive) >= unusedIndexDuration { return &serverpb.IndexRecommendation{ - TableID: i.Row.Statistics.Key.TableID, - IndexID: i.Row.Statistics.Key.IndexID, + TableID: i.TableID, + IndexID: i.IndexID, Type: serverpb.IndexRecommendation_DROP_UNUSED, Reason: fmt.Sprintf(indexExceedUsageDurationReasonPlaceholder, formatDuration(unusedIndexDuration)), } diff --git a/pkg/sql/idxusage/index_usage_stats_rec_test.go b/pkg/sql/idxusage/index_usage_stats_rec_test.go index e4f10b6b170b..99d09a567039 100644 --- a/pkg/sql/idxusage/index_usage_stats_rec_test.go +++ b/pkg/sql/idxusage/index_usage_stats_rec_test.go @@ -16,7 +16,6 @@ import ( "testing" "time" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -46,40 +45,40 @@ func TestGetRecommendationsFromIndexStats(t *testing.T) { }{ // Database name "system", expect no index recommendation. { - idxStats: IndexStatsRow{}, + idxStats: IndexStatsRow{ + TableID: 1, + IndexID: 2, + LastRead: anHourBefore, + IndexType: "secondary", + CreatedAt: nil, + UnusedIndexKnobs: nil, + }, dbName: "system", - unusedIndexDuration: defaultUnusedIndexDuration, + unusedIndexDuration: time.Hour, expectedReturn: []*serverpb.IndexRecommendation{}, }, // Index is primary index, expect no index recommendation. { idxStats: IndexStatsRow{ - Row: &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ - IndexType: "primary", - }, + TableID: 1, + IndexID: 2, + LastRead: anHourBefore, + CreatedAt: nil, + UnusedIndexKnobs: nil, + IndexType: "primary", }, dbName: "testdb", - unusedIndexDuration: defaultUnusedIndexDuration, + unusedIndexDuration: time.Hour, expectedReturn: []*serverpb.IndexRecommendation{}, }, // Index exceeds the unused index duration, expect index recommendation. { idxStats: IndexStatsRow{ - Row: &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ - Statistics: &roachpb.CollectedIndexUsageStatistics{ - Key: roachpb.IndexUsageKey{ - TableID: 1, - IndexID: 2, - }, - Stats: roachpb.IndexUsageStatistics{ - LastRead: anHourBefore, - }, - }, - IndexName: "test_idx", - IndexType: "secondary", - CreatedAt: nil, - CreateStatement: "", - }, + TableID: 1, + IndexID: 2, + LastRead: anHourBefore, + IndexType: "secondary", + CreatedAt: nil, UnusedIndexKnobs: nil, }, dbName: "testdb", @@ -96,21 +95,12 @@ func TestGetRecommendationsFromIndexStats(t *testing.T) { // Index has never been used and has no creation time, expect never used index recommendation. { idxStats: IndexStatsRow{ - Row: &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ - Statistics: &roachpb.CollectedIndexUsageStatistics{ - Key: roachpb.IndexUsageKey{ - TableID: 1, - IndexID: 3, - }, - Stats: roachpb.IndexUsageStatistics{ - LastRead: time.Time{}, - }, - }, - IndexName: "test_idx", - IndexType: "secondary", - CreatedAt: nil, - CreateStatement: "", - }, + TableID: 1, + IndexID: 3, + LastRead: time.Time{}, + + IndexType: "secondary", + CreatedAt: nil, UnusedIndexKnobs: nil, }, dbName: "testdb", @@ -127,21 +117,11 @@ func TestGetRecommendationsFromIndexStats(t *testing.T) { // Index has been used recently, expect no index recommendations. { idxStats: IndexStatsRow{ - Row: &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ - Statistics: &roachpb.CollectedIndexUsageStatistics{ - Key: roachpb.IndexUsageKey{ - TableID: 1, - IndexID: 4, - }, - Stats: roachpb.IndexUsageStatistics{ - LastRead: aMinuteBefore, - }, - }, - IndexName: "test_idx", - IndexType: "secondary", - CreatedAt: nil, - CreateStatement: "", - }, + TableID: 1, + IndexID: 4, + LastRead: aMinuteBefore, + IndexType: "secondary", + CreatedAt: nil, UnusedIndexKnobs: nil, }, dbName: "testdb", @@ -168,17 +148,9 @@ func TestRecommendDropUnusedIndex(t *testing.T) { recommendation *serverpb.IndexRecommendation } - stubIndexStatsRow := &serverpb.TableIndexStatsResponse_ExtendedCollectedIndexUsageStatistics{ - Statistics: &roachpb.CollectedIndexUsageStatistics{ - Key: roachpb.IndexUsageKey{ - TableID: 1, - IndexID: 1, - }, - }, - } - indexStatsRow := IndexStatsRow{ - Row: stubIndexStatsRow, + TableID: 1, + IndexID: 1, } testData := []struct { From 5d6236e77f2535986c793b51986ce5db3f753000 Mon Sep 17 00:00:00 2001 From: gtr Date: Wed, 14 Dec 2022 18:17:56 -0500 Subject: [PATCH 2/4] ui: Populate database filter dropdown in stmts page with `SHOW DATABASES` sql-over-http call Fixes: #70461. Previously, the databases filter dropdown was populated by the `StatementsResponse` API call. This would result in some databases for which we do not receive any stmts to be ignored. According to above issue, the database filter-drop down should always be populated with cluster databases even when there are no statements or transactions for them. This commit populates the database filter dropdown using the `getDatabasesList()` API call which itself executes the `SHOW DATABASES` SQL query. Release note (ui change): The databases filter dropdown in the stmts page now uses the `getDatabasesList()` API call, resulting in all cluster databases showing up. --- .../statementsPage/statementsPage.fixture.ts | 1 + .../statementsPage.selectors.ts | 29 ++++-------- .../src/statementsPage/statementsPage.tsx | 10 ++++ .../statementsPageConnected.tsx | 3 ++ .../databasesList/databasesList.reducers.ts | 47 +++++++++++++++++++ .../store/databasesList/databasesList.saga.ts | 34 ++++++++++++++ .../databasesList/databasesList.selectors.ts | 17 +++++++ .../src/store/databasesList/index.ts | 12 +++++ .../cluster-ui/src/store/reducers.ts | 6 +++ .../src/views/statements/statementsPage.tsx | 22 ++++----- 10 files changed, 149 insertions(+), 32 deletions(-) create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.reducers.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.saga.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/databasesList/index.ts diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts index 61b9f70cfb30..3fc36bf58b55 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.fixture.ts @@ -929,6 +929,7 @@ const statementsPagePropsFixture: StatementsPageProps = { isTenant: false, hasViewActivityRedactedRole: false, dismissAlertMessage: noop, + refreshDatabases: noop, refreshStatementDiagnosticsRequests: noop, refreshStatements: noop, refreshUserSQLRoles: noop, diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts index c42d81fa0663..814f0be340c3 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts @@ -32,6 +32,7 @@ import { AggregateStatistics } from "../statementsTable"; import { sqlStatsSelector } from "../store/sqlStats/sqlStats.selector"; import { SQLStatsState } from "../store/sqlStats"; import { localStorageSelector } from "../store/utils/selectors"; +import { databasesListSelector } from "src/store/databasesList/databasesList.selectors"; type ICollectedStatementStatistics = cockroach.server.serverpb.StatementsResponse.ICollectedStatementStatistics; @@ -86,26 +87,16 @@ export const selectApps = createSelector(sqlStatsSelector, sqlStatsState => { .concat(Object.keys(apps).sort()); }); -// selectDatabases returns the array of all databases with statement statistics present -// in the data. -export const selectDatabases = createSelector( - sqlStatsSelector, - sqlStatsState => { - if (!sqlStatsState.data) { - return []; - } +// selectDatabases returns the array of all databases in the cluster. +export const selectDatabases = createSelector(databasesListSelector, state => { + if (!state.data) { + return []; + } - return Array.from( - new Set( - sqlStatsState.data.statements.map(s => - s.key.key_data.database ? s.key.key_data.database : unset, - ), - ), - ) - .filter((dbName: string) => dbName !== null && dbName.length > 0) - .sort(); - }, -); + return state.data.databases + .filter((dbName: string) => dbName !== null && dbName.length > 0) + .sort(); +}); // selectTotalFingerprints returns the count of distinct statement fingerprints // present in the data. diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx index c4315310c499..1b3564e36fec 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx @@ -81,7 +81,9 @@ import { isSelectedColumn } from "src/columnsSelector/utils"; import { StatementViewType } from "./statementPageTypes"; import moment from "moment"; import { + databasesRequest, InsertStmtDiagnosticRequest, + SqlExecutionRequest, StatementDiagnosticsReport, } from "../api"; @@ -94,6 +96,7 @@ const sortableTableCx = classNames.bind(sortableTableStyles); // provide convenient definitions for `mapDispatchToProps`, `mapStateToProps` and props that // have to be provided by parent component. export interface StatementsPageDispatchProps { + refreshDatabases: (timeout?: moment.Duration) => void; refreshStatements: (req: StatementsRequest) => void; refreshStatementDiagnosticsRequests: () => void; refreshNodes: () => void; @@ -313,6 +316,11 @@ export class StatementsPage extends React.Component< this.resetPolling(this.props.timeScale.key); }; + refreshDatabases = (): void => { + this.props.refreshDatabases(); + this.resetPolling(this.props.timeScale.key); + }; + resetSQLStats = (): void => { const req = statementsRequestFromProps(this.props); this.props.resetSQLStats(req); @@ -342,6 +350,8 @@ export class StatementsPage extends React.Component< ); } + this.refreshDatabases(); + this.props.refreshUserSQLRoles(); this.props.refreshNodes(); if (!this.props.isTenant && !this.props.hasViewActivityRedactedRole) { diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx index 23fc03ecfc96..a07861394b38 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPageConnected.tsx @@ -17,6 +17,7 @@ import { actions as statementDiagnosticsActions } from "src/store/statementDiagn import { actions as analyticsActions } from "src/store/analytics"; import { actions as localStorageActions } from "src/store/localStorage"; import { actions as sqlStatsActions } from "src/store/sqlStats"; +import { actions as databasesListActions } from "src/store/databasesList"; import { actions as nodesActions } from "../store/nodes"; import { StatementsPageDispatchProps, @@ -57,6 +58,7 @@ import { } from "./recentStatementsPage.selectors"; import { InsertStmtDiagnosticRequest, + SqlExecutionRequest, StatementDiagnosticsReport, } from "../api"; @@ -100,6 +102,7 @@ export const ConnectedStatementsPage = withRouter( }), (dispatch: Dispatch) => ({ fingerprintsPageProps: { + refreshDatabases: () => dispatch(databasesListActions.refresh()), refreshStatements: (req: StatementsRequest) => dispatch(sqlStatsActions.refresh(req)), onTimeScaleChange: (ts: TimeScale) => { diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.reducers.ts new file mode 100644 index 000000000000..4658dc62ab08 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.reducers.ts @@ -0,0 +1,47 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { createSlice, PayloadAction } from "@reduxjs/toolkit"; +import { DatabasesListResponse } from "src/api"; +import { DOMAIN_NAME } from "../utils"; + +import { SqlExecutionRequest } from "../../api/sqlApi"; + +export type DatabasesListState = { + data: DatabasesListResponse; + lastError: Error; + valid: boolean; +}; + +const initialState: DatabasesListState = { + data: null, + lastError: null, + valid: true, +}; + +const databasesListSlice = createSlice({ + name: `${DOMAIN_NAME}/databasesList`, + initialState, + reducers: { + received: (state, action: PayloadAction) => { + state.data = action.payload; + state.valid = true; + state.lastError = null; + }, + failed: (state, action: PayloadAction) => { + state.valid = false; + state.lastError = action.payload; + }, + refresh: (_, action: PayloadAction) => {}, + request: (_, action: PayloadAction) => {}, + }, +}); + +export const { reducer, actions } = databasesListSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.saga.ts b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.saga.ts new file mode 100644 index 000000000000..64f92277daef --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.saga.ts @@ -0,0 +1,34 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { all, call, put, takeLatest } from "redux-saga/effects"; + +import { actions } from "./databasesList.reducers"; +import { getDatabasesList } from "src/api"; + +export function* refreshDatabasesListSaga() { + yield put(actions.request()); +} + +export function* requestDatabasesListSaga(): any { + try { + const result = yield call(getDatabasesList); + yield put(actions.received(result)); + } catch (e) { + yield put(actions.failed(e)); + } +} + +export function* databasesListSaga() { + yield all([ + takeLatest(actions.refresh, refreshDatabasesListSaga), + takeLatest(actions.request, requestDatabasesListSaga), + ]); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts new file mode 100644 index 000000000000..bc6c4dcc86bf --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/databasesList.selectors.ts @@ -0,0 +1,17 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { createSelector } from "reselect"; +import { adminUISelector } from "../utils/selectors"; + +export const databasesListSelector = createSelector( + adminUISelector, + adminUiState => adminUiState.databasesList, +); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/databasesList/index.ts b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/index.ts new file mode 100644 index 000000000000..8eccb1028374 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/databasesList/index.ts @@ -0,0 +1,12 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export * from "./databasesList.reducers"; +export * from "./databasesList.saga"; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts index d90af8377218..29598b83af83 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts @@ -15,6 +15,10 @@ import { ClusterLocksReqState, reducer as clusterLocks, } from "./clusterLocks/clusterLocks.reducer"; +import { + DatabasesListState, + reducer as databasesList, +} from "./databasesList/databasesList.reducers"; import { IndexStatsReducerState, reducer as indexStats, @@ -71,6 +75,7 @@ export type AdminUiState = { jobs: JobsState; job: JobState; clusterLocks: ClusterLocksReqState; + databasesList: DatabasesListState; transactionInsights: TransactionInsightsState; transactionInsightDetails: TransactionInsightDetailsCachedState; executionInsights: ExecutionInsightsState; @@ -98,6 +103,7 @@ export const reducers = combineReducers({ jobs, job, clusterLocks, + databasesList, schemaInsights, }); diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx index bb92069f6ab7..3a04e664eeac 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx @@ -15,6 +15,7 @@ import { RouteComponentProps, withRouter } from "react-router-dom"; import * as protos from "src/js/protos"; import { refreshNodes, + refreshDatabases, refreshStatementDiagnosticsRequests, refreshStatements, refreshUserSQLRoles, @@ -217,21 +218,15 @@ export const selectApps = createSelector( }, ); -// selectDatabases returns the array of all databases with statement statistics present -// in the data. +// selectDatabases returns the array of all databases in the cluster. export const selectDatabases = createSelector( - (state: AdminUIState) => state.cachedData.statements, - (state: CachedDataReducerState) => { + (state: AdminUIState) => state.cachedData.databases, + (state: CachedDataReducerState) => { if (!state.data) { return []; } - return Array.from( - new Set( - state.data.statements.map(s => - s.key.key_data.database ? s.key.key_data.database : unset, - ), - ), - ) + + return state.data.databases .filter((dbName: string) => dbName !== null && dbName.length > 0) .sort(); }, @@ -287,7 +282,8 @@ export const searchLocalSetting = new LocalSetting( ); const fingerprintsPageActions = { - refreshStatements, + refreshStatements: refreshStatements, + refreshDatabases: refreshDatabases, onTimeScaleChange: setGlobalTimeScaleAction, refreshStatementDiagnosticsRequests, refreshNodes, @@ -387,7 +383,7 @@ export default withRouter( }, activePageProps: mapStateToRecentStatementViewProps(state), }), - dispatch => ({ + (dispatch: AppDispatch): DispatchProps => ({ fingerprintsPageProps: bindActionCreators( fingerprintsPageActions, dispatch, From 08829cf27623a43fa444d6c2ca287aec1640fcac Mon Sep 17 00:00:00 2001 From: Abby Hersh Date: Thu, 22 Dec 2022 13:22:17 -0500 Subject: [PATCH 3/4] roachtest: update version map for 22.2.1 links epic https://cockroachlabs.atlassian.net/browse/REL-228 Release note: none --- pkg/cmd/roachtest/tests/predecessor_version.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cmd/roachtest/tests/predecessor_version.json b/pkg/cmd/roachtest/tests/predecessor_version.json index 17cfca79f9b8..06d0d64613b4 100644 --- a/pkg/cmd/roachtest/tests/predecessor_version.json +++ b/pkg/cmd/roachtest/tests/predecessor_version.json @@ -9,5 +9,5 @@ "21.2": "21.1.12", "22.1": "21.2.7", "22.2": "22.1.6", - "23.1": "22.2.0" + "23.1": "22.2.1" } From e67d2fd95546fb5af9e765a5cde1ae597f357373 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 14 Dec 2022 22:42:22 +0100 Subject: [PATCH 4/4] sql: enhance SHOW RANGES MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The output of `crdb_internal.ranges{,_no_leases}` and `SHOW RANGES` was irreparably broken by the introduction of range coalescing (ranges spanning multiple tables/databases). Moreover, the start/end keys of SHOW RANGES were often empty or NULL due to incorrect/excessive truncation. This commit fixes this by introducing a new design for SHOW RANGES and tweaking the definition of `crdb_internal.ranges{,_no_leases}`. Note: THIS IS A BREAKING CHANGE. See the "backward-incompatible change" release notes below for suggestions on updating client code. ---- **Short documentation.**
The revised syntax is now: ``` SHOW CLUSTER RANGES [WITH ] SHOW RANGES [FROM DATABASE | FROM CURRENT_CATALOG] [ WITH ] SHOW RANGES FROM TABLE [ WITH ] SHOW RANGES FROM INDEX [ WITH ] is a combination of TABLES, INDEXES, KEYS, DETAILS and EXPLAIN. ``` New syntax: `SHOW CLUSTER RANGES`, `SHOW RANGES` with no `FROM`, `FROM CURRENT_CATALOG`, `WITH` clause. In summary, we have: - `SHOW CLUSTER RANGES` which includes all ranges, including those not belonging to any table. - `SHOW RANGES [FROM DATABASE | FROM CURRENT_CATALOG]` which includes only ranges overlapping with any table in the target db. Note: `SHOW RANGES` without target (NEW!) is an alias for `SHOW RANGES FROM CURRENT_CATALOG`. - `SHOW RANGES FROM TABLE` selects only ranges that overlap with the given table. - `SHOW RANGES FROM INDEX` selects only ranges that overlap with the given index. Then: - if `WITH TABLES` is specified, the rows are duplicated to detail each table included in each range (1 row per range-table intersection). - if `WITH INDEXES` is specified, the rows are duplicated to detail each index included in each range (1 row per range-index intersection). - otherwise, there is just 1 row per range. In summary: | Statement | Row identity | Before | After | |----------------------------------------------------|----------------------------------|----------------------------|----------------------------------| | `SHOW RANGES FROM DATABASE` | rangeID | Includes schema/table name | (CHANGE) No schema/table name | | `SHOW RANGES FROM TABLE` | rangeID | Includes index name | (CHANGE) No index name | | `SHOW RANGES FROM INDEX` | rangeID | Includes index name | Unchanged | | `SHOW RANGES FROM DATABASE ... WITH TABLES` (NEW) | rangeID, schema/table name | N/A | Includes schema/table name | | `SHOW RANGES FROM DATABASE ... WITH INDEXES` (NEW) | rangeID, schema/table/index name | N/A | Includes schema/table/index name | | `SHOW RANGES FROM TABLE ... WITH INDEXES` (NEW) | rangeID, index name | N/A | Includes index name | | `SHOW CLUSTER RANGES` (NEW) | rangeID | N/A | | | `SHOW CLUSTER RANGES WITH TABLES` (NEW) | rangeID, schema/table name | N/A | Includes db/schema/table name | | `SHOW CLUSTER RANGES WITH INDEXES` (NEW) | rangeID, schema/table/index name | N/A | Includes db/sch/table/index name | | Statement | Start/end key column, before | Start/end key column, after | |----------------------------------------------------|------------------------------|---------------------------------------| | `SHOW RANGES FROM DATABASE` | Truncates table/index IDs | (CHANGE) Includes table/index ID | | `SHOW RANGES FROM TABLE` | Truncates table/index IDs | (CHANGE) Includes table/index ID | | `SHOW RANGES FROM INDEX` | Truncates table/index IDs | Unchanged | | `SHOW RANGES FROM DATABASE ... WITH TABLES` (NEW) | N/A | Includes table/index ID | | `SHOW RANGES FROM DATABASE ... WITH INDEXES` (NEW) | N/A | Includes table/index ID | | `SHOW RANGES FROM TABLE ... WITH INDEXES` (NEW) | N/A | Truncates table ID, includes index ID | | `SHOW CLUSTER RANGES` (NEW) | N/A | Includes table/index ID | | `SHOW CLUSTER RANGES WITH TABLES` (NEW) | N/A | Includes table/index ID | | `SHOW CLUSTER RANGES WITH INDEXES` (NEW) | N/A | Includes table/index ID | In any case, all the columns from `crdb_internal.ranges_no_leases` are included. By default, the start/end key boundaries are pretty-printed as in previous versions. Then: - if `WITH KEYS` is specified, the raw key bytes are exposed alongside the pretty-printed key boundaries. - if `WITH DETAILS` is specified, extra _expensive_ information is included in the result, as of `crdb_internal.ranges`. (requires more roundtrips; makes the operation slower overall) Then: - if `WITH EXPLAIN` is specified, the statement simply returns the text of the SQL query it would use if `WITH EXPLAIN` was not specified. This can be used for learning or troubleshooting.
See text of release notes below for more details; also the explanatory comment at the top of `pkg/sql/delegate/show_ranges.go`. ---- **Example use.**
To test this, use for example the following setup: ``` > -- Enable merge of adjacent ranges with same zone config. > set cluster setting spanconfig.host_coalesce_adjacent.enabled = true; > -- Table t has two indexes with some split points. > create table t(x int primary key, y int); > create index sec_idx on t(y); > alter index t@primary split at values(3); > alter index t@sec_idx split at values(3); > -- Tables u and v share a range with t@sec_idx. > create table u(x int); > create table v(x int); > -- Make some other tables with forced split points due to different > -- zone configs. > create schema otherschema; > create table otherschema.w(x int); > create table otherschema.z(x int); > alter table otherschema.w configure zone using num_replicas = 5; > alter table otherschema.z configure zone using num_replicas = 7; ``` Example output for `SHOW RANGES FROM DATABASE`: ``` > show ranges from database defaultdb; -- 1 row per range > show ranges from current_catalog; -- implicit db from session start_key | end_key | range_id | ... -----------------+----------------+----------+---- /Table/104 | /Table/104/1/3 | 56 | ... /Table/104/1/3 | /Table/104/2 | 57 | ... /Table/104/2 | /Table/104/2/3 | 55 | ... /Table/104/2/3 | /Table/108 | 58 | ... /Table/108 | /Table/109 | 59 | ... /Table/109 | /Max | 60 | ... ``` New syntax: `WITH TABLES` / `WITH INDEXES`: ``` > show ranges from database defaultdb with tables; -- 1 row per range/table intersection start_key | end_key | range_id | schema_name | table_name | table_start_key | table_end_key | ... -----------------+----------------+----------+-------------+------------+-----------------+----------------------+---- /Table/104 | /Table/104/1/3 | 56 | public | t | /Table/104 | /Table/105 | ... /Table/104/1/3 | /Table/104/2 | 57 | public | t | /Table/104 | /Table/105 | ... /Table/104/2 | /Table/104/2/3 | 55 | public | t | /Table/104 | /Table/105 | ... /Table/104/2/3 | /Table/108 | 58 | public | t | /Table/104 | /Table/105 | ... /Table/104/2/3 | /Table/108 | 58 | public | u | /Table/105 | /Table/106 | ... /Table/104/2/3 | /Table/108 | 58 | public | v | /Table/106 | /Table/107 | ... /Table/108 | /Table/109 | 59 | otherschema | w | /Table/108 | /Table/109 | ... /Table/109 | /Max | 60 | otherschema | z | /Table/109 | /Table/109/PrefixEnd | ... ``` ``` > show ranges from database defaultdb with indexes; -- 1 row per range/index intersection start_key | end_key | range_id | schema_name | table_name | index_name | index_start_key | index_end_key | ... -----------------+----------------+----------+-------------+------------+------------+-----------------+---------------+---- /Table/104 | /Table/104/1/3 | 56 | public | t | t_pkey | /Table/104/1 | /Table/104/2 | ... /Table/104/1/3 | /Table/104/2 | 57 | public | t | t_pkey | /Table/104/1 | /Table/104/2 | ... /Table/104/2 | /Table/104/2/3 | 55 | public | t | sec_idx | /Table/104/2 | /Table/104/3 | ... /Table/104/2/3 | /Table/108 | 58 | public | t | sec_idx | /Table/104/2 | /Table/104/3 | ... /Table/104/2/3 | /Table/108 | 58 | public | u | u_pkey | /Table/105/1 | /Table/105/2 | ... /Table/104/2/3 | /Table/108 | 58 | public | v | v_pkey | /Table/106/1 | /Table/106/2 | ... /Table/108 | /Table/109 | 59 | otherschema | w | w_pkey | /Table/108/1 | /Table/108/2 | ... /Table/109 | /Max | 60 | otherschema | z | z_pkey | /Table/109/1 | /Table/109/2 | ... ``` Example output for `SHOW RANGES FROM TABLE`: ``` > show ranges from table t; start_key | end_key | range_id | ... ---------------+--------------------+----------+---- …/ | …/1/3 | 56 | ... …/1/3 | …/2 | 57 | ... …/2 | …/2/3 | 55 | ... …/2/3 | | 58 | ... ``` ``` > show ranges from table u; start_key | end_key | range_id | ... --------------------------+--------------------+----------+---- | | 58 | ... ``` ``` > show ranges from table otherschema.w; start_key | end_key | range_id | ... ---------------+--------------+----------+---- …/ | …/ | 59 | ... ``` New syntax: `SHOW RANGES FROM TABLE ... WITH INDEXES`: ``` > show ranges from table t with indexes; start_key | end_key | range_id | index_name | index_start_key | index_end_key | ... ---------------+--------------------+----------+------------+-----------------+---------------+---- …/ | …/1/3 | 56 | t_pkey | …/1 | …/2 | ... …/1/3 | …/ | 57 | t_pkey | …/1 | …/2 | ... …/ | …/2/3 | 55 | sec_idx | …/2 | …/3 | ... …/2/3 | | 58 | sec_idx | …/2 | …/3 | ... ``` ``` > show ranges from table u with indexes; start_key | end_key | range_id | index_name | index_start_key | index_end_key | ... --------------------------+--------------------+----------+------------+-----------------+---------------+---- | | 58 | u_pkey | …/1 | …/2 | ... ``` ``` > show ranges from table otherschema.w with indexes; start_key | end_key | range_id | index_name | index_start_key | index_end_key | ... ---------------+--------------+----------+------------+-----------------+---------------+---- …/ | …/ | 59 | w_pkey | …/1 | …/2 | ... ``` Example output for `SHOW RANGES FROM INDEX`: ``` > show ranges from index t@t_pkey; start_key | end_key | range_id | ... ---------------+--------------+----------+---- …/ | …/3 | 56 | ... …/3 | …/ | 57 | ... ``` ``` > show ranges from index t@sec_idx; start_key | end_key | range_id | ... ---------------+--------------------+----------+---- …/ | …/3 | 55 | ... …/3 | | 58 | ... ``` ``` > show ranges from index u@u_pkey; start_key | end_key | range_id | ... --------------------------+--------------------+----------+---- | | 58 | ... ``` ``` > show ranges from index otherschema.w@w_pkey; start_key | end_key | range_id | ... ---------------+--------------+----------+---- …/ | …/ | 59 | ... ``` See release notes below for details.
---- ![attention banner](https://media.tenor.com/-AeK-iJpxuoAAAAM/pay-attention-warning.gif) **Backward-incompatible changes.** Release note (backward-incompatible change): CockroachDB now supports sharing storage ranges across multiple indexes/tables. As a result, there is no more guarantee that there is at most one SQL object (e.g. table/index/sequence/materialized view) per storage range. Therefore, the columns `table_id`, `database_name`, `schema_name`, `table_name` and `index_name` in `crdb_internal.ranges` and `.ranges_no_leases` have become nonsensical: a range cannot be attributed to a single table/index any more. As a result: - The aforementioned columns in the `crdb_internal` virtual tables have been removed. Existing code can use the SHOW RANGES statement instead, optionally using WITH KEYS to expose the raw start/end keys. - `SHOW RANGES FROM DATABASE` continues to report one row per range, but stops returning the database / schema / table / index name. - `SHOW RANGES FROM TABLE` continues to report one row per range, but stops returning the index name. Suggested replacements: - Instead of `SELECT range_id FROM crdb_internal.ranges WHERE table_name = 'x'` Use: `SELECT range_id FROM [SHOW RANGES FROM TABLE x]` - Instead of `SELECT range_id FROM crdb_internal.ranges WHERE table_name = $1 OR table_id = $2` (variable / unpredictable table name or ID) Use: `SELECT range_id FROM [SHOW RANGES FROM CURRENT_CATALOG WITH TABLES] WHERE table_name = $1 OR table_id = $2` - Instead of `SELECT start_key FROM crdb_internal.ranges WHERE table_name = 'x'` Use: `SELECT raw_start_key FROM [SHOW RANGES FROM TABLE x WITH KEYS]` - Instead of `SELECT start_key FROM crdb_internal.ranges WHERE table_name = $1 OR table_id = $2` (unpredictable / variable table name or ID) Use: `SELECT raw_start_key FROM [SHOW RANGES FROM CURRENT_CATALOG WITH TABLES, KEYS] WHERE table_name = $1 OR table_id = $2` Release note (backward-incompatible change): The format of the columns `start_key` and `end_key` for `SHOW RANGES FROM DATABASE` and `SHOW RANGES FROM TABLE` have been extended to include which table/index the key belong to. This is necessary because a range can now contain data from more than one table/index. Release note (backward-incompatible change): The format of the columns `start_key` and `end_key` for `SHOW RANGE ... FOR ROW` has been changed to stay consistent with the output of `SHOW RANGES FROM INDEX`. Release note (backward-incompatible change): The output of `SHOW RANGES` does not include `range_size`, `range_size_mb`, `lease_holder` and `lease_holder_localities` any more by default. This ensures that `SHOW RANGES` remains fast in the common case. Use the (NEW) option `WITH DETAILS` to include these columns. ---- **Other changes.** Release note (bug fix): In some cases the start/end key columns of the output of `SHOW RANGES` was missing. This was corrected. Release note (sql change): Two new virtual tables `crdb_internal.index_spans` and `.table_spans` have been introduced, which list the logical keyspace used by each index/table. ---- **New features.** Release note (sql change): The following new statements are introduced: - `SHOW RANGES FROM CURRENT_CATALOG` and `SHOW RANGES` without parameter: alias for `SHOW RANGES FROM DATABASE` on the session's current database. - `SHOW RANGES FROM DATABASE ... WITH TABLES` Reports at least one row per table. It's possible for the same range ID to be repeated across multiple rows, when a range spans multiple tables. - `SHOW RANGES FROM DATABASE ... WITH INDEXES` Reports at least one row per index. It's possible for the same range ID to be repeated across multiple rows, when a range spans multiple indexes. - `SHOW RANGES FROM TABLE ... WITH INDEXES` Reports at least one row per index. It's possible for the same range ID to be repeated across multiple rows, when a range spans multiple indexes. - `SHOW CLUSTER RANGES [ WITH { INDEXES | TABLES } ]` Reports ranges across the entire cluster, including ranges that don't contain table data. The behavior of `WITH INDEXES` and `WITH TABLES` is the same as for `SHOW RANGES FROM DATABASE`. Additionally, the following new options have been added to the `SHOW RANGES` statement: - `WITH KEYS`: produce the raw bytes of the start/end key boundaries. - `WITH DETAILS`: produce more details, using computations that require extra network roundtrips. Makes the operation slower overall. - `WITH EXPLAIN`: produce the text of the SQL query used to run the statement. --- docs/generated/sql/bnf/show_ranges_stmt.bnf | 9 +- docs/generated/sql/bnf/stmt_block.bnf | 16 +- pkg/ccl/backupccl/backup_test.go | 2 +- .../testdata/backup-restore/column-families | 8 +- pkg/ccl/backupccl/utils_test.go | 41 +- pkg/ccl/changefeedccl/changefeed_test.go | 4 +- .../testdata/logic_test/crdb_internal_tenant | 2 + .../partitioning_hash_sharded_index | 47 +- .../logic_test/regional_by_row_query_behavior | 22 +- .../testdata/logic_test/tenant_unsupported | 5 +- pkg/ccl/multiregionccl/region_test.go | 6 +- pkg/cli/zip_test.go | 2 + pkg/cmd/roachtest/tests/BUILD.bazel | 1 + pkg/cmd/roachtest/tests/backup.go | 12 +- pkg/cmd/roachtest/tests/clearrange.go | 4 +- pkg/cmd/roachtest/tests/follower_reads.go | 10 +- pkg/cmd/roachtest/tests/indexes.go | 3 +- pkg/cmd/roachtest/tests/mvcc_gc.go | 23 +- pkg/cmd/roachtest/tests/network.go | 6 +- pkg/cmd/roachtest/tests/rebalance_load.go | 12 +- pkg/kv/kvserver/client_lease_test.go | 4 +- pkg/kv/kvserver/client_protectedts_test.go | 18 +- pkg/kv/kvserver/client_raft_test.go | 2 +- pkg/kv/kvserver/closed_timestamp_test.go | 3 +- pkg/kv/kvserver/gc/gc_int_test.go | 12 +- pkg/kv/kvserver/replicate_queue_test.go | 19 +- pkg/server/admin.go | 13 +- pkg/server/admin_test.go | 4 +- pkg/sql/backfill_protected_timestamp_test.go | 2 +- pkg/sql/catalog/colinfo/result_columns.go | 71 ++ pkg/sql/crdb_internal.go | 203 +++- pkg/sql/delegate/show_range_for_row.go | 34 +- pkg/sql/delegate/show_ranges.go | 774 ++++++++++-- pkg/sql/distsql_physical_planner_test.go | 16 +- pkg/sql/importer/import_into_test.go | 11 +- .../testdata/logic_test/cluster_locks | 16 +- .../testdata/logic_test/crdb_internal | 49 +- .../testdata/logic_test/crdb_internal_catalog | 1042 +++++++++-------- .../testdata/logic_test/dist_vectorize | 28 +- .../logictest/testdata/logic_test/distsql_agg | 54 +- .../testdata/logic_test/distsql_crdb_internal | 24 +- .../testdata/logic_test/distsql_distinct_on | 30 +- .../testdata/logic_test/distsql_numtables | 22 +- .../testdata/logic_test/distsql_stats | 26 +- .../testdata/logic_test/distsql_subquery | 8 +- .../testdata/logic_test/distsql_union | 16 +- .../experimental_distsql_planning_5node | 32 +- .../logictest/testdata/logic_test/grant_table | 2 + .../testdata/logic_test/hash_sharded_index | 26 +- .../testdata/logic_test/information_schema | 11 + .../inverted_filter_geospatial_dist | 30 +- .../logic_test/inverted_join_geospatial_dist | 10 +- .../logic_test/kv_builtin_functions_local | 2 +- .../testdata/logic_test/merge_join_dist | 20 +- .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 798 ++++++------- pkg/sql/logictest/testdata/logic_test/ranges | 474 ++++---- .../logictest/testdata/logic_test/show_ranges | 448 +++++++ .../logictest/testdata/logic_test/show_source | 16 +- .../logictest/testdata/logic_test/split_at | 16 +- .../logictest/testdata/logic_test/sql_keys | 4 +- pkg/sql/logictest/testdata/logic_test/table | 2 + .../logictest/tests/local/generated_test.go | 7 + pkg/sql/multitenant_admin_function_test.go | 14 +- .../exec/execbuilder/testdata/dist_vectorize | 28 +- .../opt/exec/execbuilder/testdata/distsql_agg | 72 +- .../execbuilder/testdata/distsql_distinct_on | 30 +- .../execbuilder/testdata/distsql_group_join | 24 +- .../execbuilder/testdata/distsql_indexjoin | 14 +- .../testdata/distsql_inverted_index | 40 +- .../exec/execbuilder/testdata/distsql_join | 24 +- .../execbuilder/testdata/distsql_merge_join | 60 +- .../exec/execbuilder/testdata/distsql_misc | 24 +- .../execbuilder/testdata/distsql_numtables | 20 +- .../execbuilder/testdata/distsql_ordinality | 14 +- .../testdata/distsql_tighten_spans | 36 +- .../exec/execbuilder/testdata/distsql_union | 14 +- .../exec/execbuilder/testdata/distsql_window | 24 +- .../experimental_distsql_planning_5node | 32 +- .../testdata/explain_analyze_plans | 32 +- .../testdata/inverted_filter_geospatial_dist | 32 +- .../testdata/inverted_filter_json_array_dist | 20 +- .../testdata/inverted_join_geospatial_dist | 10 +- .../inverted_join_geospatial_dist_vec | 10 +- .../testdata/inverted_join_json_array_dist | 18 +- .../testdata/inverted_join_multi_column_dist | 20 +- .../opt/exec/execbuilder/testdata/lookup_join | 30 +- .../execbuilder/testdata/merge_join_dist_vec | 20 +- .../exec/execbuilder/testdata/scan_parallel | 24 +- pkg/sql/parser/sql.y | 91 +- pkg/sql/parser/testdata/show | 145 +++ pkg/sql/scatter_test.go | 5 +- pkg/sql/sem/catconstants/constants.go | 2 + pkg/sql/sem/tree/show.go | 107 +- pkg/sql/show_ranges_test.go | 9 +- pkg/sql/tests/truncate_test.go | 4 +- pkg/sql/ttl/ttljob/ttljob_test.go | 2 +- pkg/sql/unsplit.go | 32 +- pkg/sql/virtual_schema.go | 23 + 99 files changed, 3668 insertions(+), 2144 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/show_ranges diff --git a/docs/generated/sql/bnf/show_ranges_stmt.bnf b/docs/generated/sql/bnf/show_ranges_stmt.bnf index 684a456f8d9d..5a369f54b195 100644 --- a/docs/generated/sql/bnf/show_ranges_stmt.bnf +++ b/docs/generated/sql/bnf/show_ranges_stmt.bnf @@ -1,4 +1,7 @@ show_ranges_stmt ::= - 'SHOW' 'RANGES' 'FROM' 'TABLE' table_name - | 'SHOW' 'RANGES' 'FROM' 'INDEX' table_index_name - | 'SHOW' 'RANGES' 'FROM' 'DATABASE' database_name + 'SHOW' 'RANGES' 'FROM' 'INDEX' table_index_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'TABLE' table_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'DATABASE' database_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'CURRENT_CATALOG' opt_show_ranges_options + | 'SHOW' 'RANGES' opt_show_ranges_options + | 'SHOW' 'CLUSTER' 'RANGES' opt_show_ranges_options diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index ae53ffedcb52..2bc6fb657204 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -867,9 +867,12 @@ show_statements_stmt ::= | 'SHOW' 'ALL' opt_cluster statements_or_queries show_ranges_stmt ::= - 'SHOW' 'RANGES' 'FROM' 'TABLE' table_name - | 'SHOW' 'RANGES' 'FROM' 'INDEX' table_index_name - | 'SHOW' 'RANGES' 'FROM' 'DATABASE' database_name + 'SHOW' 'RANGES' 'FROM' 'INDEX' table_index_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'TABLE' table_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'DATABASE' database_name opt_show_ranges_options + | 'SHOW' 'RANGES' 'FROM' 'CURRENT_CATALOG' opt_show_ranges_options + | 'SHOW' 'RANGES' opt_show_ranges_options + | 'SHOW' 'CLUSTER' 'RANGES' opt_show_ranges_options show_range_for_row_stmt ::= 'SHOW' 'RANGE' 'FROM' 'TABLE' table_name 'FOR' 'ROW' '(' expr_list ')' @@ -1077,6 +1080,7 @@ unreserved_keyword ::= | 'DEPENDS' | 'DESTINATION' | 'DETACHED' + | 'DETAILS' | 'DISCARD' | 'DOMAIN' | 'DOUBLE' @@ -1950,6 +1954,9 @@ statements_or_queries ::= 'STATEMENTS' | 'QUERIES' +opt_show_ranges_options ::= + 'WITH' show_ranges_options + opt_compact ::= 'COMPACT' | @@ -2645,6 +2652,9 @@ targets_roles ::= | 'TYPE' type_name_list | grant_targets +show_ranges_options ::= + ( 'TABLES' | 'INDEXES' | 'DETAILS' | 'KEYS' | 'EXPLAIN' ) ( ( ',' 'TABLES' | ',' 'INDEXES' | ',' 'DETAILS' | ',' 'EXPLAIN' | ',' 'KEYS' ) )* + partition ::= 'PARTITION' partition_name diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index fe5e6ae70bee..d893b81d8b0b 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -1386,7 +1386,7 @@ WITH AS prev_lease_holder, lease_holder FROM - [SHOW RANGES FROM TABLE data.bank] + [SHOW RANGES FROM TABLE data.bank WITH DETAILS] ) SELECT count(*) diff --git a/pkg/ccl/backupccl/testdata/backup-restore/column-families b/pkg/ccl/backupccl/testdata/backup-restore/column-families index 9861fb8b1c22..2cfd4b656c6d 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/column-families +++ b/pkg/ccl/backupccl/testdata/backup-restore/column-families @@ -47,6 +47,10 @@ RESTORE cfs FROM LATEST IN 'nodelocal://1/foo' WITH into_db='r1'; ---- query-sql -SELECT max(length(start_key)) FROM [SHOW RANGES FROM TABLE orig.cfs]; +SELECT start_key FROM [SHOW RANGES FROM TABLE orig.cfs] ---- -2 +…/ +…/1/0 +…/1/1 +…/1/2 +…/1/3 diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 248b7da527a3..13709ad80834 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -399,11 +400,8 @@ func waitForTableSplit(t *testing.T, conn *gosql.DB, tableName, dbName string) { testutils.SucceedsSoon(t, func() error { count := 0 if err := conn.QueryRow( - "SELECT count(*) "+ - "FROM crdb_internal.ranges_no_leases "+ - "WHERE table_name = $1 "+ - "AND database_name = $2", - tableName, dbName).Scan(&count); err != nil { + fmt.Sprintf("SELECT count(*) FROM [SHOW RANGES FROM TABLE %s.%s]", + tree.NameString(dbName), tree.NameString(tableName))).Scan(&count); err != nil { return err } if count == 0 { @@ -416,13 +414,8 @@ func waitForTableSplit(t *testing.T, conn *gosql.DB, tableName, dbName string) { func getTableStartKey(t *testing.T, conn *gosql.DB, tableName, dbName string) roachpb.Key { t.Helper() row := conn.QueryRow( - "SELECT start_key "+ - "FROM crdb_internal.ranges_no_leases "+ - "WHERE table_name = $1 "+ - "AND database_name = $2 "+ - "ORDER BY start_key ASC "+ - "LIMIT 1", - tableName, dbName) + fmt.Sprintf(`SELECT crdb_internal.table_span('%s.%s'::regclass::oid::int)[1]`, + tree.NameString(dbName), tree.NameString(tableName))) var startKey roachpb.Key require.NoError(t, row.Scan(&startKey)) return startKey @@ -450,10 +443,22 @@ func getStoreAndReplica( ) (*kvserver.Store, *kvserver.Replica) { t.Helper() startKey := getTableStartKey(t, conn, tableName, dbName) + // Okay great now we have a key and can go find replicas and stores and what not. r := tc.LookupRangeOrFatal(t, startKey) - l, _, err := tc.FindRangeLease(r, nil) - require.NoError(t, err) + + var l roachpb.Lease + testutils.SucceedsSoon(t, func() error { + var err error + l, _, err = tc.FindRangeLease(r, nil) + if err != nil { + return err + } + if l.Replica.NodeID == 0 { + return errors.New("range does not have a lease yet") + } + return nil + }) lhServer := tc.Server(int(l.Replica.NodeID) - 1) return getFirstStoreReplica(t, lhServer, startKey) @@ -572,10 +577,10 @@ func runGCAndCheckTraceOnCluster( t.Helper() var startKey roachpb.Key testutils.SucceedsSoon(t, func() error { - err := runner.DB.QueryRowContext(ctx, ` -SELECT start_key FROM crdb_internal.ranges_no_leases -WHERE table_name = $1 AND database_name = $2 -ORDER BY start_key ASC`, tableName, databaseName).Scan(&startKey) + err := runner.DB.QueryRowContext(ctx, fmt.Sprintf(` +SELECT raw_start_key +FROM [SHOW RANGES FROM TABLE %s.%s WITH KEYS] +ORDER BY raw_start_key ASC`, tree.NameString(databaseName), tree.NameString(tableName))).Scan(&startKey) if err != nil { return errors.Wrap(err, "failed to query start_key ") } diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 8f331e7e96b2..566ae519327a 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2763,7 +2763,7 @@ func TestChangefeedRestartMultiNode(t *testing.T) { sqlDB.Exec(t, `CREATE TABLE test_tab (a INT PRIMARY KEY, b INT UNIQUE NOT NULL)`) sqlDB.Exec(t, `INSERT INTO test_tab VALUES (0, 0)`) - row := sqlDB.QueryRow(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test_tab] LIMIT 1`) + row := sqlDB.QueryRow(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test_tab WITH DETAILS] LIMIT 1`) var rangeID, leaseHolder int row.Scan(&rangeID, &leaseHolder) @@ -2821,7 +2821,7 @@ func TestChangefeedStopPolicyMultiNode(t *testing.T) { sqlDB.Exec(t, `CREATE TABLE test_tab (a INT PRIMARY KEY)`) sqlDB.Exec(t, `INSERT INTO test_tab VALUES (0)`) - row := sqlDB.QueryRow(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test_tab] LIMIT 1`) + row := sqlDB.QueryRow(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test_tab WITH DETAILS] LIMIT 1`) var rangeID, leaseHolder int row.Scan(&rangeID, &leaseHolder) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index d87bc03ba26b..99b7ecc68a8d 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -54,6 +54,7 @@ crdb_internal gossip_liveness table admin NULL NULL crdb_internal gossip_network table admin NULL NULL crdb_internal gossip_nodes table admin NULL NULL crdb_internal index_columns table admin NULL NULL +crdb_internal index_spans table admin NULL NULL crdb_internal index_usage_statistics table admin NULL NULL crdb_internal invalid_objects table admin NULL NULL crdb_internal jobs table admin NULL NULL @@ -93,6 +94,7 @@ crdb_internal system_jobs table admin NULL NULL crdb_internal table_columns table admin NULL NULL crdb_internal table_indexes table admin NULL NULL crdb_internal table_row_statistics table admin NULL NULL +crdb_internal table_spans table admin NULL NULL crdb_internal tables table admin NULL NULL crdb_internal tenant_usage_details view admin NULL NULL crdb_internal transaction_contention_events table admin NULL NULL diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index index 881aa7337ec2..3da44e5e06a2 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_hash_sharded_index @@ -257,28 +257,27 @@ CREATE INDEX t_presplit_idx_member_id ON t_presplit (member_id) USING HASH WITH skipif config 3node-tenant query TITTT colnames,retry -SELECT t.name, r.table_id, r.index_name, r.start_pretty, r.end_pretty -FROM crdb_internal.tables t -JOIN crdb_internal.ranges r ON t.table_id = r.table_id -WHERE t.name = 't_presplit' -AND t.state = 'PUBLIC' -AND r.split_enforced_until IS NOT NULL; +SELECT table_name, table_id, index_name, start_key, end_key + FROM [SHOW RANGES FROM DATABASE test WITH INDEXES] + WHERE table_name = 't_presplit' +ORDER BY start_key ---- -name table_id index_name start_pretty end_pretty -t_presplit 112 t_presplit_idx_member_id /Table/112/2 /Table/112/2/"new york"/0 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/0 /Table/112/2/"new york"/1 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/1 /Table/112/2/"new york"/2 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/2 /Table/112/2/"new york"/3 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/3 /Table/112/2/"new york"/4 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/4 /Table/112/2/"new york"/5 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/5 /Table/112/2/"new york"/6 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/6 /Table/112/2/"new york"/7 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/7 /Table/112/2/"seattle"/0 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/0 /Table/112/2/"seattle"/1 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/1 /Table/112/2/"seattle"/2 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/2 /Table/112/2/"seattle"/3 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/3 /Table/112/2/"seattle"/4 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/4 /Table/112/2/"seattle"/5 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/5 /Table/112/2/"seattle"/6 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/6 /Table/112/2/"seattle"/7 -t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/7 /Table/112/3/"new york"/0 +table_name table_id index_name start_key end_key +t_presplit 112 t_presplit_pkey /Table/109/11/"seattle"/15 /Table/112/2 +t_presplit 112 t_presplit_idx_member_id /Table/112/2 /Table/112/2/"new york"/0 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/0 /Table/112/2/"new york"/1 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/1 /Table/112/2/"new york"/2 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/2 /Table/112/2/"new york"/3 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/3 /Table/112/2/"new york"/4 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/4 /Table/112/2/"new york"/5 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/5 /Table/112/2/"new york"/6 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/6 /Table/112/2/"new york"/7 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"new york"/7 /Table/112/2/"seattle"/0 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/0 /Table/112/2/"seattle"/1 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/1 /Table/112/2/"seattle"/2 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/2 /Table/112/2/"seattle"/3 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/3 /Table/112/2/"seattle"/4 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/4 /Table/112/2/"seattle"/5 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/5 /Table/112/2/"seattle"/6 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/6 /Table/112/2/"seattle"/7 +t_presplit 112 t_presplit_idx_member_id /Table/112/2/"seattle"/7 /Table/112/3/"new york"/0 diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_query_behavior b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_query_behavior index 2dde13f0ba99..21664472d52d 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_query_behavior +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row_query_behavior @@ -262,7 +262,7 @@ ap-southeast-2 23 query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE regional_by_row_table FOR ROW ('ap-southeast-2', 1)] ---- -NULL NULL + query TIIII SELECT crdb_region, pk, pk2, a, b FROM regional_by_row_table @@ -392,12 +392,12 @@ ALTER TABLE regional_by_row_table SPLIT AT VALUES ('ca-central-1', 0), ('us-east ALTER TABLE regional_by_row_table EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'ap-southeast-2', 0), (ARRAY[4], 'ca-central-1', 0), (ARRAY[7], 'us-east-1', 0); query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX regional_by_row_table@primary] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX regional_by_row_table@primary WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /"\x80"/0 {1} 1 -/"\x80"/0 /"\xc0"/0 {4} 4 -/"\xc0"/0 NULL {7} 7 +start_key end_key replicas lease_holder + …/"\x80"/0 {1} 1 +…/"\x80"/0 …/"\xc0"/0 {4} 4 +…/"\xc0"/0 {7} 7 statement ok SET locality_optimized_partitioned_index_scan = false @@ -640,12 +640,12 @@ ALTER TABLE child SPLIT AT VALUES ('ca-central-1', 0), ('us-east-1', 0); ALTER TABLE child EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'ap-southeast-2', 0), (ARRAY[4], 'ca-central-1', 0), (ARRAY[7], 'us-east-1', 0); query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX child@primary] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX child@primary WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /"\x80"/0 {1} 1 -/"\x80"/0 /"\xc0"/0 {4} 4 -/"\xc0"/0 NULL {7} 7 +start_key end_key replicas lease_holder + …/"\x80"/0 {1} 1 +…/"\x80"/0 …/"\xc0"/0 {4} 4 +…/"\xc0"/0 {7} 7 statement ok SET locality_optimized_partitioned_index_scan = false diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported b/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported index 94161208f9bd..65aa736e17fd 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported @@ -51,6 +51,7 @@ ALTER TABLE kv SPLIT AT VALUES ('foo') statement error pq: could not UNSPLIT AT \('foo'\): rpc error: code = Unauthenticated desc = request \[1 AdmUnsplit\] not permitted ALTER TABLE kv UNSPLIT AT VALUES ('foo') +#UNSPLIT ALL succeeds because there is no split point at this time. statement ok ALTER TABLE kv UNSPLIT ALL @@ -70,10 +71,10 @@ statement error operation is unsupported in multi-tenancy mode ALTER RANGE 1 RELOCATE LEASE TO 2 statement error operation is unsupported in multi-tenancy mode -ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id from crdb_internal.ranges where table_name = 'kv' +ALTER RANGE RELOCATE LEASE TO 2 FOR SELECT range_id FROM [SHOW RANGES FROM TABLE kv] statement error operation is unsupported in multi-tenancy mode ALTER RANGE 1 RELOCATE FROM 1 TO 2 statement error operation is unsupported in multi-tenancy mode -ALTER RANGE RELOCATE FROM 1 TO 2 FOR SELECT range_id from crdb_internal.ranges where table_name = 'kv' +ALTER RANGE RELOCATE FROM 1 TO 2 FOR SELECT range_id FROM [SHOW RANGES FROM TABLE kv] diff --git a/pkg/ccl/multiregionccl/region_test.go b/pkg/ccl/multiregionccl/region_test.go index ef6fc259b19c..96e4258b5edf 100644 --- a/pkg/ccl/multiregionccl/region_test.go +++ b/pkg/ccl/multiregionccl/region_test.go @@ -57,7 +57,7 @@ func TestMultiRegionDatabaseStats(t *testing.T) { _, err := db.ExecContext(ctx, `CREATE DATABASE test PRIMARY REGION "us-west"; - use test; + USE test; CREATE TABLE a(id uuid primary key);`) require.NoError(t, err) @@ -66,8 +66,8 @@ func TestMultiRegionDatabaseStats(t *testing.T) { testutils.SucceedsWithin(t, func() error { // Get the list of nodes from ranges table row := db.QueryRowContext(ctx, - `use test; - with x as (show ranges from table a) select replicas from x;`) + `USE test; + WITH x AS (SHOW RANGES FROM TABLE a) SELECT replicas FROM x;`) var nodesStr string err = row.Scan(&nodesStr) if err != nil { diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 513ac0edd491..2d5eb3887c96 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -78,6 +78,7 @@ table_name NOT IN ( 'forward_dependencies', 'gossip_network', 'index_columns', + 'index_spans', 'kv_catalog_comments', 'kv_catalog_descriptor', 'kv_catalog_namespace', @@ -90,6 +91,7 @@ table_name NOT IN ( 'predefined_comments', 'session_trace', 'session_variables', + 'table_spans', 'tables', 'cluster_statement_statistics', 'cluster_transaction_statistics', diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index c2d8e57198d3..34d934d444eb 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -204,6 +204,7 @@ go_library( "//pkg/sql", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/sem/tree", "//pkg/storage/enginepb", "//pkg/testutils", "//pkg/testutils/skip", diff --git a/pkg/cmd/roachtest/tests/backup.go b/pkg/cmd/roachtest/tests/backup.go index d53d54f39ead..cccc163ccd25 100644 --- a/pkg/cmd/roachtest/tests/backup.go +++ b/pkg/cmd/roachtest/tests/backup.go @@ -1281,10 +1281,9 @@ func runBackupMVCCRangeTombstones(ctx context.Context, t test.Test, c cluster.Cl // Check that we actually wrote MVCC range tombstones. var rangeKeys int require.NoError(t, conn.QueryRowContext(ctx, ` - SELECT sum((crdb_internal.range_stats(start_key)->'range_key_count')::INT) - FROM crdb_internal.ranges - WHERE database_name = 'tpch' AND table_name = 'orders' - `).Scan(&rangeKeys)) + SELECT sum((crdb_internal.range_stats(raw_start_key)->'range_key_count')::INT) + FROM [SHOW RANGES FROM TABLE tpch.orders WITH KEYS] +`).Scan(&rangeKeys)) require.NotZero(t, rangeKeys, "no MVCC range tombstones found") // Fingerprint for restore comparison, and assert that it matches the initial @@ -1327,9 +1326,8 @@ func runBackupMVCCRangeTombstones(ctx context.Context, t test.Test, c cluster.Cl err = conn.QueryRowContext(ctx, ` SELECT range_id, stats::STRING FROM [ - SELECT range_id, crdb_internal.range_stats(start_key) AS stats - FROM crdb_internal.ranges - WHERE database_name = 'tpch' + SELECT range_id, crdb_internal.range_stats(raw_start_key) AS stats + FROM [SHOW RANGES FROM DATABASE tpch WITH KEYS] ] WHERE (stats->'live_count')::INT != 0 OR ( (stats->'key_count')::INT > 0 AND (stats->'range_key_count')::INT = 0 diff --git a/pkg/cmd/roachtest/tests/clearrange.go b/pkg/cmd/roachtest/tests/clearrange.go index e17c5feed671..6e622cd71bbb 100644 --- a/pkg/cmd/roachtest/tests/clearrange.go +++ b/pkg/cmd/roachtest/tests/clearrange.go @@ -134,7 +134,9 @@ func runClearRange( var startHex string if err := conn.QueryRow( - `SELECT to_hex(start_key) FROM crdb_internal.ranges_no_leases WHERE database_name = 'bigbank' AND table_name = 'bank' ORDER BY start_key ASC LIMIT 1`, + `SELECT to_hex(raw_start_key) +FROM [SHOW RANGES FROM TABLE bigbank.bank WITH KEYS] +ORDER BY raw_start_key ASC LIMIT 1`, ).Scan(&startHex); err != nil { t.Fatal(err) } diff --git a/pkg/cmd/roachtest/tests/follower_reads.go b/pkg/cmd/roachtest/tests/follower_reads.go index a2ee91ef5a7b..c034879f0256 100644 --- a/pkg/cmd/roachtest/tests/follower_reads.go +++ b/pkg/cmd/roachtest/tests/follower_reads.go @@ -477,10 +477,7 @@ func initFollowerReadsDB( q1 := fmt.Sprintf(` SELECT %s, %s - FROM - crdb_internal.ranges_no_leases - WHERE - table_name = 'test'`, votersCol, nonVotersCol) + FROM [SHOW RANGES FROM TABLE test]`, votersCol, nonVotersCol) var voters, nonVoters int err := db.QueryRowContext(ctx, q1).Scan(&voters, &nonVoters) @@ -515,10 +512,7 @@ func initFollowerReadsDB( const q2 = ` SELECT count(distinct substring(unnest(replica_localities), 'region=([^,]*)')) - FROM - crdb_internal.ranges_no_leases - WHERE - table_name = 'test'` + FROM [SHOW RANGES FROM TABLE test]` var distinctRegions int require.NoError(t, db.QueryRowContext(ctx, q2).Scan(&distinctRegions)) diff --git a/pkg/cmd/roachtest/tests/indexes.go b/pkg/cmd/roachtest/tests/indexes.go index 3e46b1033aa8..d40610c15056 100644 --- a/pkg/cmd/roachtest/tests/indexes.go +++ b/pkg/cmd/roachtest/tests/indexes.go @@ -98,8 +98,7 @@ func registerNIndexes(r registry.Registry, secondaryIndexes int) { var ok bool if err := conn.QueryRowContext(ctx, ` SELECT lease_holder <= $1 - FROM crdb_internal.ranges - WHERE table_name = 'indexes'`, + FROM [SHOW RANGES FROM TABLE indexes WITH DETAILS]`, nodes/3, ).Scan(&ok); err != nil { return err diff --git a/pkg/cmd/roachtest/tests/mvcc_gc.go b/pkg/cmd/roachtest/tests/mvcc_gc.go index 85fd38b00726..ce34d3a56b03 100644 --- a/pkg/cmd/roachtest/tests/mvcc_gc.go +++ b/pkg/cmd/roachtest/tests/mvcc_gc.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/roachprod/install" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -264,16 +265,11 @@ func collectTableMVCCStatsOrFatal( t test.Test, conn *gosql.DB, m tableMetadata, ) (enginepb.MVCCStats, int) { t.Helper() - rows, err := conn.Query(` -select - range_id, start_key, crdb_internal.range_stats(start_key) -from - crdb_internal.ranges_no_leases -where - database_name = $1 and table_name = $2 -order by - start_key`, - m.databaseName, m.tableName) + rows, err := conn.Query(fmt.Sprintf(` +SELECT range_id, raw_start_key, crdb_internal.range_stats(raw_start_key) +FROM [SHOW RANGES FROM %s.%s WITH KEYS] +ORDER BY r.start_key`, + tree.NameString(m.databaseName), tree.NameString(m.tableName))) if err != nil { t.Fatalf("failed to run consistency check query on table: %s", err) } @@ -314,10 +310,11 @@ func collectStatsAndConsistencyOrFail( t.Helper() var startKey, endKey roachpb.Key queryRowOrFatal(t, conn, - `select min(start_key), max(end_key) from crdb_internal.ranges_no_leases where database_name = $1 and table_name = $2`, - []interface{}{m.tableName, m.databaseName}, []interface{}{&startKey, &endKey}) + fmt.Sprintf(`SELECT min(raw_start_key), max(raw_end_key) +FROM [SHOW RANGES FROM TABLE %s.%s WITH KEYS]`, + tree.NameString(m.tableName), tree.NameString(m.databaseName)), nil, []interface{}{&startKey, &endKey}) - rows, err := conn.Query(`select range_id, start_key, status, detail, crdb_internal.range_stats(start_key) from crdb_internal.check_consistency(false, $1, $2) order by start_key`, + rows, err := conn.Query(`SELECT range_id, start_key, status, detail, crdb_internal.range_stats(start_key) FROM crdb_internal.check_consistency(false, $1, $2) ORDER BY start_key`, startKey, endKey) if err != nil { t.Fatalf("failed to run consistency check query on table: %s", err) diff --git a/pkg/cmd/roachtest/tests/network.go b/pkg/cmd/roachtest/tests/network.go index 83c81cea26b8..d00dd33b749d 100644 --- a/pkg/cmd/roachtest/tests/network.go +++ b/pkg/cmd/roachtest/tests/network.go @@ -138,9 +138,9 @@ func runNetworkAuthentication(ctx context.Context, t test.Test, c cluster.Cluste const waitLeases = ` SELECT $1::INT = ALL ( SELECT lease_holder - FROM crdb_internal.ranges - WHERE (start_pretty = '/System/NodeLiveness' AND end_pretty = '/System/NodeLivenessMax') - OR (database_name = 'system' AND table_name IN ('users', 'role_members', 'role_options')) + FROM [SHOW CLUSTER RANGES WITH TABLES, DETAILS] + WHERE (start_key = '/System/NodeLiveness' AND end_key = '/System/NodeLivenessMax') + OR (table_name IN ('users', 'role_members', 'role_options') )` t.L().Printf("SQL: %s", waitLeases) require.NoError(t, db.QueryRow(waitLeases, expectedLeaseholder).Scan(&ok)) diff --git a/pkg/cmd/roachtest/tests/rebalance_load.go b/pkg/cmd/roachtest/tests/rebalance_load.go index d3223a0000d2..ddf3d68e770e 100644 --- a/pkg/cmd/roachtest/tests/rebalance_load.go +++ b/pkg/cmd/roachtest/tests/rebalance_load.go @@ -239,17 +239,17 @@ func registerRebalanceLoad(r registry.Registry) { func isLoadEvenlyDistributed(l *logger.Logger, db *gosql.DB, numStores int) (bool, error) { rows, err := db.Query( - `select lease_holder, count(*) ` + - `from [show ranges from table kv.kv] ` + - `group by lease_holder;`) + `SELECT lease_holder, count(*) ` + + `FROM [SHOW RANGES FROM TABLE kv.kv WITH DETAILS] ` + + `GROUP BY lease_holder;`) if err != nil { // TODO(rafi): Remove experimental_ranges query once we stop testing 19.1 or // earlier. if strings.Contains(err.Error(), "syntax error at or near \"ranges\"") { rows, err = db.Query( - `select lease_holder, count(*) ` + - `from [show experimental_ranges from table kv.kv] ` + - `group by lease_holder;`) + `SELECT lease_holder, count(*) ` + + `FROM [SHOW RANGES FROM TABLE kv.kv WITH DETAILS] ` + + `GROUP BY lease_holder;`) } } if err != nil { diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index 284a7429058f..2111c5e16db9 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -45,7 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" - "go.etcd.io/raft/v3" + raft "go.etcd.io/raft/v3" "go.etcd.io/raft/v3/tracker" ) @@ -1316,7 +1316,7 @@ func TestAlterRangeRelocate(t *testing.T) { }) // Move lease 3 -> 5. - _, err = db.Exec("ALTER RANGE RELOCATE FROM $1 TO $2 FOR (SELECT range_id from crdb_internal.ranges where range_id = $3)", 3, 5, rhsDesc.RangeID) + _, err = db.Exec("ALTER RANGE RELOCATE FROM $1 TO $2 FOR (SELECT range_id from crdb_internal.ranges_no_leases where range_id = $3)", 3, 5, rhsDesc.RangeID) require.NoError(t, err) require.NoError(t, tc.WaitForVoters(rhsDesc.StartKey.AsRawKey(), tc.Targets(0, 3, 4)...)) } diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index d49261ac1f99..b4ec681cd048 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -96,11 +96,7 @@ func TestProtectedTimestamps(t *testing.T) { testutils.SucceedsSoon(t, func() error { count := 0 if err := conn.QueryRow( - "SELECT count(*) "+ - "FROM crdb_internal.ranges_no_leases "+ - "WHERE table_name = $1 "+ - "AND database_name = current_database()", - "foo").Scan(&count); err != nil { + "SELECT count(*) FROM [SHOW RANGES FROM TABLE foo]").Scan(&count); err != nil { return err } if count == 0 { @@ -111,14 +107,10 @@ func TestProtectedTimestamps(t *testing.T) { } getTableStartKey := func() roachpb.Key { - row := conn.QueryRow( - "SELECT start_key "+ - "FROM crdb_internal.ranges_no_leases "+ - "WHERE table_name = $1 "+ - "AND database_name = current_database() "+ - "ORDER BY start_key ASC "+ - "LIMIT 1", - "foo") + row := conn.QueryRow(` +SELECT raw_start_key +FROM [SHOW RANGES FROM TABLE foo WITH KEYS] +ORDER BY raw_start_key ASC LIMIT 1`) var startKey roachpb.Key require.NoError(t, row.Scan(&startKey)) diff --git a/pkg/kv/kvserver/client_raft_test.go b/pkg/kv/kvserver/client_raft_test.go index 5ec94f847a12..13c5e159c997 100644 --- a/pkg/kv/kvserver/client_raft_test.go +++ b/pkg/kv/kvserver/client_raft_test.go @@ -5750,7 +5750,7 @@ func TestElectionAfterRestart(t *testing.T) { testutils.SucceedsSoon(t, func() error { for _, row := range sqlutils.MakeSQLRunner(tc.Conns[0]).QueryStr( - t, `SELECT range_id FROM crdb_internal.ranges_no_leases WHERE table_name = 't';`, + t, `SELECT range_id FROM [SHOW RANGES FROM TABLE t]`, ) { n, err := strconv.Atoi(row[0]) require.NoError(t, err) diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index 1acfd480404b..e8cc1350a1d3 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -1233,8 +1233,7 @@ RESET statement_timeout; testutils.SucceedsSoon(t, func() error { if err := db0.QueryRow( fmt.Sprintf( - `SELECT array_length(replicas, 1) FROM crdb_internal.ranges -WHERE table_name = '%s' AND database_name = '%s'`, tableName, dbName), + `SELECT array_length(replicas, 1) FROM [SHOW RANGES FROM TABLE %s.%s]`, dbName, tableName), ).Scan(&numReplicas); err != nil { return err } diff --git a/pkg/kv/kvserver/gc/gc_int_test.go b/pkg/kv/kvserver/gc/gc_int_test.go index fb1c26f70c69..325067ba2a3c 100644 --- a/pkg/kv/kvserver/gc/gc_int_test.go +++ b/pkg/kv/kvserver/gc/gc_int_test.go @@ -111,7 +111,7 @@ func TestEndToEndGC(t *testing.T) { getTableRangeIDs := func(t *testing.T, db *gosql.DB) ids { t.Helper() - rows, err := db.Query("with r as (show ranges from table kv) select range_id from r order by start_key") + rows, err := db.Query("WITH r AS (SHOW RANGES FROM TABLE kv) SELECT range_id FROM r ORDER BY start_key") require.NoError(t, err, "failed to query ranges") var rangeIDs []int64 for rows.Next() { @@ -125,7 +125,7 @@ func TestEndToEndGC(t *testing.T) { readSomeKeys := func(t *testing.T, db *gosql.DB) []int64 { t.Helper() var ids []int64 - rows, err := db.Query("select k from kv limit 5") + rows, err := db.Query("SELECT k FROM kv LIMIT 5") require.NoError(t, err, "failed to query kv data") for rows.Next() { var id int64 @@ -137,7 +137,7 @@ func TestEndToEndGC(t *testing.T) { getRangeInfo := func(t *testing.T, rangeID int64, db *gosql.DB) (startKey, endKey []byte) { t.Helper() - row := db.QueryRow("select start_key, end_key from crdb_internal.ranges_no_leases where range_id=$1", + row := db.QueryRow("SELECT start_key, end_key FROM crdb_internal.ranges_no_leases WHERE range_id=$1", rangeID) require.NoError(t, row.Err(), "failed to query range info") require.NoError(t, row.Scan(&startKey, &endKey), "failed to scan range info") @@ -202,10 +202,10 @@ func TestEndToEndGC(t *testing.T) { execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 0`) } - execOrFatal(t, sqlDb, `create table kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`) + execOrFatal(t, sqlDb, `CREATE TABLE kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`) for i := 0; i < 1000; i++ { - execOrFatal(t, sqlDb, "upsert into kv values ($1, $2)", rng.Int63(), "hello") + execOrFatal(t, sqlDb, "UPSERT INTO kv VALUES ($1, $2)", rng.Int63(), "hello") } t.Logf("found table range after initializing table data: %s", getTableRangeIDs(t, sqlDb)) @@ -236,7 +236,7 @@ func TestEndToEndGC(t *testing.T) { if d.rangeTombstones { deleteRangeDataWithRangeTombstone(t, kvDb, sqlDb) } else { - execOrFatal(t, sqlDb, "delete from kv where k is not null") + execOrFatal(t, sqlDb, "DELETE FROM kv WHERE k IS NOT NULL") } t.Logf("found table ranges after range deletion: %s", getTableRangeIDs(t, sqlDb)) diff --git a/pkg/kv/kvserver/replicate_queue_test.go b/pkg/kv/kvserver/replicate_queue_test.go index e21d46ed01ef..2593ec195d77 100644 --- a/pkg/kv/kvserver/replicate_queue_test.go +++ b/pkg/kv/kvserver/replicate_queue_test.go @@ -1781,9 +1781,9 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) { for i := 0; i < 1.5*rangeMaxSize; i++ { sb.WriteRune('a') } - _, err = db.Exec("insert into t(i,s) values (1, $1)", sb.String()) + _, err = db.Exec("INSERT INTO t(i,s) VALUES (1, $1)", sb.String()) require.NoError(t, err) - _, err = db.Exec("insert into t(i,s) values (2, 'b')") + _, err = db.Exec("INSERT INTO t(i,s) VALUES (2, 'b')") require.NoError(t, err) // Now ask everybody to up-replicate. @@ -1804,11 +1804,12 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) { testutils.SucceedsSoon(t, func() error { forceProcess() r := db.QueryRow( - "select replicas from [show ranges from table t] where start_key='/2'") + "SELECT replicas FROM [SHOW RANGES FROM TABLE t] WHERE start_key LIKE '%/2'") var repl string if err := r.Scan(&repl); err != nil { return err } + t.Logf("replicas: %v", repl) if repl != "{1,2,3,4,5}" { return fmt.Errorf("not up-replicated yet. replicas: %s", repl) } @@ -1819,7 +1820,7 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) { testutils.SucceedsSoon(t, func() error { forceProcess() r := db.QueryRow( - "select replicas from [show ranges from table t] where start_key is null") + "SELECT replicas FROM [SHOW RANGES FROM TABLE t] WHERE start_key LIKE '%TableMin%'") var repl string if err := r.Scan(&repl); err != nil { return err @@ -1902,10 +1903,10 @@ func TestTransferLeaseToLaggingNode(t *testing.T) { var rangeID roachpb.RangeID var leaseHolderNodeID uint64 s := sqlutils.MakeSQLRunner(tc.Conns[0]) - s.Exec(t, "insert into system.comments values(0,0,0,'abc')") + s.Exec(t, "INSERT INTO system.comments VALUES(0,0,0,'abc')") s.QueryRow(t, - "select range_id, lease_holder from "+ - "[show ranges from table system.comments] limit 1", + "SELECT range_id, lease_holder FROM "+ + "[SHOW RANGES FROM TABLE system.comments WITH DETAILS] LIMIT 1", ).Scan(&rangeID, &leaseHolderNodeID) remoteNodeID := uint64(1) if leaseHolderNodeID == 1 { @@ -2250,7 +2251,7 @@ func TestPromoteNonVoterInAddVoter(t *testing.T) { } var rangeID roachpb.RangeID - if err := db.QueryRow("select range_id from [show ranges from table t] limit 1").Scan(&rangeID); err != nil { + if err := db.QueryRow("SELECT range_id FROM [SHOW RANGES FROM TABLE t] LIMIT 1").Scan(&rangeID); err != nil { return 0, 0, err } iterateOverAllStores(t, tc, func(s *kvserver.Store) error { @@ -2297,7 +2298,7 @@ func TestPromoteNonVoterInAddVoter(t *testing.T) { // Retrieve the add voter events from the range log. var rangeID roachpb.RangeID - err = db.QueryRow("select range_id from [show ranges from table t] limit 1").Scan(&rangeID) + err = db.QueryRow("SELECT range_id FROM [SHOW RANGES FROM TABLE t] LIMIT 1").Scan(&rangeID) require.NoError(t, err) addVoterEvents, err := filterRangeLog(tc.Conns[0], rangeID, kvserverpb.RangeLogEventType_add_voter, kvserverpb.ReasonRangeUnderReplicated) require.NoError(t, err) diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 655933c1050d..b13f3791a456 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1051,14 +1051,11 @@ func (s *adminServer) tableDetailsHelper( row, cols, err = s.internalExecutor.QueryRowExWithCols( ctx, "admin-show-mvcc-garbage-info", nil, sessiondata.InternalExecutorOverride{User: userName}, - `WITH + fmt.Sprintf( + `WITH range_stats AS ( - SELECT - crdb_internal.range_stats(start_key) AS d - FROM - crdb_internal.ranges_no_leases - WHERE - table_id = $1::REGCLASS + SELECT crdb_internal.range_stats(raw_start_key) AS d + FROM [SHOW RANGES FROM TABLE %s WITH KEYS] ), aggregated AS ( SELECT @@ -1077,7 +1074,7 @@ func (s *adminServer) tableDetailsHelper( COALESCE(live, 0)::INT8 as live_bytes, COALESCE(live / NULLIF(total,0), 0)::FLOAT8 as live_percentage FROM aggregated`, - escQualTable, + escQualTable), ) if err != nil { return nil, err diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 2013f1dce86a..8a17367ea43b 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -637,9 +637,7 @@ func TestRangeCount(t *testing.T) { defer db.Close() runner := sqlutils.MakeSQLRunner(db) - s := sqlutils.MatrixToStr(runner.QueryStr(t, ` -select range_id, database_name, table_name, start_pretty, end_pretty from crdb_internal.ranges order by range_id asc`, - )) + s := sqlutils.MatrixToStr(runner.QueryStr(t, `SHOW CLUSTER RANGES`)) t.Logf("actual ranges:\n%s", s) } } diff --git a/pkg/sql/backfill_protected_timestamp_test.go b/pkg/sql/backfill_protected_timestamp_test.go index 7fc23f9cf247..7c1439972c7d 100644 --- a/pkg/sql/backfill_protected_timestamp_test.go +++ b/pkg/sql/backfill_protected_timestamp_test.go @@ -135,7 +135,7 @@ func TestValidationWithProtectedTS(t *testing.T) { <-indexValidationQueryWait getTableRangeIDs := func(t *testing.T) []int64 { t.Helper() - rows, err := dbConn2.QueryContext(ctx, "with r as (show ranges from table t) select range_id from r order by start_key") + rows, err := dbConn2.QueryContext(ctx, "WITH r AS (SHOW RANGES FROM TABLE t) SELECT range_id FROM r ORDER BY start_key") require.NoError(t, err, "failed to query ranges") var rangeIDs []int64 for rows.Next() { diff --git a/pkg/sql/catalog/colinfo/result_columns.go b/pkg/sql/catalog/colinfo/result_columns.go index 92ac197bbee0..8eaed05ba3e0 100644 --- a/pkg/sql/catalog/colinfo/result_columns.go +++ b/pkg/sql/catalog/colinfo/result_columns.go @@ -11,6 +11,8 @@ package colinfo import ( + "fmt" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -293,3 +295,72 @@ var TenantColumnsWithReplication = ResultColumns{ {Name: "retained_time", Typ: types.Timestamp}, {Name: "replication_start_time", Typ: types.Timestamp}, } + +// RangesNoLeases is the schema for crdb_internal.ranges_no_leases. +var RangesNoLeases = ResultColumns{ + {Name: "range_id", Typ: types.Int}, + {Name: "start_key", Typ: types.Bytes}, + {Name: "start_pretty", Typ: types.String}, + {Name: "end_key", Typ: types.Bytes}, + {Name: "end_pretty", Typ: types.String}, + {Name: "replicas", Typ: types.IntArray}, + {Name: "replica_localities", Typ: types.StringArray}, + {Name: "voting_replicas", Typ: types.IntArray}, + {Name: "non_voting_replicas", Typ: types.IntArray}, + {Name: "learner_replicas", Typ: types.IntArray}, + {Name: "split_enforced_until", Typ: types.Timestamp}, +} + +// Ranges is the schema for crdb_internal.ranges. +var Ranges = ResultColumns{ + // All the first columns from Ranges are really those from + // RangesNoLeases. So we would really like to include RangesNoLeases + // as a prefix here with a simple Go append(). + // + // Unfortunately, the result type of crdb_internal.ranges for the + // arrays is INT2VECTOR, which gives them zero-indexing. This is + // weird! crdb_internal.ranges_no_leases use regular INT[]. So the + // indexing behavior between crdb_internal.ranges and + // crdb_internal.ranges_no_leases is inconsistent. See: + // https://github.com/cockroachdb/cockroach/issues/93788 + // + // Until that bug is fixed, we need to list the columns explicitly, + // with a run-time check below (in the init function) that they are + // consistent. + {Name: "range_id", Typ: types.Int}, + {Name: "start_key", Typ: types.Bytes}, + {Name: "start_pretty", Typ: types.String}, + {Name: "end_key", Typ: types.Bytes}, + {Name: "end_pretty", Typ: types.String}, + {Name: "replicas", Typ: types.Int2Vector}, + {Name: "replica_localities", Typ: types.StringArray}, + {Name: "voting_replicas", Typ: types.Int2Vector}, + {Name: "non_voting_replicas", Typ: types.Int2Vector}, + {Name: "learner_replicas", Typ: types.Int2Vector}, + {Name: "split_enforced_until", Typ: types.Timestamp}, + + // The following columns are computed by RangesExtraRenders below. + {Name: "lease_holder", Typ: types.Int}, + {Name: "range_size", Typ: types.Int}, +} + +func init() { + // This check can be removed once Ranges can be constructed + // using append(RangesNoLeases, ...). + for i, c := range RangesNoLeases { + if c.Name != Ranges[i].Name || + c.Hidden != Ranges[i].Hidden { + panic(fmt.Sprintf("inconsistent definitions: %#v vs %#v", c, Ranges[i])) + } + } +} + +// RangesExtraRenders describes the extra projections in +// crdb_internal.ranges not included in crdb_internal.ranges_no_leases. +const RangesExtraRenders = ` + crdb_internal.lease_holder(start_key) AS lease_holder, + (crdb_internal.range_stats(start_key)->>'key_bytes')::INT + + (crdb_internal.range_stats(start_key)->>'val_bytes')::INT + + coalesce((crdb_internal.range_stats(start_key)->>'range_key_bytes')::INT, 0) + + coalesce((crdb_internal.range_stats(start_key)->>'range_val_bytes')::INT, 0) AS range_size +` diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 16cd651e83d1..05023bb43690 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -67,6 +67,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -142,6 +143,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable, catconstants.CrdbInternalTransactionContentionEvents: crdbInternalTransactionContentionEventsTable, catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable, + catconstants.CrdbInternalIndexSpansTableID: crdbInternalIndexSpansTable, catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics, catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable, catconstants.CrdbInternalJobsTableID: crdbInternalJobsTable, @@ -168,6 +170,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalStmtStatsTableID: crdbInternalStmtStatsView, catconstants.CrdbInternalTableColumnsTableID: crdbInternalTableColumnsTable, catconstants.CrdbInternalTableIndexesTableID: crdbInternalTableIndexesTable, + catconstants.CrdbInternalTableSpansTableID: crdbInternalTableSpansTable, catconstants.CrdbInternalTablesTableLastStatsID: crdbInternalTablesTableLastStats, catconstants.CrdbInternalTablesTableID: crdbInternalTablesTable, catconstants.CrdbInternalClusterTxnStatsTableID: crdbInternalClusterTxnStatsTable, @@ -3651,53 +3654,28 @@ CREATE TABLE crdb_internal.forward_dependencies ( }, } +func listColumnNames(cols colinfo.ResultColumns) string { + var buf strings.Builder + comma := "" + for _, c := range cols { + buf.WriteString(comma) + buf.WriteString(c.Name) + comma = "," + } + return buf.String() +} + // crdbInternalRangesView exposes system ranges. var crdbInternalRangesView = virtualSchemaView{ schema: ` -CREATE VIEW crdb_internal.ranges AS SELECT - range_id, - start_key, - start_pretty, - end_key, - end_pretty, - table_id, - database_name, - schema_name, - table_name, - index_name, - 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')::INT + - (crdb_internal.range_stats(start_key)->>'val_bytes')::INT + - coalesce((crdb_internal.range_stats(start_key)->>'range_key_bytes')::INT, 0) + - coalesce((crdb_internal.range_stats(start_key)->>'range_val_bytes')::INT, 0) AS range_size -FROM crdb_internal.ranges_no_leases -`, - resultColumns: colinfo.ResultColumns{ - {Name: "range_id", Typ: types.Int}, - {Name: "start_key", Typ: types.Bytes}, - {Name: "start_pretty", Typ: types.String}, - {Name: "end_key", Typ: types.Bytes}, - {Name: "end_pretty", Typ: types.String}, - {Name: "table_id", Typ: types.Int}, - {Name: "database_name", Typ: types.String}, - {Name: "schema_name", Typ: types.String}, - {Name: "table_name", Typ: types.String}, - {Name: "index_name", Typ: types.String}, - {Name: "replicas", Typ: types.Int2Vector}, - {Name: "replica_localities", Typ: types.StringArray}, - {Name: "voting_replicas", Typ: types.Int2Vector}, - {Name: "non_voting_replicas", Typ: types.Int2Vector}, - {Name: "learner_replicas", Typ: types.Int2Vector}, - {Name: "split_enforced_until", Typ: types.Timestamp}, - {Name: "lease_holder", Typ: types.Int}, - {Name: "range_size", Typ: types.Int}, - }, +CREATE VIEW crdb_internal.ranges AS SELECT ` + + // We'd like to use `*` here but it's not supported yet by our + // dialect. + listColumnNames(colinfo.RangesNoLeases) + `,` + + // Extra columns that are "expensive to compute". + colinfo.RangesExtraRenders + + `FROM crdb_internal.ranges_no_leases`, + resultColumns: colinfo.Ranges, } // descriptorsByType is a utility function that iterates through a slice of @@ -3804,11 +3782,6 @@ CREATE TABLE crdb_internal.ranges_no_leases ( start_pretty STRING NOT NULL, end_key BYTES NOT NULL, end_pretty STRING NOT NULL, - table_id INT NOT NULL, - database_name STRING NOT NULL, - schema_name STRING NOT NULL, - table_name STRING NOT NULL, - index_name STRING NOT NULL, replicas INT[] NOT NULL, replica_localities STRING[] NOT NULL, voting_replicas INT[] NOT NULL, @@ -3817,6 +3790,7 @@ CREATE TABLE crdb_internal.ranges_no_leases ( split_enforced_until TIMESTAMP ) `, + resultColumns: colinfo.RangesNoLeases, generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { hasAdmin, err := p.HasAdminRole(ctx) if err != nil { @@ -3836,9 +3810,13 @@ CREATE TABLE crdb_internal.ranges_no_leases ( return p.CheckPrivilege(ctx, desc, privilege.ZONECONFIG) == nil } - hasPermission, dbNames, tableNames, schemaNames, indexNames, schemaParents, parents := - descriptorsByType(descs, privCheckerFunc) - + hasPermission := false + for _, desc := range descs { + if privCheckerFunc(desc) { + hasPermission = true + break + } + } // if the user has no ZONECONFIG privilege on any table/schema/database if !hasPermission { return nil, nil, pgerror.Newf(pgcode.InsufficientPrivilege, "only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases") @@ -3914,11 +3892,6 @@ CREATE TABLE crdb_internal.ranges_no_leases ( } } - tableID, dbName, schemaName, tableName, indexName := lookupNamesByKey( - p, rangeDesc.StartKey.AsRawKey(), dbNames, tableNames, schemaNames, - indexNames, schemaParents, parents, - ) - splitEnforcedUntil := tree.DNull if !rangeDesc.StickyBit.IsEmpty() { splitEnforcedUntil = eval.TimestampToInexactDTimestamp(rangeDesc.StickyBit) @@ -3930,11 +3903,6 @@ CREATE TABLE crdb_internal.ranges_no_leases ( tree.NewDString(keys.PrettyPrint(nil /* valDirs */, rangeDesc.StartKey.AsRawKey())), tree.NewDBytes(tree.DBytes(rangeDesc.EndKey)), tree.NewDString(keys.PrettyPrint(nil /* valDirs */, rangeDesc.EndKey.AsRawKey())), - tree.NewDInt(tree.DInt(tableID)), - tree.NewDString(dbName), - tree.NewDString(schemaName), - tree.NewDString(tableName), - tree.NewDString(indexName), votersAndNonVotersArr, replicaLocalityArr, votersArr, @@ -6488,6 +6456,115 @@ CREATE TABLE crdb_internal.transaction_contention_events ( }, } +var crdbInternalIndexSpansTable = virtualSchemaTable{ + comment: `key spans per table index`, + schema: ` +CREATE TABLE crdb_internal.index_spans ( + descriptor_id INT NOT NULL, + index_id INT NOT NULL, + start_key BYTES NOT NULL, + end_key BYTES NOT NULL, + INDEX(descriptor_id) +);`, + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, constraint tree.Datum, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + descID := catid.DescID(tree.MustBeDInt(constraint)) + var table catalog.TableDescriptor + // We need to include offline tables, like in + // forEachTableDescAll() below. So we can't use p.LookupByID() + // which only considers online tables. + p.runWithOptions(resolveFlags{skipCache: true}, func() { + cflags := p.CommonLookupFlagsRequired() + cflags.IncludeOffline = true + cflags.IncludeDropped = true + flags := tree.ObjectLookupFlags{CommonLookupFlags: cflags} + table, err = p.Descriptors().GetImmutableTableByID(ctx, p.txn, descID, flags) + }) + if err != nil { + return false, err + } + return true, generateIndexSpans(ctx, p, table, addRow) + }, + }, + }, + populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + return forEachTableDescAll(ctx, p, db, hideVirtual, + func(_ catalog.DatabaseDescriptor, _ catalog.SchemaDescriptor, table catalog.TableDescriptor) error { + return generateIndexSpans(ctx, p, table, addRow) + }) + }, +} + +func generateIndexSpans( + ctx context.Context, p *planner, table catalog.TableDescriptor, addRow func(...tree.Datum) error, +) error { + tabID := table.GetID() + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) error { + indexID := idx.GetID() + start := roachpb.Key(rowenc.MakeIndexKeyPrefix(p.ExecCfg().Codec, tabID, indexID)) + end := start.PrefixEnd() + return addRow( + tree.NewDInt(tree.DInt(tabID)), + tree.NewDInt(tree.DInt(indexID)), + tree.NewDBytes(tree.DBytes(start)), + tree.NewDBytes(tree.DBytes(end)), + ) + }) +} + +var crdbInternalTableSpansTable = virtualSchemaTable{ + comment: `key spans per SQL object`, + schema: ` +CREATE TABLE crdb_internal.table_spans ( + descriptor_id INT NOT NULL, + start_key BYTES NOT NULL, + end_key BYTES NOT NULL, + INDEX(descriptor_id) +);`, + indexes: []virtualIndex{ + { + populate: func(ctx context.Context, constraint tree.Datum, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (matched bool, err error) { + descID := catid.DescID(tree.MustBeDInt(constraint)) + var table catalog.TableDescriptor + // We need to include offline tables, like in + // forEachTableDescAll() below. So we can't use p.LookupByID() + // which only considers online tables. + p.runWithOptions(resolveFlags{skipCache: true}, func() { + cflags := p.CommonLookupFlagsRequired() + cflags.IncludeOffline = true + cflags.IncludeDropped = true + flags := tree.ObjectLookupFlags{CommonLookupFlags: cflags} + table, err = p.Descriptors().GetImmutableTableByID(ctx, p.txn, descID, flags) + }) + if err != nil { + return false, err + } + return true, generateTableSpan(ctx, p, table, addRow) + }, + }, + }, + populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + return forEachTableDescAll(ctx, p, db, hideVirtual, + func(_ catalog.DatabaseDescriptor, _ catalog.SchemaDescriptor, table catalog.TableDescriptor) error { + return generateTableSpan(ctx, p, table, addRow) + }) + }, +} + +func generateTableSpan( + ctx context.Context, p *planner, table catalog.TableDescriptor, addRow func(...tree.Datum) error, +) error { + tabID := table.GetID() + start := p.ExecCfg().Codec.TablePrefix(uint32(tabID)) + end := start.PrefixEnd() + return addRow( + tree.NewDInt(tree.DInt(tabID)), + tree.NewDBytes(tree.DBytes(start)), + tree.NewDBytes(tree.DBytes(end)), + ) +} + // crdbInternalClusterLocksTable exposes the state of locks, as well as lock waiters, // in range lock tables across the cluster. var crdbInternalClusterLocksTable = virtualSchemaTable{ @@ -6498,7 +6575,7 @@ CREATE TABLE crdb_internal.cluster_locks ( range_id INT NOT NULL, table_id INT NOT NULL, database_name STRING NOT NULL, - schema_name STRING, + schema_name STRING NOT NULL, table_name STRING NOT NULL, index_name STRING, lock_key BYTES NOT NULL, diff --git a/pkg/sql/delegate/show_range_for_row.go b/pkg/sql/delegate/show_range_for_row.go index f9d9a2b2e01f..4f28c3903526 100644 --- a/pkg/sql/delegate/show_range_for_row.go +++ b/pkg/sql/delegate/show_range_for_row.go @@ -15,6 +15,9 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/errors" @@ -59,8 +62,16 @@ func (d *delegator) delegateShowRangeForRow(n *tree.ShowRangeForRow) (tree.State const query = ` SELECT - CASE WHEN r.start_key < x'%[5]s' THEN NULL ELSE crdb_internal.pretty_key(r.start_key, 2) END AS start_key, - CASE WHEN r.end_key >= x'%[6]s' THEN NULL ELSE crdb_internal.pretty_key(r.end_key, 2) END AS end_key, + CASE + WHEN r.start_key = crdb_internal.table_span(%[1]d)[1] THEN '…/' + WHEN r.start_key < crdb_internal.table_span(%[1]d)[1] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.start_key, 2) + END AS start_key, + CASE + WHEN r.end_key = crdb_internal.table_span(%[1]d)[2] THEN '…/' + WHEN r.end_key < crdb_internal.table_span(%[1]d)[2] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.end_key, 2) + END AS end_key, range_id, lease_holder, replica_localities[array_position(replicas, lease_holder)] as lease_holder_locality, @@ -85,3 +96,22 @@ WHERE (r.start_key <= crdb_internal.encode_key(%[1]d, %[2]d, %[3]s)) ), ) } + +func checkPrivilegesForShowRanges(d *delegator, table cat.Table) error { + // Basic requirement is SELECT priviliges + if err := d.catalog.CheckPrivilege(d.ctx, table, privilege.SELECT); err != nil { + return err + } + hasAdmin, err := d.catalog.HasAdminRole(d.ctx) + if err != nil { + return err + } + // User needs to either have admin access or have the correct ZONECONFIG privilege + if hasAdmin { + return nil + } + if err := d.catalog.CheckPrivilege(d.ctx, table, privilege.ZONECONFIG); err != nil { + return pgerror.Wrapf(err, pgcode.InsufficientPrivilege, "only users with the ZONECONFIG privilege or the admin role can use SHOW RANGES on %s", table.Name()) + } + return nil +} diff --git a/pkg/sql/delegate/show_ranges.go b/pkg/sql/delegate/show_ranges.go index 1cd9ed990f9e..bdcaee052cb0 100644 --- a/pkg/sql/delegate/show_ranges.go +++ b/pkg/sql/delegate/show_ranges.go @@ -11,125 +11,705 @@ package delegate import ( - "encoding/hex" "fmt" + "strings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/errors" ) -func checkPrivilegesForShowRanges(d *delegator, table cat.Table) error { - // Basic requirement is SELECT priviliges - if err := d.catalog.CheckPrivilege(d.ctx, table, privilege.SELECT); err != nil { - return err - } - hasAdmin, err := d.catalog.HasAdminRole(d.ctx) - if err != nil { - return err - } - // User needs to either have admin access or have the correct ZONECONFIG privilege - if hasAdmin { - return nil - } - if err := d.catalog.CheckPrivilege(d.ctx, table, privilege.ZONECONFIG); err != nil { - return pgerror.Wrapf(err, pgcode.InsufficientPrivilege, "only users with the ZONECONFIG privilege or the admin role can use SHOW RANGES on %s", table.Name()) - } - return nil -} +// The SHOW RANGES "statement" is really a family of related statements, +// selected via various syntax forms. +// +// In summary, we have: +// - SHOW CLUSTER RANGES which also includes ranges not belonging to +// any table. +// - SHOW RANGES [FROM DATABASE] which includes only ranges +// overlapping with any table in the target db. +// - SHOW RANGES FROM TABLE selects only ranges that overlap with the +// given table. +// - SHOW RANGES FROM INDEX selects only ranges that overlap with the +// given index. +// +// Then: +// - if WITH TABLES is specified, the rows are duplicated to detail +// each table included in a range. +// - if WITH INDEXES is specified, the rows are duplicated to detail +// each index included in a range. +// - otherwise, there is just 1 row per range. +// +// Then: +// - if WITH DETAILS is specified, extra _expensive_ information is +// included in the result, as of crdb_internal.ranges. +// (requires more roundtrips; makes the operation slower overall) +// - otherwise, only data from crdb_internal.ranges_no_leases is included. +// +// Then: +// - if WITH EXPLAIN is specified, the statement simply returns the +// text of the SQL query it would use if WITH EXPLAIN was not +// specified. This can be used for learning or troubleshooting. +// +// The overall semantics are best described using a diagram; see +// the interleaved diagram comments in the delegateShowRanges logic below. // delegateShowRanges implements the SHOW RANGES statement: // // SHOW RANGES FROM TABLE t // SHOW RANGES FROM INDEX t@idx -// SHOW RANGES FROM DATABASE db +// SHOW RANGES [ FROM DATABASE db | FROM CURRENT_CATALOG ] +// SHOW CLUSTER RANGES // // These statements show the ranges corresponding to the given table or index, // along with the list of replicas and the lease holder. +// +// See the explanatory comment at the top of this file for details. func (d *delegator) delegateShowRanges(n *tree.ShowRanges) (tree.Statement, error) { sqltelemetry.IncrementShowCounter(sqltelemetry.Ranges) - if n.DatabaseName != "" { - const dbQuery = ` - SELECT - table_name, - CASE - WHEN crdb_internal.pretty_key(r.start_key, 2) = '' THEN NULL - ELSE crdb_internal.pretty_key(r.start_key, 2) - END AS start_key, - CASE - WHEN crdb_internal.pretty_key(r.end_key, 2) = '' THEN NULL - ELSE crdb_internal.pretty_key(r.end_key, 2) - END AS end_key, - range_id, - range_size / 1000000 as range_size_mb, - lease_holder, - replica_localities[array_position(replicas, lease_holder)] as lease_holder_locality, - replicas, - replica_localities, - voting_replicas, - non_voting_replicas - FROM %[1]s.crdb_internal.ranges AS r - WHERE database_name=%[2]s - ORDER BY table_name, r.start_key - ` - // Note: n.DatabaseName.String() != string(n.DatabaseName) - return parse(fmt.Sprintf(dbQuery, n.DatabaseName.String(), lexbase.EscapeSQLString(string(n.DatabaseName)))) - } - - // Remember the original syntax: Resolve below modifies the TableOrIndex struct in-place. - noIndexSpecified := n.TableOrIndex.Index == "" - - idx, resName, err := cat.ResolveTableIndex( - d.ctx, d.catalog, cat.Flags{AvoidDescriptorCaches: true}, &n.TableOrIndex, - ) - if err != nil { - return nil, err - } - - if err := checkPrivilegesForShowRanges(d, idx.Table()); err != nil { - return nil, err - } - - if idx.Table().IsVirtualTable() { - return nil, errors.New("SHOW RANGES may not be called on a virtual table") - } - - var startKey, endKey string - if noIndexSpecified { - // All indexes. - tableID := idx.Table().ID() - prefix := d.evalCtx.Codec.TablePrefix(uint32(tableID)) - startKey = hex.EncodeToString(prefix) - endKey = hex.EncodeToString(prefix.PrefixEnd()) + + // Check option compatibility. + switch n.Options.Mode { + case tree.ExpandIndexes: + switch n.Source { + case tree.ShowRangesIndex: + return nil, pgerror.Newf(pgcode.Syntax, "cannot use WITH INDEXES with SHOW RANGES FROM INDEX") + } + case tree.ExpandTables: + switch n.Source { + case tree.ShowRangesIndex, tree.ShowRangesTable: + return nil, pgerror.Newf(pgcode.Syntax, "cannot use WITH TABLES with SHOW RANGES FROM [INDEX|TABLE]") + } + } + + // Resolve the target table/index if running FROM TABLE/INDEX. + var idx cat.Index + var resName tree.TableName + switch n.Source { + case tree.ShowRangesTable, tree.ShowRangesIndex: + var err error + idx, resName, err = cat.ResolveTableIndex( + d.ctx, d.catalog, cat.Flags{ + AvoidDescriptorCaches: true, + // SHOW RANGES is valid over non-public tables or indexes; + // they may still have data worthy of inspection. + IncludeNonActiveIndexes: true, + IncludeOfflineTables: true, + }, &n.TableOrIndex, + ) + if err != nil { + return nil, err + } + if idx.Table().IsVirtualTable() { + return nil, errors.New("SHOW RANGES may not be called on a virtual table") + } + } + + var buf strings.Builder + + // The first stage constructs the list of all ranges + // augmented with rows corresponding to tables/indexes. + + // SHOW RANGES + // | + // query from + // crdb_internal.ranges_no_lease + // | + // include range_id, range boundaries + // | + // (we are going to later include + // table/index boundaries as well) + // | + // include table_id + // and target object boundaries + // | + // .---------------^-----------------. + // | | + // syntax: | + // WITH INDEXES (no index details requested) + // or | + // FROM INDEX | + // | | + // also include index_id | + // | | + // `---------------v------------------' + // | + // ... + // + + buf.WriteString(` +WITH ranges AS ( +SELECT + r.range_id, + r.start_key, + r.end_key, + `) + + // Start with all the ranges_no_leases columns. This takes + // care of range_id, start/end_key etc. + for _, c := range sharedRangesCols { + fmt.Fprintf(&buf, " r.%s,", tree.NameString(c)) + } + // Also include the object IDs from the span table included below. + buf.WriteString("\n s.descriptor_id AS table_id,") + if n.Source == tree.ShowRangesIndex || n.Options.Mode == tree.ExpandIndexes { + buf.WriteString("\n s.index_id,") + } + // As well as the span boundaries, if any. + buf.WriteString(` + s.start_key AS span_start_key, + s.end_key AS span_end_key +FROM crdb_internal.ranges_no_leases r +`) + + // + // ... + // | + // .----------------^-----------------. + // | | + // syntax: syntax: + // SHOW CLUSTER RANGES SHOW RANGES [FROM ...] + // | | + // join using join using + // LEFT OUTER JOIN INNER JOIN + // (allows NULLs in join) | + // | | + // `---------------v-----------------' + // | + // ... + // + + if n.Source == tree.ShowRangesCluster { + // The join with the spans table will _augment_ the range rows; if + // there is no matching span for a given range, the span columns + // will remain NULL. + buf.WriteString(`LEFT OUTER JOIN `) } else { - // Just one index. - span := idx.Span() - startKey = hex.EncodeToString(span.Key) - endKey = hex.EncodeToString(span.EndKey) - } - - return parse(fmt.Sprintf(` -SELECT - CASE WHEN r.start_key <= x'%[1]s' THEN NULL ELSE crdb_internal.pretty_key(r.start_key, 2) END AS start_key, - CASE WHEN r.end_key >= x'%[2]s' THEN NULL ELSE crdb_internal.pretty_key(r.end_key, 2) END AS end_key, - index_name, - range_id, + // The join with the spans table will _filter_ the range rows; + // if there is no matching span for a given range, the range will + // be omitted from the output. + buf.WriteString(`INNER JOIN `) + } + + // + // ... + // | + // .---------------^------------------. + // | | + // syntax: | + // WITH INDEXES (no index details requested) + // or | + // FROM INDEX | + // | | + // join with join with + // crdb_internal.index_spans crdb_internal.table_spans + // | | + // result: at least one row result: at least one row + // per range-index intersection per range-table intersection + // | | + // `---------------v-----------------' + // | + // ... + // + + // Filter by the target database. We use the property that the + // vtable "DB.crdb_internal.{table,index}_spans" only return spans + // for objects in the database DB. + var dbName tree.Name + switch n.Source { + case tree.ShowRangesCluster: + // This selects "".crdb_internal.table_spans, which gathers spans + // over all databases. + dbName = "" + case tree.ShowRangesDatabase: + dbName = n.DatabaseName + case tree.ShowRangesCurrentDatabase: + dbName = tree.Name(d.evalCtx.SessionData().Database) + case tree.ShowRangesTable, tree.ShowRangesIndex: + dbName = resName.CatalogName + default: + return nil, errors.AssertionFailedf("programming error: missing db prefix %#v", n) + } + buf.WriteString(dbName.String()) + + // Which span vtable to use. + if n.Source == tree.ShowRangesIndex || n.Options.Mode == tree.ExpandIndexes { + // SHOW RANGES FROM INDEX, or SHOW RANGES WITH INDEXES: we'll want index-level details. + buf.WriteString(".crdb_internal.index_spans") + } else { + // Either WITH TABLES, or no details. Even when WITH TABLES is not + // speciifed we still join on table_spans: this forces filtering + // on the database in SHOW RANGES FROM DATABASE thanks to + // the inner filtering performed by the vtable off the database + // name prefix. + buf.WriteString(".crdb_internal.table_spans") + } + + // + // ... + // | + // .----------------^------------.---------------------------------. + // | | | + // syntax: syntax: syntax: + // SHOW RANGES SHOW RANGES SHOW RANGES + // [FROM DATABASE] FROM TABLE FROM INDEX + // or | | + // SHOW CLUSTER RANGES | | + // | | | + // join condition: join condition: join condition: + // span included in range span included in range span included in range + // | AND AND + // | span for selected table span for selected index + // | | | + // `----------------v------------^---------------------------------' + // | + // ... + // + + // The "main" join condition: match each range to just the spans + // that it intersects with. + buf.WriteString(` s + ON s.start_key < r.end_key +AND s.end_key > r.start_key`) + + // Do we also filter on a specific table or index? + switch n.Source { + case tree.ShowRangesTable, tree.ShowRangesIndex: + // Yes: add the additional filter on the span table. + fmt.Fprintf(&buf, "\n AND s.descriptor_id = %d", idx.Table().ID()) + if n.Source == tree.ShowRangesIndex { + fmt.Fprintf(&buf, " AND s.index_id = %d", idx.ID()) + } + } + buf.WriteString("\n)") // end of ranges CTE. + + // Now, enhance the result set so far with additional table/index + // details if requested; or remove range duplicates if no + // table/index details were requested. + + // + // ... + // | + // .----------------^------------.---------------------------------. + // | | | + // WITH TABLES WITH INDEXES (no table/index details requested) + // | | | + // LEFT OUTER JOIN using LEFT OUTER JOIN using | + // crdb_internal.tables crdb_internal.table_indexes DISTINCT + // | and | + // | crdb_internal.tables | + // | | | + // result: result: result: + // at least one row at least one row one row per range + // per range-table per range-index | + // intersection intersection | + // | | | + // add columns: add columns: | + // schema_name, table_name schema_name, table_name, index_name, | + // table_start_key, index_start_key, | + // table_end_key index_end_key | + // | | | + // also add database_name also add database_name | + // if SHOW CLUSTER RANGES if SHOW CLUSTER RANGES | + // | | | + // `----------------v------------^---------------------------------' + // | + // ... + // + + buf.WriteString(",\nnamed_ranges AS (") + + mode := n.Options.Mode + if n.Source == tree.ShowRangesIndex { + // The index view needs to see table_id/index_id propagate. We + // won't have duplicate ranges anyways: there's just 1 index + // selected. + mode = tree.ExpandIndexes + } + + // In SHOW RANGES FROM CLUSTER WITH TABLES/INDEXES, we also list the + // database name. + dbNameCol := "" + if n.Source == tree.ShowRangesCluster && mode != tree.UniqueRanges { + dbNameCol = ", database_name" + } + + switch mode { + case tree.UniqueRanges: + // De-duplicate the range rows, keep only the range metadata. + buf.WriteString("\nSELECT DISTINCT\n range_id, start_key, end_key") + for _, c := range sharedRangesCols { + fmt.Fprintf(&buf, ", %s", tree.NameString(c)) + } + buf.WriteString("\nFROM ranges") + + case tree.ExpandTables: + // Add table name details. + fmt.Fprintf(&buf, ` + SELECT r.* %[2]s, t.schema_name, t.name AS table_name + FROM ranges r + LEFT OUTER JOIN %[1]s.crdb_internal.tables t + ON r.table_id = t.table_id`, dbName.String(), dbNameCol) + + case tree.ExpandIndexes: + // Add table/index name details. + fmt.Fprintf(&buf, ` + SELECT r.* %[2]s, t.schema_name, t.name AS table_name, ti.index_name + FROM ranges r + LEFT OUTER JOIN %[1]s.crdb_internal.table_indexes ti + ON r.table_id = ti.descriptor_id + AND r.index_id = ti.index_id + LEFT OUTER JOIN %[1]s.crdb_internal.tables t + ON r.table_id = t.table_id`, dbName.String(), dbNameCol) + } + buf.WriteString("\n),\n") // end of named_ranges CTE. + buf.WriteString("intermediate AS (SELECT r.*") + // If details were requested, also include the extra + // columns from crdb_internal.ranges. + if n.Options.Details { + fmt.Fprintf(&buf, ",\n %s", colinfo.RangesExtraRenders) + } + buf.WriteString("\nFROM named_ranges r)\n") + + // Time to assemble the final projection. + + // + // ... + // | + // .----------------^------------.---------------------------------. + // | | | + // syntax: syntax: syntax: + // SHOW RANGES SHOW RANGES SHOW RANGES + // [FROM DATABASE] FROM TABLE FROM INDEX + // or | | + // SHOW CLUSTER RANGES | | + // | | | + // leave start/end keys strip the table prefix strip the index prefix + // as-is from start/end keys from start/end keys + // | | | + // `----------------v------------^---------------------------------' + // | + // ... + // + + buf.WriteString("SELECT") + + // First output is the range start/end keys. + switch n.Source { + case tree.ShowRangesDatabase, tree.ShowRangesCurrentDatabase, tree.ShowRangesCluster: + // For the SHOW CLUSTER RANGES and SHOW RANGES FROM DATABASE variants, + // the range keys are pretty-printed in full. + buf.WriteString(` + crdb_internal.pretty_key(r.start_key, -1) AS start_key, + crdb_internal.pretty_key(r.end_key, -1) AS end_key, +`) + + case tree.ShowRangesTable: + // With SHOW RANGES FROM TABLE, we elide the common table + // prefix when possible. + switch n.Options.Mode { + case tree.UniqueRanges, tree.ExpandTables: + // Note: we cannot use span_start/end_key reliably here, because + // when Mode == tree.UniqueRanges the DISTINCT clause has + // eliminated it. So we must recompute the span manually. + fmt.Fprintf(&buf, ` + CASE + WHEN r.start_key = crdb_internal.table_span(%[1]d)[1] THEN '…/' + WHEN r.start_key < crdb_internal.table_span(%[1]d)[1] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.start_key, 1) + END AS start_key, + CASE + WHEN r.end_key = crdb_internal.table_span(%[1]d)[2] THEN '…/' + WHEN r.end_key > crdb_internal.table_span(%[1]d)[2] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.end_key, 1) + END AS end_key, +`, idx.Table().ID()) + + case tree.ExpandIndexes: + buf.WriteString(` + CASE + WHEN r.start_key = span_start_key THEN '…/' + WHEN r.start_key = crdb_internal.table_span(table_id)[1] THEN '…/' + WHEN r.start_key < crdb_internal.table_span(table_id)[1] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.start_key, 1) + END AS start_key, + CASE + WHEN r.end_key = span_end_key THEN '…/…/' + WHEN r.end_key = crdb_internal.table_span(table_id)[2] THEN '…/' + WHEN r.end_key > crdb_internal.table_span(table_id)[2] THEN '' + ELSE '…'||crdb_internal.pretty_key(r.end_key, 1) + END AS end_key, +`) + default: + return nil, errors.AssertionFailedf("programming error: missing start/end key renderer: %#v", n) + } + + case tree.ShowRangesIndex: + buf.WriteString(` + CASE + WHEN r.start_key = span_start_key THEN '…/' + WHEN r.start_key = crdb_internal.table_span(table_id)[1] THEN '…/TableMin' + WHEN r.start_key < span_start_key THEN '' + ELSE '…'||crdb_internal.pretty_key(r.start_key, 2) + END AS start_key, + CASE + WHEN r.end_key = span_end_key THEN '…/' + WHEN r.end_key = crdb_internal.table_span(table_id)[2] THEN '…/' + WHEN r.end_key > span_end_key THEN '' + ELSE '…'||crdb_internal.pretty_key(r.end_key, 2) + END AS end_key, +`) + + default: + return nil, errors.AssertionFailedf("programming error: missing start/end key rendered: %#v", n) + } + + // + // ... + // | + // .---------------^-----------------. + // | | + // syntax: | + // WITH KEYS (no raw keys requested) + // | | + // add columns: | + // raw_start_key | + // raw_end_key | + // | | + // `---------------v-----------------' + // | + // also include range_id + // | + // ... + // + + // If the raw keys are requested, include them now. + if n.Options.Keys { + buf.WriteString(" r.start_key AS raw_start_key, r.end_key AS raw_end_key,\n") + } + + // Next output: range_id. We place the range_id close to the + // start of the row because alongside the start/end keys + // it is the most important piece of information. + // + // This is also where we switch from adding a comma at the end of + // every column, to adding it at the beginning of new columns. We + // need to switch because the column additions below are optional + // and SQL is not happy with a standalone trailing comma. + buf.WriteString(" range_id") + + // ... + // | + // .---------------^-----------------. + // | | + // syntax: | + // FROM TABLE | + // or | + // CLUSTER/FROM DATABASE | + // WITH INDEXES (no table details requested, + // | or not needed in FROM TABLES) + // propagate columns: | + // schema_name, table_name, | + // table_id, | + // table_start_key, | + // table_end_key | + // | | + // `---------------v-----------------' + // | + // ... + // + + // Spell out the additional name/ID columns stemming from WITH TABLES / WITH INDEXES. + // Note: we omit the table name in FROM TABLE because the table is already specified. + if n.Options.Mode == tree.ExpandTables || + (n.Options.Mode == tree.ExpandIndexes && n.Source != tree.ShowRangesTable) { + buf.WriteString(dbNameCol + `, schema_name, table_name, table_id`) + } + + // | + // .---------------^-----------------. + // | | + // syntax: | + // WITH INDEXES (no index details requested) + // | | + // propagate columns: | + // index_name, index_id | + // | | + // `---------------v-----------------' + // | + + if n.Options.Mode == tree.ExpandIndexes { + buf.WriteString(`, index_name, index_id`) + } + + // ... + // | + // .----------------^------------.---------------------------------. + // | | | + // syntax: syntax: (no table/index details requested) + // WITH INDEXES WITH TABLES | + // | | | + // propagate columns: propagate columns: | + // index_start_key, table_start_key | + // index_end_key table_end_key | + // | | | + // .-------^-----------------. | | + // | | | | + // syntax: syntax: | | + // FROM TABLE CLUSTER | | + // | or | | + // elide table FROM DATABASE | | + // prefix from keys | | | + // | | | | + // `-------v-----------------' | | + // | | | + // also include also include | + // raw index span boundaries raw table span boundaries | + // if if | + // WITH KEYS is specified WITH KEYS is specified | + // | | | + // `----------------v------------^---------------------------------' + // | + // ... + // + + // Spell out the spans of the included objets. + switch n.Options.Mode { + case tree.ExpandIndexes: + // In SHOW CLUSTER RANGES / SHOW RANGES FROM DATABASE, we + // spell out keys in full. In SHOW RANGES FROM TABLE, we trim the prefix. + shift := -1 + prefix := "" + if n.Source == tree.ShowRangesTable { + shift = 1 + prefix = "'…'||" + } + fmt.Fprintf(&buf, `, + %[2]scrdb_internal.pretty_key(span_start_key, %[1]d) AS index_start_key, + %[2]scrdb_internal.pretty_key(span_end_key, %[1]d) AS index_end_key`, shift, prefix) + + if n.Options.Keys { + buf.WriteString(`, + span_start_key AS raw_index_start_key, + span_end_key AS raw_index_end_key`) + } + + case tree.ExpandTables: + buf.WriteString(`, + crdb_internal.pretty_key(span_start_key, -1) AS table_start_key, + crdb_internal.pretty_key(span_end_key, -1) AS table_end_key`) + + if n.Options.Keys { + buf.WriteString(`, + span_start_key AS raw_table_start_key, + span_end_key AS raw_table_end_key`) + } + } + + // + // ... + // | + // include common projections + // from crdb_internal.ranges_no_leases + // | + // .---------------^-----------------. + // | | + // syntax: | + // WITH DETAILS (no details requested) + // | | + // add columns: | + // extra projections from | + // crdb_internal.ranges | + // and extra | + // computed columns | + // | | + // `---------------v-----------------' + // | + // ... + // + + // If WITH DETAILS was specified, include some convenience + // computed columns. + if n.Options.Details { + buf.WriteString(`, range_size / 1000000 as range_size_mb, lease_holder, - replica_localities[array_position(replicas, lease_holder)] as lease_holder_locality, - replicas, - replica_localities, - voting_replicas, - non_voting_replicas -FROM %[3]s.crdb_internal.ranges AS r -WHERE (r.start_key < x'%[2]s') - AND (r.end_key > x'%[1]s') ORDER BY index_name, r.start_key -`, - startKey, endKey, resName.CatalogName.String(), // note: CatalogName.String() != Catalog() - )) + replica_localities[array_position(replicas, lease_holder)] as lease_holder_locality`) + } + + // Then include all the remaining columns from crdb_internal.ranges_no_leases. + for _, c := range sharedRangesCols { + fmt.Fprintf(&buf, ",\n %s", tree.NameString(c)) + } + + // If details were requested, also include the extra + // columns from crdb_internal.ranges. + if n.Options.Details { + for i := len(colinfo.RangesNoLeases); i < len(colinfo.Ranges); i++ { + // NB: we've already output lease_holder above. + if colinfo.Ranges[i].Name == "lease_holder" { + continue + } + fmt.Fprintf(&buf, ",\n %s", tree.NameString(colinfo.Ranges[i].Name)) + } + } + + // Complete this CTE. and add an order if needed. + buf.WriteString("\nFROM intermediate r ORDER BY r.start_key") + switch n.Options.Mode { + case tree.ExpandIndexes, tree.ExpandTables: + buf.WriteString(", r.span_start_key") + } + + // + // ... + // | + // .----------------^-----------------. + // | | + // syntax: | + // WITH EXPLAIN (explain not requested) + // | | + // quote the generated simply + // SQL and render it execute the query + // as query result | + // for inspection | + // | | + // `---------------v-----------------' + // | + // ... + // + + // Finally, explain the query if requested. + fullQuery := buf.String() + if n.Options.Explain { + // Note: don't add prettify_statement here. It would make + // debugging invalid syntax more difficult. + fullQuery = fmt.Sprintf(`SELECT %s AS query`, lexbase.EscapeSQLString(fullQuery)) + } + return parse(fullQuery) } + +// In the shared logic above, we propagate the set of columns +// from crdb_internal.ranges_no_leases from one stage to another. +// Let's build it here. +var sharedRangesCols = func() (res []string) { + for _, c := range colinfo.RangesNoLeases { + if c.Hidden { + continue + } + // Skip over start/end key and the range ID, which we handle manually. + if c.Name == "range_id" || c.Name == "start_key" || c.Name == "end_key" { + continue + } + // Skip over the prettified key boundaries, which we will override. + if c.Name == "start_pretty" || c.Name == "end_pretty" { + continue + } + res = append(res, c.Name) + } + return res +}() diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 8a30c596187a..fb478755a178 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -418,14 +418,16 @@ func TestDistSQLDeadHosts(t *testing.T) { )) } r.CheckQueryResults(t, - "SELECT start_key, end_key, lease_holder, replicas FROM [SHOW RANGES FROM TABLE t]", + `SELECT IF(substring(start_key for 1)='…',start_key,NULL), + IF(substring(end_key for 1)='…',end_key,NULL), + lease_holder, replicas FROM [SHOW RANGES FROM TABLE t WITH DETAILS]`, [][]string{ - {"NULL", "/0", "1", "{1}"}, - {"/0", "/20", "1", "{1,2,3}"}, - {"/20", "/40", "2", "{2,3,4}"}, - {"/40", "/60", "3", "{1,3,4}"}, - {"/60", "/80", "4", "{1,2,4}"}, - {"/80", "NULL", "5", "{2,3,5}"}, + {"NULL", "…/1/0", "1", "{1}"}, + {"…/1/0", "…/1/20", "1", "{1,2,3}"}, + {"…/1/20", "…/1/40", "2", "{2,3,4}"}, + {"…/1/40", "…/1/60", "3", "{1,3,4}"}, + {"…/1/60", "…/1/80", "4", "{1,2,4}"}, + {"…/1/80", "NULL", "5", "{2,3,5}"}, }, ) diff --git a/pkg/sql/importer/import_into_test.go b/pkg/sql/importer/import_into_test.go index 6b52d9a5b382..30255210fe7d 100644 --- a/pkg/sql/importer/import_into_test.go +++ b/pkg/sql/importer/import_into_test.go @@ -120,11 +120,12 @@ func TestProtectedTimestampsDuringImportInto(t *testing.T) { time.Sleep(3 * time.Second) // Wait for the data to definitely be expired and GC to run. gcTable := func(skipShouldQueue bool) (traceStr string) { - rows := runner.Query(t, "SELECT start_key"+ - " FROM crdb_internal.ranges_no_leases"+ - " WHERE table_name = $1"+ - " AND database_name = current_database()"+ - " ORDER BY start_key ASC", "foo") + // Note: we cannot use SHOW RANGES FROM TABLE here because 'foo' + // is being imported and is not ready yet. + rows := runner.Query(t, ` +SELECT raw_start_key +FROM [SHOW RANGES FROM TABLE foo WITH KEYS] +ORDER BY raw_start_key ASC`) var traceBuf strings.Builder for rows.Next() { var startKey roachpb.Key diff --git a/pkg/sql/logictest/testdata/logic_test/cluster_locks b/pkg/sql/logictest/testdata/logic_test/cluster_locks index 4de80fb1c17d..65608c5661d1 100644 --- a/pkg/sql/logictest/testdata/logic_test/cluster_locks +++ b/pkg/sql/logictest/testdata/logic_test/cluster_locks @@ -18,12 +18,12 @@ key pretty split_enforced_until [242 137 18 114 0 1] /"r" 2262-04-11 23:47:16.854776 +0000 +0000 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /"d" {1} 1 -/"d" /"r" {1} 1 -/"r" NULL {1} 1 +start_key end_key replicas lease_holder + …/1/"d" {1} 1 +…/1/"d" …/1/"r" {1} 1 +…/1/"r" {1} 1 # Also create an additional user with VIEWACTIVITYREDACTED, with only permissions on t statement ok @@ -66,13 +66,13 @@ let $txn2 SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$testuser_session' let $r1 -SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key='/"d"' +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key LIKE '%/"d"' let $r2 -SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key='/"r"' +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key LIKE '%/"r"' let $r3 -SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key IS NULL +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key LIKE '%Max%' user testuser diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 13123f37b197..4eb86aaf9e31 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -46,6 +46,7 @@ crdb_internal gossip_liveness table admin NULL NULL crdb_internal gossip_network table admin NULL NULL crdb_internal gossip_nodes table admin NULL NULL crdb_internal index_columns table admin NULL NULL +crdb_internal index_spans table admin NULL NULL crdb_internal index_usage_statistics table admin NULL NULL crdb_internal invalid_objects table admin NULL NULL crdb_internal jobs table admin NULL NULL @@ -85,6 +86,7 @@ crdb_internal system_jobs table admin NULL NULL crdb_internal table_columns table admin NULL NULL crdb_internal table_indexes table admin NULL NULL crdb_internal table_row_statistics table admin NULL NULL +crdb_internal table_spans table admin NULL NULL crdb_internal tables table admin NULL NULL crdb_internal tenant_usage_details view admin NULL NULL crdb_internal transaction_contention_events table admin NULL NULL @@ -446,15 +448,15 @@ SELECT * FROM crdb_internal.node_inflight_trace_spans WHERE span_id < 0 ---- trace_id parent_span_id span_id goroutine_id finished start_time duration operation -query ITTTTITTTTTTTTTTTI colnames +query ITTTTITTTTTTI colnames SELECT * FROM crdb_internal.ranges WHERE range_id < 0 ---- -range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until lease_holder range_size +range_id start_key start_pretty end_key end_pretty replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until lease_holder range_size -query ITTTTITTTTTTTTTT colnames +query ITTTTTTTTTT colnames SELECT * FROM crdb_internal.ranges_no_leases WHERE range_id < 0 ---- -range_id start_key start_pretty end_key end_pretty table_id database_name schema_name table_name index_name replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until +range_id start_key start_pretty end_key end_pretty replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until statement ok CREATE SCHEMA schema; CREATE TABLE schema.bar (y INT PRIMARY KEY) @@ -596,38 +598,27 @@ start_pretty end_pretty split_enforced_until /Table/112/1/2 /Max 2262-04-11 23:47:16.854776 +0000 +0000 query TTT colnames -SELECT start_pretty, end_pretty, split_enforced_until FROM crdb_internal.ranges_no_leases WHERE split_enforced_until IS NOT NULL AND table_name = 'foo' +SELECT start_key, end_key, split_enforced_until FROM [SHOW RANGES FROM TABLE foo] WHERE split_enforced_until IS NOT NULL ---- -start_pretty end_pretty split_enforced_until -/Table/112/1/2 /Max 2262-04-11 23:47:16.854776 +0000 +0000 +start_key end_key split_enforced_until +…/1/2 2262-04-11 23:47:16.854776 +0000 +0000 statement ok ALTER TABLE foo UNSPLIT AT VALUES(2) query TT colnames -SELECT start_pretty, end_pretty FROM crdb_internal.ranges WHERE split_enforced_until IS NOT NULL AND table_name = 'foo' +SELECT start_key, end_key FROM [SHOW RANGES FROM TABLE foo] WHERE split_enforced_until IS NOT NULL ---- -start_pretty end_pretty - -query TT colnames -SELECT start_pretty, end_pretty FROM crdb_internal.ranges_no_leases WHERE split_enforced_until IS NOT NULL AND table_name = 'foo' ----- -start_pretty end_pretty +start_key end_key statement ok ALTER TABLE foo SPLIT AT VALUES(2) WITH EXPIRATION '2200-01-01 00:00:00.0' query TTT colnames -SELECT start_pretty, end_pretty, split_enforced_until FROM crdb_internal.ranges WHERE split_enforced_until IS NOT NULL AND table_name = 'foo' ----- -start_pretty end_pretty split_enforced_until -/Table/112/1/2 /Max 2200-01-01 00:00:00 +0000 +0000 - -query TTT colnames -SELECT start_pretty, end_pretty, split_enforced_until FROM crdb_internal.ranges_no_leases WHERE split_enforced_until IS NOT NULL AND table_name = 'foo' +SELECT start_key, end_key, split_enforced_until FROM [SHOW RANGES FROM TABLE foo] WHERE split_enforced_until IS NOT NULL ---- -start_pretty end_pretty split_enforced_until -/Table/112/1/2 /Max 2200-01-01 00:00:00 +0000 +0000 +start_key end_key split_enforced_until +…/1/2 2200-01-01 00:00:00 +0000 +0000 statement ok ALTER TABLE foo SPLIT AT VALUES (1), (2), (3) @@ -638,12 +629,12 @@ ALTER TABLE foo UNSPLIT ALL query TT colnames SELECT start_pretty, end_pretty FROM crdb_internal.ranges WHERE split_enforced_until IS NOT NULL ---- -start_pretty end_pretty +start_pretty end_pretty query TT colnames SELECT start_pretty, end_pretty FROM crdb_internal.ranges_no_leases WHERE split_enforced_until IS NOT NULL ---- -start_pretty end_pretty +start_pretty end_pretty # Make sure that the cluster id isn't unset. query B @@ -672,6 +663,9 @@ select * from crdb_internal.node_runtime_info query error pq: only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases select * from crdb_internal.ranges +query error pq: only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases +SHOW RANGES FROM TABLE foo + query error pq: only users with the admin role are allowed to read crdb_internal.gossip_nodes select * from crdb_internal.gossip_nodes @@ -883,8 +877,9 @@ node_id store_id # Do the compaction. query B colnames -SELECT crdb_internal.compact_engine_span(1, 1, start_key, end_key) -FROM crdb_internal.ranges_no_leases WHERE table_name = 'foo' LIMIT 1 +SELECT crdb_internal.compact_engine_span(1, 1, raw_start_key, raw_end_key) +FROM [SHOW RANGES FROM TABLE foo WITH KEYS] +LIMIT 1 ---- crdb_internal.compact_engine_span true diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index d24f32e2ee92..a774f25392aa 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -161,269 +161,271 @@ 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": "6"}} 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": "10", "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"}} -4294966998 {"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": 4294966998, "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": 4294967001, "version": "1"}} -4294966999 {"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": 4294966999, "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": 4294967001, "version": "1"}} -4294967000 {"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": 4294967000, "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": 4294967001, "version": "1"}} -4294967001 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967001, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} -4294967002 {"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": 4294967002, "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": 4294967131, "version": "1"}} -4294967003 {"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": 4294967003, "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": 4294967131, "version": "1"}} -4294967004 {"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": 4294967004, "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": 4294967131, "version": "1"}} -4294967005 {"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": 4294967005, "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": 4294967131, "version": "1"}} -4294967006 {"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": 4294967006, "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": 4294967131, "version": "1"}} -4294967007 {"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": 4294967007, "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": 4294967131, "version": "1"}} -4294967008 {"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": 4294967008, "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": 4294967131, "version": "1"}} -4294967009 {"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": 4294967009, "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": 4294967131, "version": "1"}} -4294967010 {"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": 4294967010, "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": 4294967131, "version": "1"}} -4294967011 {"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": 4294967011, "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": 4294967131, "version": "1"}} -4294967012 {"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": 4294967012, "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": 4294967131, "version": "1"}} -4294967013 {"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": 4294967013, "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": 4294967131, "version": "1"}} -4294967014 {"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": 4294967014, "name": "pg_timezone_names", "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": 4294967131, "version": "1"}} -4294967015 {"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": 4294967015, "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": 4294967131, "version": "1"}} -4294967016 {"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": 4294967016, "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": 4294967131, "version": "1"}} -4294967017 {"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": 4294967017, "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": 4294967131, "version": "1"}} -4294967018 {"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": 4294967018, "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": 4294967131, "version": "1"}} -4294967019 {"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": 4294967019, "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": 4294967131, "version": "1"}} -4294967020 {"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": 4294967020, "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": 4294967131, "version": "1"}} -4294967021 {"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": 4294967021, "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": 4294967131, "version": "1"}} -4294967022 {"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": 4294967022, "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": 4294967131, "version": "1"}} -4294967023 {"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": 4294967023, "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": 4294967131, "version": "1"}} -4294967024 {"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": 4294967024, "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": 4294967131, "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_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": 4294967131, "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_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": 4294967131, "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_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": 4294967131, "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": "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": 4294967028, "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": 4294967131, "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": "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": 4294967029, "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": 4294967131, "version": "1"}} -4294967030 {"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": 4294967030, "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": 4294967131, "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": "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": 4294967031, "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": 4294967131, "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": "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": 4294967032, "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": 4294967131, "version": "1"}} -4294967033 {"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": 4294967033, "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": 4294967131, "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}}], "formatVersion": 3, "id": 4294967034, "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": 4294967131, "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_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": 4294967131, "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}}], "formatVersion": 3, "id": 4294967036, "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": 4294967131, "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}}], "formatVersion": 3, "id": 4294967037, "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": 4294967131, "version": "1"}} -4294967038 {"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": 4294967038, "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": 4294967131, "version": "1"}} -4294967039 {"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": 4294967039, "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": 4294967131, "version": "1"}} -4294967040 {"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": 4294967040, "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": 4294967131, "version": "1"}} -4294967041 {"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": 4294967041, "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": 4294967131, "version": "1"}} -4294967042 {"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": 4294967042, "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": 4294967131, "version": "1"}} -4294967043 {"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": 4294967043, "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": 4294967131, "version": "1"}} -4294967044 {"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": 4294967044, "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": 4294967131, "version": "1"}} -4294967045 {"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": 4294967045, "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": 4294967131, "version": "1"}} -4294967046 {"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": 4294967046, "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": 4294967131, "version": "1"}} -4294967047 {"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": 4294967047, "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": 4294967131, "version": "1"}} -4294967048 {"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": 4294967048, "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": 4294967131, "version": "1"}} -4294967049 {"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": 4294967049, "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": 4294967131, "version": "1"}} -4294967050 {"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": 4294967050, "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": 4294967131, "version": "1"}} -4294967051 {"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": 4294967051, "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": 4294967131, "version": "1"}} -4294967052 {"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": 4294967052, "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": 4294967131, "version": "1"}} -4294967053 {"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": 4294967053, "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": 4294967131, "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": "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": 4294967054, "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": 4294967131, "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": "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": 4294967055, "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": 4294967131, "version": "1"}} -4294967056 {"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": 4294967056, "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": 4294967131, "version": "1"}} -4294967057 {"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": 4294967057, "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": 4294967131, "version": "1"}} -4294967058 {"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": 4294967058, "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": 4294967131, "version": "1"}} -4294967059 {"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": 4294967059, "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": 4294967131, "version": "1"}} -4294967060 {"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": 4294967060, "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": 4294967131, "version": "1"}} -4294967061 {"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": 4294967061, "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": 4294967131, "version": "1"}} -4294967062 {"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": 4294967062, "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": 4294967131, "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": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967063, "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": 4294967131, "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": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967064, "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": 4294967131, "version": "1"}} -4294967065 {"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": 4294967065, "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": 4294967131, "version": "1"}} -4294967066 {"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": 4294967066, "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": 4294967131, "version": "1"}} -4294967067 {"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": 4294967067, "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": 4294967131, "version": "1"}} -4294967068 {"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": 4294967068, "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": 4294967131, "version": "1"}} -4294967069 {"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": 4294967069, "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": 4294967131, "version": "1"}} -4294967070 {"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": 4294967070, "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": 4294967131, "version": "1"}} -4294967071 {"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": 4294967071, "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": 4294967131, "version": "1"}} -4294967072 {"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": 4294967072, "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": 4294967131, "version": "1"}} -4294967073 {"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": 4294967073, "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": 4294967131, "version": "1"}} -4294967074 {"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": 4294967074, "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": 4294967131, "version": "1"}} -4294967075 {"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": 4294967075, "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": 4294967131, "version": "1"}} -4294967076 {"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": 4294967076, "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": 4294967131, "version": "1"}} -4294967077 {"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": 4294967077, "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": 4294967131, "version": "1"}} -4294967078 {"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": 4294967078, "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": 4294967131, "version": "1"}} -4294967079 {"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": 4294967079, "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": 4294967131, "version": "1"}} -4294967080 {"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": 4294967080, "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": 4294967131, "version": "1"}} -4294967081 {"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": 4294967081, "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": 4294967131, "version": "1"}} -4294967082 {"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": 4294967082, "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": 4294967131, "version": "1"}} -4294967083 {"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": 4294967083, "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": 4294967131, "version": "1"}} -4294967084 {"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": 4294967084, "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": 4294967131, "version": "1"}} -4294967085 {"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": 4294967085, "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": 4294967131, "version": "1"}} -4294967086 {"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": 4294967086, "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": 4294967131, "version": "1"}} -4294967087 {"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": 4294967087, "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": 4294967131, "version": "1"}} -4294967088 {"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": 4294967088, "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": 4294967131, "version": "1"}} -4294967089 {"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": 4294967089, "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": 4294967131, "version": "1"}} -4294967090 {"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": 4294967090, "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": 4294967131, "version": "1"}} -4294967091 {"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": 4294967091, "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": 4294967131, "version": "1"}} -4294967092 {"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": 4294967092, "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": 4294967131, "version": "1"}} -4294967093 {"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": 4294967093, "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": 4294967131, "version": "1"}} -4294967094 {"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": 4294967094, "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": 4294967131, "version": "1"}} -4294967095 {"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": 4294967095, "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": 4294967131, "version": "1"}} -4294967096 {"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": 4294967096, "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": 4294967131, "version": "1"}} -4294967097 {"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": 4294967097, "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": 4294967131, "version": "1"}} -4294967098 {"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": 4294967098, "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": 4294967131, "version": "1"}} -4294967099 {"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": 4294967099, "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": 4294967131, "version": "1"}} -4294967100 {"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": 4294967100, "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": 4294967131, "version": "1"}} -4294967101 {"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": 4294967101, "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": 4294967131, "version": "1"}} -4294967102 {"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": 4294967102, "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": 4294967131, "version": "1"}} -4294967103 {"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": 4294967103, "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": 4294967131, "version": "1"}} -4294967104 {"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": 4294967104, "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": 4294967131, "version": "1"}} -4294967105 {"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": 4294967105, "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": 4294967131, "version": "1"}} -4294967106 {"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": 4294967106, "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": 4294967131, "version": "1"}} -4294967107 {"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": 4294967107, "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": 4294967131, "version": "1"}} -4294967108 {"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": 4294967108, "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": 4294967131, "version": "1"}} -4294967109 {"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": 4294967109, "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": 4294967131, "version": "1"}} -4294967110 {"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": 4294967110, "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": 4294967131, "version": "1"}} -4294967111 {"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": 4294967111, "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": 4294967131, "version": "1"}} -4294967112 {"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": 4294967112, "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": 4294967131, "version": "1"}} -4294967113 {"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": 4294967113, "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": 4294967131, "version": "1"}} -4294967114 {"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": 4294967114, "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": 4294967131, "version": "1"}} -4294967115 {"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": 4294967115, "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": 4294967131, "version": "1"}} -4294967116 {"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": 4294967116, "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": 4294967131, "version": "1"}} -4294967117 {"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": 4294967117, "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": 4294967131, "version": "1"}} -4294967118 {"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": 4294967118, "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": 4294967131, "version": "1"}} -4294967119 {"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": 4294967119, "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": 4294967131, "version": "1"}} -4294967120 {"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": 4294967120, "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": 4294967131, "version": "1"}} -4294967121 {"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": 4294967121, "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": 4294967131, "version": "1"}} -4294967122 {"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": 4294967122, "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": 4294967131, "version": "1"}} -4294967123 {"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": 4294967123, "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": 4294967131, "version": "1"}} -4294967124 {"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": 4294967124, "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": 4294967131, "version": "1"}} -4294967125 {"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": 4294967125, "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": 4294967131, "version": "1"}} -4294967126 {"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": 4294967126, "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": 4294967131, "version": "1"}} -4294967127 {"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": 4294967127, "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": 4294967131, "version": "1"}} -4294967128 {"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": 4294967128, "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": 4294967131, "version": "1"}} -4294967129 {"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": 4294967129, "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": 4294967131, "version": "1"}} -4294967130 {"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": 4294967130, "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": 4294967131, "version": "1"}} -4294967131 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967131, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} -4294967132 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "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": 4294967132, "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": 4294967218, "version": "1"}} -4294967133 {"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": 4294967133, "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": 4294967218, "version": "1"}} -4294967134 {"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": 4294967134, "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": 4294967218, "version": "1"}} -4294967135 {"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": 4294967135, "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": 4294967218, "version": "1"}} -4294967136 {"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": 4294967136, "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": 4294967218, "version": "1"}} -4294967137 {"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": 4294967137, "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": 4294967218, "version": "1"}} -4294967138 {"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": 4294967138, "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": 4294967218, "version": "1"}} -4294967139 {"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": 4294967139, "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": 4294967218, "version": "1"}} -4294967140 {"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": 4294967140, "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": 4294967218, "version": "1"}} -4294967141 {"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": 4294967141, "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": 4294967218, "version": "1"}} -4294967142 {"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": 4294967142, "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": 4294967218, "version": "1"}} -4294967143 {"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": 4294967143, "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": 4294967218, "version": "1"}} -4294967144 {"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": 4294967144, "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": 4294967218, "version": "1"}} -4294967145 {"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": 4294967145, "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": 4294967218, "version": "1"}} -4294967146 {"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": 4294967146, "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": 4294967218, "version": "1"}} -4294967147 {"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": 4294967147, "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": 4294967218, "version": "1"}} -4294967148 {"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": 4294967148, "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": 4294967218, "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": "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": 4294967149, "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": 4294967218, "version": "1"}} -4294967150 {"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": 4294967150, "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": 4294967218, "version": "1"}} -4294967151 {"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": 4294967151, "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": 4294967218, "version": "1"}} -4294967152 {"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": 4294967152, "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": 4294967218, "version": "1"}} -4294967153 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "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": 4294967153, "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": 4294967218, "version": "1"}} -4294967154 {"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": 4294967154, "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": 4294967218, "version": "1"}} -4294967155 {"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": 4294967155, "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": 4294967218, "version": "1"}} -4294967156 {"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": 4294967156, "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": 4294967218, "version": "1"}} -4294967157 {"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": 4294967157, "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": 4294967218, "version": "1"}} -4294967158 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967158, "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": 4294967218, "version": "1"}} -4294967159 {"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": 4294967159, "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": 4294967218, "version": "1"}} -4294967160 {"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": 4294967160, "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": 4294967218, "version": "1"}} -4294967161 {"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": 4294967161, "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": 4294967218, "version": "1"}} -4294967162 {"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": 4294967162, "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": 4294967218, "version": "1"}} -4294967163 {"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": 4294967163, "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": 4294967218, "version": "1"}} -4294967164 {"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": 4294967164, "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": 4294967218, "version": "1"}} -4294967165 {"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": 4294967165, "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": 4294967218, "version": "1"}} -4294967166 {"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": 4294967166, "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": 4294967218, "version": "1"}} -4294967167 {"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": 4294967167, "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": 4294967218, "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": "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": 4294967168, "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": 4294967218, "version": "1"}} -4294967169 {"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": 4294967169, "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": 4294967218, "version": "1"}} -4294967170 {"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": 4294967170, "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": 4294967218, "version": "1"}} -4294967171 {"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": 4294967171, "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": 4294967218, "version": "1"}} -4294967172 {"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": 4294967172, "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": 4294967218, "version": "1"}} -4294967173 {"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": 4294967173, "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": 4294967218, "version": "1"}} -4294967174 {"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": 4294967174, "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": 4294967218, "version": "1"}} -4294967175 {"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": 4294967175, "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": 4294967218, "version": "1"}} -4294967176 {"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": 4294967176, "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": 4294967218, "version": "1"}} -4294967177 {"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": 4294967177, "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": 4294967218, "version": "1"}} -4294967178 {"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": 4294967178, "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": 4294967218, "version": "1"}} -4294967179 {"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": 4294967179, "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": 4294967218, "version": "1"}} -4294967180 {"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": 4294967180, "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": 4294967218, "version": "1"}} -4294967181 {"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": 4294967181, "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": 4294967218, "version": "1"}} -4294967182 {"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": 4294967182, "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": 4294967218, "version": "1"}} -4294967183 {"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": 4294967183, "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": 4294967218, "version": "1"}} -4294967184 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967184, "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": 4294967218, "version": "1"}} -4294967185 {"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": 4294967185, "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": 4294967218, "version": "1"}} -4294967186 {"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": 4294967186, "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": 4294967218, "version": "1"}} -4294967187 {"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": 4294967187, "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": 4294967218, "version": "1"}} -4294967188 {"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": 4294967188, "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": 4294967218, "version": "1"}} -4294967189 {"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": 4294967189, "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": 4294967218, "version": "1"}} -4294967190 {"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": 4294967190, "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": 4294967218, "version": "1"}} -4294967191 {"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": 4294967191, "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": 4294967218, "version": "1"}} -4294967192 {"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": 4294967192, "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": 4294967218, "version": "1"}} -4294967193 {"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": 4294967193, "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": 4294967218, "version": "1"}} -4294967194 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967194, "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": 4294967218, "version": "1"}} -4294967195 {"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": 4294967195, "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": 4294967218, "version": "1"}} -4294967196 {"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": 4294967196, "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": 4294967218, "version": "1"}} -4294967197 {"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": 4294967197, "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": 4294967218, "version": "1"}} -4294967198 {"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": 4294967198, "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": 4294967218, "version": "1"}} -4294967199 {"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": 4294967199, "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": 4294967218, "version": "1"}} -4294967200 {"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": 4294967200, "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": 4294967218, "version": "1"}} -4294967201 {"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": 4294967201, "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": 4294967218, "version": "1"}} -4294967202 {"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": 4294967202, "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": 4294967218, "version": "1"}} -4294967203 {"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": 4294967203, "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": 4294967218, "version": "1"}} -4294967204 {"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": 4294967204, "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": 4294967218, "version": "1"}} -4294967205 {"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": 4294967205, "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": 4294967218, "version": "1"}} -4294967206 {"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": 4294967206, "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": 4294967218, "version": "1"}} -4294967207 {"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": 4294967207, "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": 4294967218, "version": "1"}} -4294967208 {"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": 4294967208, "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": 4294967218, "version": "1"}} -4294967209 {"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": 4294967209, "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": 4294967218, "version": "1"}} -4294967210 {"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": 4294967210, "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": 4294967218, "version": "1"}} -4294967211 {"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": 4294967211, "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": 4294967218, "version": "1"}} -4294967212 {"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": 4294967212, "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": 4294967218, "version": "1"}} -4294967213 {"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": 4294967213, "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": 4294967218, "version": "1"}} -4294967214 {"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": 4294967214, "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": 4294967218, "version": "1"}} -4294967215 {"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": 4294967215, "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": 4294967218, "version": "1"}} -4294967216 {"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": 4294967216, "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": 4294967218, "version": "1"}} -4294967217 {"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": 4294967217, "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": 4294967218, "version": "1"}} -4294967218 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967218, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} -4294967219 {"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": 4294967219, "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"}} -4294967220 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967220, "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"}} -4294967221 {"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": 4294967221, "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)"}} -4294967222 {"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": 4294967222, "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"}} -4294967223 {"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": 4294967223, "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"}} -4294967224 {"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": 4294967224, "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"}} -4294967225 {"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": 4294967225, "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"}} -4294967226 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967226, "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"}} -4294967227 {"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": 4294967227, "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"}} -4294967228 {"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": 4294967228, "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"}} -4294967229 {"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": 4294967229, "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"}} -4294967230 {"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": 4294967230, "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"}} -4294967231 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "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": 4294967231, "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"}} -4294967232 {"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}}], "formatVersion": 3, "id": 4294967232, "name": "node_transaction_statistics", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 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": "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": 4294967233, "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"}} -4294967234 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "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": 4294967234, "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"}} -4294967235 {"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": 4294967235, "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"}} -4294967236 {"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": 4294967236, "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"}} -4294967237 {"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": 4294967237, "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"}} -4294967238 {"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": 4294967238, "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"}} -4294967239 {"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": 4294967239, "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"}} -4294967240 {"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": 4294967240, "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"}} -4294967241 {"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": 4294967241, "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"}} -4294967242 {"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": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 12, "name": "replica_localities", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 14, "name": "non_voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 15, "name": "learner_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 16, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "lease_holder", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "range_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967242, "name": "ranges", "nextColumnId": 19, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT range_id, start_key, start_pretty, end_key, end_pretty, table_id, database_name, schema_name, table_name, index_name, 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"}} -4294967243 {"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": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 12, "name": "replica_localities", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 14, "name": "non_voting_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 15, "name": "learner_replicas", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 16, "name": "split_enforced_until", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967243, "name": "ranges_no_leases", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967244 {"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": 4294967244, "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"}} -4294967245 {"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": 4294967245, "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"}} -4294967246 {"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": "rows_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "rows_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": "parse_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "parse_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "plan_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "plan_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "run_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "run_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "overhead_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "overhead_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "bytes_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "bytes_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "rows_written_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "rows_written_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 42, "name": "full_scan", "type": {"oid": 16}}, {"id": 43, "name": "sample_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 44, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 46, "name": "txn_fingerprint_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967246, "name": "node_statement_statistics", "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": 4294967295, "version": "1"}} -4294967247 {"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": 4294967247, "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"}} -4294967248 {"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": 4294967248, "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"}} -4294967249 {"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": 4294967249, "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"}} -4294967250 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967250, "name": "node_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967251 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "contention_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 25, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 26, "name": "implicit_txn", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967251, "name": "node_execution_insights", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967252 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "status", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967252, "name": "node_distsql_flows", "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"}} -4294967253 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967253, "name": "node_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967254 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "deleted", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967254, "name": "leases", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967255 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 4, "name": "capacity", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "available", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "used", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "logical_bytes", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lease_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "writes_per_second", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "bytes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "writes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 14, "name": "properties", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967255, "name": "kv_store_status", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967256 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "go_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "time", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "revision", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "cgo_compiler", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "platform", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "distribution", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "dependencies", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "updated_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 18, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 19, "name": "args", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 20, "name": "env", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 21, "name": "activity", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967256, "name": "kv_node_status", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967257 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967257, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "system_jobs_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "name": "system_jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["id", "status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["status"], "name": "system_jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["id", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}], "name": "system_jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967258 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967258, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "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"}} -4294967259 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "goroutine_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "finished", "type": {"oid": 16}}, {"id": 6, "name": "start_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967259, "name": "node_inflight_trace_spans", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967260 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967260, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294966996 {"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": 4294966996, "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": 4294966999, "version": "1"}} +4294966997 {"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": 4294966997, "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": 4294966999, "version": "1"}} +4294966998 {"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": 4294966998, "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": 4294966999, "version": "1"}} +4294966999 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966999, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} +4294967000 {"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": 4294967000, "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": 4294967129, "version": "1"}} +4294967001 {"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": 4294967001, "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": 4294967129, "version": "1"}} +4294967002 {"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": 4294967002, "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": 4294967129, "version": "1"}} +4294967003 {"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": 4294967003, "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": 4294967129, "version": "1"}} +4294967004 {"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": 4294967004, "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": 4294967129, "version": "1"}} +4294967005 {"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": 4294967005, "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": 4294967129, "version": "1"}} +4294967006 {"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": 4294967006, "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": 4294967129, "version": "1"}} +4294967007 {"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": 4294967007, "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": 4294967129, "version": "1"}} +4294967008 {"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": 4294967008, "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": 4294967129, "version": "1"}} +4294967009 {"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": 4294967009, "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": 4294967129, "version": "1"}} +4294967010 {"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": 4294967010, "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": 4294967129, "version": "1"}} +4294967011 {"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": 4294967011, "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": 4294967129, "version": "1"}} +4294967012 {"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": 4294967012, "name": "pg_timezone_names", "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": 4294967129, "version": "1"}} +4294967013 {"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": 4294967013, "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": 4294967129, "version": "1"}} +4294967014 {"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": 4294967014, "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": 4294967129, "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": "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": 4294967015, "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": 4294967129, "version": "1"}} +4294967016 {"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": 4294967016, "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": 4294967129, "version": "1"}} +4294967017 {"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": 4294967017, "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": 4294967129, "version": "1"}} +4294967018 {"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": 4294967018, "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": 4294967129, "version": "1"}} +4294967019 {"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": 4294967019, "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": 4294967129, "version": "1"}} +4294967020 {"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": 4294967020, "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": 4294967129, "version": "1"}} +4294967021 {"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": 4294967021, "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": 4294967129, "version": "1"}} +4294967022 {"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": 4294967022, "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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "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": "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": 4294967029, "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": 4294967129, "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": "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": 4294967030, "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": 4294967129, "version": "1"}} +4294967031 {"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": 4294967031, "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": 4294967129, "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_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": 4294967129, "version": "1"}} +4294967033 {"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": 4294967033, "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": 4294967129, "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}}], "formatVersion": 3, "id": 4294967034, "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": 4294967129, "version": "1"}} +4294967035 {"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": 4294967035, "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": 4294967129, "version": "1"}} +4294967036 {"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": 4294967036, "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": 4294967129, "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_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": 4294967129, "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_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": 4294967129, "version": "1"}} +4294967039 {"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": 4294967039, "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": 4294967129, "version": "1"}} +4294967040 {"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": 4294967040, "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": 4294967129, "version": "1"}} +4294967041 {"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": 4294967041, "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": 4294967129, "version": "1"}} +4294967042 {"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": 4294967042, "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": 4294967129, "version": "1"}} +4294967043 {"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": 4294967043, "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": 4294967129, "version": "1"}} +4294967044 {"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": 4294967044, "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": 4294967129, "version": "1"}} +4294967045 {"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": 4294967045, "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": 4294967129, "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": "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": 4294967046, "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": 4294967129, "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": "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": 4294967047, "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": 4294967129, "version": "1"}} +4294967048 {"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": 4294967048, "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": 4294967129, "version": "1"}} +4294967049 {"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": 4294967049, "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": 4294967129, "version": "1"}} +4294967050 {"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": 4294967050, "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": 4294967129, "version": "1"}} +4294967051 {"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": 4294967051, "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": 4294967129, "version": "1"}} +4294967052 {"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": 4294967052, "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": 4294967129, "version": "1"}} +4294967053 {"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": 4294967053, "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": 4294967129, "version": "1"}} +4294967054 {"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": 4294967054, "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": 4294967129, "version": "1"}} +4294967055 {"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": 4294967055, "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": 4294967129, "version": "1"}} +4294967056 {"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": 4294967056, "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": 4294967129, "version": "1"}} +4294967057 {"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": 4294967057, "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": 4294967129, "version": "1"}} +4294967058 {"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": 4294967058, "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": 4294967129, "version": "1"}} +4294967059 {"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": 4294967059, "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": 4294967129, "version": "1"}} +4294967060 {"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": 4294967060, "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": 4294967129, "version": "1"}} +4294967061 {"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": 4294967061, "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": 4294967129, "version": "1"}} +4294967062 {"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": 4294967062, "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": 4294967129, "version": "1"}} +4294967063 {"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": 4294967063, "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": 4294967129, "version": "1"}} +4294967064 {"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": 4294967064, "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": 4294967129, "version": "1"}} +4294967065 {"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": 4294967065, "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": 4294967129, "version": "1"}} +4294967066 {"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": 4294967066, "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": 4294967129, "version": "1"}} +4294967067 {"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": 4294967067, "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": 4294967129, "version": "1"}} +4294967068 {"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": 4294967068, "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": 4294967129, "version": "1"}} +4294967069 {"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": 4294967069, "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": 4294967129, "version": "1"}} +4294967070 {"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": 4294967070, "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": 4294967129, "version": "1"}} +4294967071 {"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": 4294967071, "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": 4294967129, "version": "1"}} +4294967072 {"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": 4294967072, "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": 4294967129, "version": "1"}} +4294967073 {"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": 4294967073, "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": 4294967129, "version": "1"}} +4294967074 {"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": 4294967074, "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": 4294967129, "version": "1"}} +4294967075 {"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": 4294967075, "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": 4294967129, "version": "1"}} +4294967076 {"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": 4294967076, "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": 4294967129, "version": "1"}} +4294967077 {"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": 4294967077, "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": 4294967129, "version": "1"}} +4294967078 {"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": 4294967078, "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": 4294967129, "version": "1"}} +4294967079 {"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": 4294967079, "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": 4294967129, "version": "1"}} +4294967080 {"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": 4294967080, "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": 4294967129, "version": "1"}} +4294967081 {"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": 4294967081, "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": 4294967129, "version": "1"}} +4294967082 {"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": 4294967082, "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": 4294967129, "version": "1"}} +4294967083 {"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": 4294967083, "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": 4294967129, "version": "1"}} +4294967084 {"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": 4294967084, "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": 4294967129, "version": "1"}} +4294967085 {"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": 4294967085, "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": 4294967129, "version": "1"}} +4294967086 {"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": 4294967086, "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": 4294967129, "version": "1"}} +4294967087 {"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": 4294967087, "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": 4294967129, "version": "1"}} +4294967088 {"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": 4294967088, "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": 4294967129, "version": "1"}} +4294967089 {"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": 4294967089, "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": 4294967129, "version": "1"}} +4294967090 {"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": 4294967090, "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": 4294967129, "version": "1"}} +4294967091 {"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": 4294967091, "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": 4294967129, "version": "1"}} +4294967092 {"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": 4294967092, "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": 4294967129, "version": "1"}} +4294967093 {"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": 4294967093, "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": 4294967129, "version": "1"}} +4294967094 {"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": 4294967094, "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": 4294967129, "version": "1"}} +4294967095 {"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": 4294967095, "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": 4294967129, "version": "1"}} +4294967096 {"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": 4294967096, "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": 4294967129, "version": "1"}} +4294967097 {"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": 4294967097, "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": 4294967129, "version": "1"}} +4294967098 {"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": 4294967098, "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": 4294967129, "version": "1"}} +4294967099 {"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": 4294967099, "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": 4294967129, "version": "1"}} +4294967100 {"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": 4294967100, "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": 4294967129, "version": "1"}} +4294967101 {"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": 4294967101, "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": 4294967129, "version": "1"}} +4294967102 {"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": 4294967102, "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": 4294967129, "version": "1"}} +4294967103 {"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": 4294967103, "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": 4294967129, "version": "1"}} +4294967104 {"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": 4294967104, "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": 4294967129, "version": "1"}} +4294967105 {"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": 4294967105, "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": 4294967129, "version": "1"}} +4294967106 {"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": 4294967106, "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": 4294967129, "version": "1"}} +4294967107 {"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": 4294967107, "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": 4294967129, "version": "1"}} +4294967108 {"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": 4294967108, "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": 4294967129, "version": "1"}} +4294967109 {"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": 4294967109, "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": 4294967129, "version": "1"}} +4294967110 {"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": 4294967110, "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": 4294967129, "version": "1"}} +4294967111 {"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": 4294967111, "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": 4294967129, "version": "1"}} +4294967112 {"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": 4294967112, "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": 4294967129, "version": "1"}} +4294967113 {"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": 4294967113, "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": 4294967129, "version": "1"}} +4294967114 {"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": 4294967114, "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": 4294967129, "version": "1"}} +4294967115 {"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": 4294967115, "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": 4294967129, "version": "1"}} +4294967116 {"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": 4294967116, "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": 4294967129, "version": "1"}} +4294967117 {"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": 4294967117, "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": 4294967129, "version": "1"}} +4294967118 {"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": 4294967118, "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": 4294967129, "version": "1"}} +4294967119 {"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": 4294967119, "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": 4294967129, "version": "1"}} +4294967120 {"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": 4294967120, "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": 4294967129, "version": "1"}} +4294967121 {"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": 4294967121, "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": 4294967129, "version": "1"}} +4294967122 {"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": 4294967122, "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": 4294967129, "version": "1"}} +4294967123 {"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": 4294967123, "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": 4294967129, "version": "1"}} +4294967124 {"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": 4294967124, "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": 4294967129, "version": "1"}} +4294967125 {"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": 4294967125, "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": 4294967129, "version": "1"}} +4294967126 {"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": 4294967126, "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": 4294967129, "version": "1"}} +4294967127 {"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": 4294967127, "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": 4294967129, "version": "1"}} +4294967128 {"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": 4294967128, "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": 4294967129, "version": "1"}} +4294967129 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967129, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} +4294967130 {"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": 4294967130, "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": 4294967216, "version": "1"}} +4294967131 {"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": 4294967131, "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": 4294967216, "version": "1"}} +4294967132 {"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": 4294967132, "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": 4294967216, "version": "1"}} +4294967133 {"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": 4294967133, "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": 4294967216, "version": "1"}} +4294967134 {"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": 4294967134, "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": 4294967216, "version": "1"}} +4294967135 {"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": 4294967135, "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": 4294967216, "version": "1"}} +4294967136 {"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": 4294967136, "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": 4294967216, "version": "1"}} +4294967137 {"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": 4294967137, "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": 4294967216, "version": "1"}} +4294967138 {"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": 4294967138, "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": 4294967216, "version": "1"}} +4294967139 {"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": 4294967139, "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": 4294967216, "version": "1"}} +4294967140 {"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": 4294967140, "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": 4294967216, "version": "1"}} +4294967141 {"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": 4294967141, "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": 4294967216, "version": "1"}} +4294967142 {"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": 4294967142, "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": 4294967216, "version": "1"}} +4294967143 {"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": 4294967143, "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": 4294967216, "version": "1"}} +4294967144 {"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": 4294967144, "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": 4294967216, "version": "1"}} +4294967145 {"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": 4294967145, "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": 4294967216, "version": "1"}} +4294967146 {"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": 4294967146, "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": 4294967216, "version": "1"}} +4294967147 {"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": 4294967147, "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": 4294967216, "version": "1"}} +4294967148 {"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": 4294967148, "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": 4294967216, "version": "1"}} +4294967149 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967149, "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": 4294967216, "version": "1"}} +4294967150 {"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": 4294967150, "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": 4294967216, "version": "1"}} +4294967151 {"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": 4294967151, "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": 4294967216, "version": "1"}} +4294967152 {"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": 4294967152, "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": 4294967216, "version": "1"}} +4294967153 {"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": 4294967153, "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": 4294967216, "version": "1"}} +4294967154 {"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": 4294967154, "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": 4294967216, "version": "1"}} +4294967155 {"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": 4294967155, "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": 4294967216, "version": "1"}} +4294967156 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967156, "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": 4294967216, "version": "1"}} +4294967157 {"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": 4294967157, "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": 4294967216, "version": "1"}} +4294967158 {"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": 4294967158, "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": 4294967216, "version": "1"}} +4294967159 {"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": 4294967159, "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": 4294967216, "version": "1"}} +4294967160 {"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": 4294967160, "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": 4294967216, "version": "1"}} +4294967161 {"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": 4294967161, "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": 4294967216, "version": "1"}} +4294967162 {"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": 4294967162, "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": 4294967216, "version": "1"}} +4294967163 {"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": 4294967163, "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": 4294967216, "version": "1"}} +4294967164 {"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": 4294967164, "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": 4294967216, "version": "1"}} +4294967165 {"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": 4294967165, "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": 4294967216, "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": "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": 4294967216, "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": "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": 4294967167, "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": 4294967216, "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": "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": 4294967168, "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": 4294967216, "version": "1"}} +4294967169 {"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": 4294967169, "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": 4294967216, "version": "1"}} +4294967170 {"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": 4294967170, "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": 4294967216, "version": "1"}} +4294967171 {"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": 4294967171, "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": 4294967216, "version": "1"}} +4294967172 {"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": 4294967172, "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": 4294967216, "version": "1"}} +4294967173 {"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": 4294967173, "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": 4294967216, "version": "1"}} +4294967174 {"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": 4294967174, "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": 4294967216, "version": "1"}} +4294967175 {"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": 4294967175, "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": 4294967216, "version": "1"}} +4294967176 {"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": 4294967176, "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": 4294967216, "version": "1"}} +4294967177 {"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": 4294967177, "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": 4294967216, "version": "1"}} +4294967178 {"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": 4294967178, "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": 4294967216, "version": "1"}} +4294967179 {"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": 4294967179, "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": 4294967216, "version": "1"}} +4294967180 {"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": 4294967180, "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": 4294967216, "version": "1"}} +4294967181 {"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": 4294967181, "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": 4294967216, "version": "1"}} +4294967182 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967182, "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": 4294967216, "version": "1"}} +4294967183 {"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": 4294967183, "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": 4294967216, "version": "1"}} +4294967184 {"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": 4294967184, "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": 4294967216, "version": "1"}} +4294967185 {"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": 4294967185, "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": 4294967216, "version": "1"}} +4294967186 {"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": 4294967186, "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": 4294967216, "version": "1"}} +4294967187 {"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": 4294967187, "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": 4294967216, "version": "1"}} +4294967188 {"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": 4294967188, "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": 4294967216, "version": "1"}} +4294967189 {"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": 4294967189, "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": 4294967216, "version": "1"}} +4294967190 {"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": 4294967190, "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": 4294967216, "version": "1"}} +4294967191 {"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": 4294967191, "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": 4294967216, "version": "1"}} +4294967192 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967192, "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": 4294967216, "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": "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": 4294967193, "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": 4294967216, "version": "1"}} +4294967194 {"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": 4294967194, "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": 4294967216, "version": "1"}} +4294967195 {"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": 4294967195, "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": 4294967216, "version": "1"}} +4294967196 {"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": 4294967196, "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": 4294967216, "version": "1"}} +4294967197 {"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": 4294967197, "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": 4294967216, "version": "1"}} +4294967198 {"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": 4294967198, "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": 4294967216, "version": "1"}} +4294967199 {"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": 4294967199, "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": 4294967216, "version": "1"}} +4294967200 {"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": 4294967200, "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": 4294967216, "version": "1"}} +4294967201 {"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": 4294967201, "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": 4294967216, "version": "1"}} +4294967202 {"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": 4294967202, "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": 4294967216, "version": "1"}} +4294967203 {"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": 4294967203, "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": 4294967216, "version": "1"}} +4294967204 {"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": 4294967204, "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": 4294967216, "version": "1"}} +4294967205 {"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": 4294967205, "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": 4294967216, "version": "1"}} +4294967206 {"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": 4294967206, "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": 4294967216, "version": "1"}} +4294967207 {"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": 4294967207, "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": 4294967216, "version": "1"}} +4294967208 {"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": 4294967208, "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": 4294967216, "version": "1"}} +4294967209 {"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": 4294967209, "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": 4294967216, "version": "1"}} +4294967210 {"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": 4294967210, "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": 4294967216, "version": "1"}} +4294967211 {"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": 4294967211, "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": 4294967216, "version": "1"}} +4294967212 {"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": 4294967212, "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": 4294967216, "version": "1"}} +4294967213 {"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": 4294967213, "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": 4294967216, "version": "1"}} +4294967214 {"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": 4294967214, "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": 4294967216, "version": "1"}} +4294967215 {"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": 4294967215, "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": 4294967216, "version": "1"}} +4294967216 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967216, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": 512, "userProto": "public"}], "version": 2}, "version": "1"}} +4294967217 {"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": 4294967217, "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"}} +4294967218 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967218, "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"}} +4294967219 {"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": 4294967219, "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)"}} +4294967220 {"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": 4294967220, "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"}} +4294967221 {"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": 4294967221, "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"}} +4294967222 {"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": 4294967222, "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"}} +4294967223 {"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": 4294967223, "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"}} +4294967224 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967224, "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"}} +4294967225 {"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": 4294967225, "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"}} +4294967226 {"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": 4294967226, "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"}} +4294967227 {"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": 4294967227, "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"}} +4294967228 {"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": 4294967228, "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"}} +4294967229 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "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": 4294967229, "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"}} +4294967230 {"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}}], "formatVersion": 3, "id": 4294967230, "name": "node_transaction_statistics", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 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": "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": 4294967231, "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"}} +4294967232 {"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": 4294967232, "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"}} +4294967233 {"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": 4294967233, "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"}} +4294967234 {"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": 4294967234, "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"}} +4294967235 {"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": 4294967235, "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"}} +4294967236 {"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": 4294967236, "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"}} +4294967237 {"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": 4294967237, "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"}} +4294967238 {"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": 4294967238, "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"}} +4294967239 {"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": 4294967239, "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"}} +4294967240 {"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": 4294967240, "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"}} +4294967241 {"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": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"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": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 9, "name": "non_voting_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 10, "name": "learner_replicas", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"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": 4294967241, "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"}} +4294967242 {"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": 4294967242, "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"}} +4294967243 {"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": 4294967243, "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"}} +4294967244 {"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": 4294967244, "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"}} +4294967245 {"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": "rows_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "rows_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": "parse_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "parse_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "plan_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "plan_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "run_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "run_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "overhead_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "overhead_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "bytes_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "bytes_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "rows_written_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "rows_written_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "implicit_txn", "type": {"oid": 16}}, {"id": 42, "name": "full_scan", "type": {"oid": 16}}, {"id": 43, "name": "sample_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 44, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 46, "name": "txn_fingerprint_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967245, "name": "node_statement_statistics", "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": 4294967295, "version": "1"}} +4294967246 {"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": 4294967246, "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"}} +4294967247 {"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": 4294967247, "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"}} +4294967248 {"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": 4294967248, "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"}} +4294967249 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967249, "name": "node_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967250 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "contention_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 25, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 26, "name": "implicit_txn", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967250, "name": "node_execution_insights", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967251 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "status", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967251, "name": "node_distsql_flows", "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"}} +4294967252 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967252, "name": "node_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967253 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "deleted", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967253, "name": "leases", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967254 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 4, "name": "capacity", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "available", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "used", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "logical_bytes", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lease_count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "writes_per_second", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "bytes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 12, "name": "writes_per_replica", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 13, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 14, "name": "properties", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967254, "name": "kv_store_status", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967255 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "network", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "address", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attrs", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "locality", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "server_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "go_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "time", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "revision", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "cgo_compiler", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "platform", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "distribution", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "dependencies", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "started_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 17, "name": "updated_at", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 18, "name": "metrics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 19, "name": "args", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 20, "name": "env", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 21, "name": "activity", "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967255, "name": "kv_node_status", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967256 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "payload", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "progress", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "created_by_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "created_by_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "claim_session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "claim_instance_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967256, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "system_jobs_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [12], "keyColumnNames": ["job_type"], "name": "system_jobs_job_type_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], "storeColumnNames": ["id", "status", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["status"], "name": "system_jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["id", "created", "payload", "progress", "created_by_type", "created_by_id", "claim_session_id", "claim_instance_id", "num_runs", "last_run", "job_type"], "version": 3}], "name": "system_jobs", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 5, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967257 {"table": {"columns": [{"id": 1, "name": "job_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "job_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "descriptor_ids", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 7, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "running_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "created", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 10, "name": "started", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "finished", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "modified", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "fraction_completed", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "high_water_timestamp", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 15, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "coordinator_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "trace_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "last_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 19, "name": "next_run", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 20, "name": "num_runs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "execution_errors", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "execution_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967257, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [7], "keyColumnNames": ["status"], "name": "jobs_status_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22], "storeColumnNames": ["job_id", "job_type", "description", "statement", "user_name", "descriptor_ids", "running_status", "created", "started", "finished", "modified", "fraction_completed", "high_water_timestamp", "error", "coordinator_id", "trace_id", "last_run", "next_run", "num_runs", "execution_errors", "execution_events"], "version": 3}], "name": "jobs", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967258 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "span_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "goroutine_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "finished", "type": {"oid": 16}}, {"id": 6, "name": "start_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967258, "name": "node_inflight_trace_spans", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967259 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "total_reads", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_read", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967259, "name": "index_usage_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967260 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967260, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "index_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["index_id", "start_key", "end_key"], "version": 3}], "name": "index_spans", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967261 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "column_direction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "implicit", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967261, "name": "index_columns", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967262 {"table": {"columns": [{"id": 1, "name": "collection_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "blocking_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "blocking_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "waiting_txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "waiting_txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "contention_duration", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "contending_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967262, "name": "transaction_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967263 {"table": {"columns": [{"id": 1, "name": "source_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "target_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967263, "name": "gossip_network", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -444,7 +446,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 4294967278 {"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": 4294967278, "name": "cluster_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"}} 4294967279 {"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": 4294967279, "name": "cluster_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"}} 4294967280 {"table": {"columns": [{"id": 1, "name": "query_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "session_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "user_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "start", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "distributed", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "full_scan", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "plan_gist", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "database", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967280, "name": "cluster_queries", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967281 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "lock_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "lock_key_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 10, "name": "ts", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "lock_strength", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "durability", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "contended", "type": {"oid": 16}}, {"id": 15, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967281, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["table_id"], "name": "cluster_locks_table_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["database_name"], "name": "cluster_locks_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [5], "keyColumnNames": ["table_name"], "name": "cluster_locks_table_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 5, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [14], "keyColumnNames": ["contended"], "name": "cluster_locks_contended_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "duration"], "version": 3}], "name": "cluster_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 6, "nextMutationId": 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"}} +4294967281 {"table": {"columns": [{"id": 1, "name": "range_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "lock_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 8, "name": "lock_key_pretty", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "txn_id", "nullable": true, "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 10, "name": "ts", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "lock_strength", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "durability", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "contended", "type": {"oid": 16}}, {"id": 15, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967281, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["table_id"], "name": "cluster_locks_table_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["database_name"], "name": "cluster_locks_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 4, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [5], "keyColumnNames": ["table_name"], "name": "cluster_locks_table_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "contended", "duration"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 5, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [14], "keyColumnNames": ["contended"], "name": "cluster_locks_contended_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15], "storeColumnNames": ["range_id", "table_id", "database_name", "schema_name", "table_name", "index_name", "lock_key", "lock_key_pretty", "txn_id", "ts", "lock_strength", "durability", "granted", "duration"], "version": 3}], "name": "cluster_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 6, "nextMutationId": 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"}} 4294967282 {"table": {"columns": [{"id": 1, "name": "session_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "txn_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stmt_id", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "stmt_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "problem", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "causes", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "query", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "status", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "start_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 11, "name": "end_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "full_scan", "type": {"oid": 16}}, {"id": 13, "name": "user_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "plan_gist", "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "rows_read", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "rows_written", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "priority", "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "retries", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "last_retry_reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "exec_node_ids", "type": {"arrayContents": {"family": "IntFamily", "oid": 20, "width": 64}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1016, "width": 64}}, {"id": 23, "name": "contention", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 24, "name": "contention_events", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 25, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 26, "name": "implicit_txn", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967282, "name": "cluster_execution_insights", "nextColumnId": 27, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 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"}} 4294967283 {"table": {"columns": [{"id": 1, "name": "flow_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "stmt", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "since", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 5, "name": "status", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967283, "name": "cluster_distsql_flows", "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"}} 4294967284 {"table": {"columns": [{"id": 1, "name": "table_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "index_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "num_contention_events", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "cumulative_contention_time", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "txn_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 7, "name": "count", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967284, "name": "cluster_contention_events", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": 32, "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -545,261 +547,263 @@ FROM "".crdb_internal.kv_catalog_comments ---- DatabaseCommentType 104 0 "this is the test database" TableCommentType 111 0 "this is a table" -TableCommentType 4294966998 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." -TableCommentType 4294966999 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." -TableCommentType 4294967000 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." -TableCommentType 4294967002 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" -TableCommentType 4294967003 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" -TableCommentType 4294967004 0 "pg_user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967005 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" -TableCommentType 4294967006 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" -TableCommentType 4294967007 0 "pg_ts_template was created for compatibility and is currently unimplemented" -TableCommentType 4294967008 0 "pg_ts_parser was created for compatibility and is currently unimplemented" -TableCommentType 4294967009 0 "pg_ts_dict was created for compatibility and is currently unimplemented" -TableCommentType 4294967010 0 "pg_ts_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967011 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" -TableCommentType 4294967012 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" -TableCommentType 4294967013 0 "pg_transform was created for compatibility and is currently unimplemented" -TableCommentType 4294967014 0 "pg_timezone_names was created for compatibility and is currently unimplemented" -TableCommentType 4294967015 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" -TableCommentType 4294967016 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" -TableCommentType 4294967017 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" -TableCommentType 4294967018 0 "pg_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967019 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967020 0 "pg_stats was created for compatibility and is currently unimplemented" -TableCommentType 4294967021 0 "pg_stats_ext was created for compatibility and is currently unimplemented" -TableCommentType 4294967022 0 "pg_statistic was created for compatibility and is currently unimplemented" -TableCommentType 4294967023 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" -TableCommentType 4294967024 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" -TableCommentType 4294967025 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967026 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967027 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967028 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967029 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967030 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967031 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967032 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967033 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967034 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967035 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967036 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967037 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967038 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967039 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967040 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967041 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967042 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967043 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967044 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967045 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" -TableCommentType 4294967046 0 "pg_stat_slru was created for compatibility and is currently unimplemented" -TableCommentType 4294967047 0 "pg_stat_replication was created for compatibility and is currently unimplemented" -TableCommentType 4294967048 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" -TableCommentType 4294967049 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" -TableCommentType 4294967050 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" -TableCommentType 4294967051 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" -TableCommentType 4294967052 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" -TableCommentType 4294967053 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" -TableCommentType 4294967054 0 "pg_stat_database was created for compatibility and is currently unimplemented" -TableCommentType 4294967055 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" -TableCommentType 4294967056 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" -TableCommentType 4294967057 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967058 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967059 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967060 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 4294967061 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" -TableCommentType 4294967062 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" -TableCommentType 4294967063 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" -TableCommentType 4294967064 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" -TableCommentType 4294967065 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 4294967066 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" -TableCommentType 4294967067 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" -TableCommentType 4294967068 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" -TableCommentType 4294967069 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" -TableCommentType 4294967070 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" -TableCommentType 4294967071 0 "pg_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967072 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" -TableCommentType 4294967073 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 4294967074 0 "pg_replication_slots was created for compatibility and is currently unimplemented" -TableCommentType 4294967075 0 "pg_replication_origin was created for compatibility and is currently unimplemented" -TableCommentType 4294967076 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" -TableCommentType 4294967077 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" -TableCommentType 4294967078 0 "pg_publication_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967079 0 "pg_publication was created for compatibility and is currently unimplemented" -TableCommentType 4294967080 0 "pg_publication_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967081 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" -TableCommentType 4294967082 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" -TableCommentType 4294967083 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" -TableCommentType 4294967084 0 "pg_policy was created for compatibility and is currently unimplemented" -TableCommentType 4294967085 0 "pg_policies was created for compatibility and is currently unimplemented" -TableCommentType 4294967086 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" -TableCommentType 4294967087 0 "pg_opfamily was created for compatibility and is currently unimplemented" -TableCommentType 4294967088 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" -TableCommentType 4294967089 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" -TableCommentType 4294967090 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" -TableCommentType 4294967091 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" -TableCommentType 4294967092 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" -TableCommentType 4294967093 0 "pg_largeobject was created for compatibility and is currently unimplemented" -TableCommentType 4294967094 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" -TableCommentType 4294967095 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" -TableCommentType 4294967096 0 "pg_init_privs was created for compatibility and is currently unimplemented" -TableCommentType 4294967097 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" -TableCommentType 4294967098 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" -TableCommentType 4294967099 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" -TableCommentType 4294967100 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967101 0 "pg_group was created for compatibility and is currently unimplemented" -TableCommentType 4294967102 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" -TableCommentType 4294967103 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" -TableCommentType 4294967104 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" -TableCommentType 4294967105 0 "pg_file_settings was created for compatibility and is currently unimplemented" -TableCommentType 4294967106 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" -TableCommentType 4294967107 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" -TableCommentType 4294967108 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" -TableCommentType 4294967109 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" -TableCommentType 4294967110 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" -TableCommentType 4294967111 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 4294967112 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 4294967113 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" -TableCommentType 4294967114 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" -TableCommentType 4294967115 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" -TableCommentType 4294967116 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" -TableCommentType 4294967117 0 "pg_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967118 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" -TableCommentType 4294967119 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 4294967120 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" -TableCommentType 4294967121 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" -TableCommentType 4294967122 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" -TableCommentType 4294967123 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" -TableCommentType 4294967124 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 4294967125 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" -TableCommentType 4294967126 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" -TableCommentType 4294967127 0 "pg_amproc was created for compatibility and is currently unimplemented" -TableCommentType 4294967128 0 "pg_amop was created for compatibility and is currently unimplemented" -TableCommentType 4294967129 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" -TableCommentType 4294967130 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" -TableCommentType 4294967132 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" -TableCommentType 4294967133 0 "view_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967134 0 "view_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967135 0 "view_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967136 0 "grantable privileges (incomplete)" -TableCommentType 4294967137 0 "user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967138 0 "user_mapping_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967139 0 "user_defined_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967140 0 "user_attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967141 0 "usage_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967142 0 "udt_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967143 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" -TableCommentType 4294967144 0 "triggers was created for compatibility and is currently unimplemented" -TableCommentType 4294967145 0 "triggered_update_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967146 0 "transforms was created for compatibility and is currently unimplemented" -TableCommentType 4294967147 0 "tablespaces was created for compatibility and is currently unimplemented" -TableCommentType 4294967148 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967149 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 4294967150 0 "tables_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967151 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 4294967152 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967153 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 4294967154 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" -TableCommentType 4294967155 0 "st_units_of_measure was created for compatibility and is currently unimplemented" -TableCommentType 4294967156 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" -TableCommentType 4294967157 0 "st_geometry_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967158 0 "exposes the session variables." -TableCommentType 4294967159 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" -TableCommentType 4294967160 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" -TableCommentType 4294967161 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 4294967162 0 "schemata_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967163 0 "sql_sizing was created for compatibility and is currently unimplemented" -TableCommentType 4294967164 0 "sql_parts was created for compatibility and is currently unimplemented" -TableCommentType 4294967165 0 "sql_implementation_info was created for compatibility and is currently unimplemented" -TableCommentType 4294967166 0 "sql_features was created for compatibility and is currently unimplemented" -TableCommentType 4294967167 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" -TableCommentType 4294967168 0 "routine_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967169 0 "role_usage_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967170 0 "role_udt_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967171 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 4294967172 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" -TableCommentType 4294967173 0 "role_column_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967174 0 "resource_groups was created for compatibility and is currently unimplemented" -TableCommentType 4294967175 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 4294967176 0 "profiling was created for compatibility and is currently unimplemented" -TableCommentType 4294967177 0 "processlist was created for compatibility and is currently unimplemented" -TableCommentType 4294967178 0 "plugins was created for compatibility and is currently unimplemented" -TableCommentType 4294967179 0 "partitions was created for compatibility and is currently unimplemented" -TableCommentType 4294967180 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" -TableCommentType 4294967181 0 "optimizer_trace was created for compatibility and is currently unimplemented" -TableCommentType 4294967182 0 "keywords was created for compatibility and is currently unimplemented" -TableCommentType 4294967183 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 4294967184 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" -TableCommentType 4294967185 0 "foreign_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967186 0 "foreign_table_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967187 0 "foreign_servers was created for compatibility and is currently unimplemented" -TableCommentType 4294967188 0 "foreign_server_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967189 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" -TableCommentType 4294967190 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967191 0 "files was created for compatibility and is currently unimplemented" -TableCommentType 4294967192 0 "events was created for compatibility and is currently unimplemented" -TableCommentType 4294967193 0 "engines was created for compatibility and is currently unimplemented" -TableCommentType 4294967194 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 4294967195 0 "element_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967196 0 "domains was created for compatibility and is currently unimplemented" -TableCommentType 4294967197 0 "domain_udt_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967198 0 "domain_constraints was created for compatibility and is currently unimplemented" -TableCommentType 4294967199 0 "data_type_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967200 0 "constraint_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967201 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" -TableCommentType 4294967202 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 4294967203 0 "columns_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967204 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 4294967205 0 "column_statistics was created for compatibility and is currently unimplemented" -TableCommentType 4294967206 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 4294967207 0 "column_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967208 0 "column_domain_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967209 0 "column_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967210 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" -TableCommentType 4294967211 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 4294967212 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 4294967213 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967214 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 4294967215 0 "attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967216 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 4294967217 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 4294967219 0 "list super regions of databases visible to the current user" -TableCommentType 4294967220 0 "which entries of pg_catalog are implemented in this version of CockroachDB" -TableCommentType 4294967222 0 "node-level table listing all currently running range feeds" -TableCommentType 4294967223 0 "virtual table with default privileges" -TableCommentType 4294967224 0 "available regions for the cluster" -TableCommentType 4294967225 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" -TableCommentType 4294967226 0 "virtual table with table descriptors that still have data" -TableCommentType 4294967227 0 "virtual table with cross db references" -TableCommentType 4294967228 0 "virtual table with database privileges" -TableCommentType 4294967229 0 "virtual table to validate descriptors" -TableCommentType 4294967230 0 "decoded zone configurations from system.zones (KV scan)" -TableCommentType 4294967232 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 4294967233 0 "stats for all tables accessible by current user in current database as of 10s ago" -TableCommentType 4294967234 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" -TableCommentType 4294967235 0 "indexes accessible by current user in current database (KV scan)" -TableCommentType 4294967236 0 "details for all columns accessible by current user in current database (KV scan)" -TableCommentType 4294967238 0 "session variables (RAM)" -TableCommentType 4294967239 0 "session trace accumulated so far (RAM)" -TableCommentType 4294967240 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" -TableCommentType 4294967241 0 "server parameters, useful to construct connection URLs (RAM, local node only)" -TableCommentType 4294967243 0 "range metadata without leaseholder details (KV join; expensive!)" -TableCommentType 4294967244 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" -TableCommentType 4294967245 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 4294967246 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 4294967247 0 "current values for metrics (RAM; local node only)" -TableCommentType 4294967248 0 "running sessions visible by current user (RAM; local node only)" -TableCommentType 4294967249 0 "running user transactions visible by the current user (RAM; local node only)" -TableCommentType 4294967250 0 "running queries visible by current user (RAM; local node only)" -TableCommentType 4294967252 0 "DistSQL remote flows information (RAM; local node only)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on this node. The local flows (those that are\nrunning on the same node as the query originated on) are not included." -TableCommentType 4294967253 0 "contention information (RAM; local node only)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." -TableCommentType 4294967254 0 "acquired table leases (RAM; local node only)" -TableCommentType 4294967255 0 "store details and status (cluster RPC; expensive!)" -TableCommentType 4294967256 0 "node details across the entire cluster (cluster RPC; expensive!)" -TableCommentType 4294967257 0 "wrapper over system.jobs with row access control (KV scan)" -TableCommentType 4294967258 0 "decoded job metadata from crdb_internal.system_jobs (KV scan)" -TableCommentType 4294967259 0 "in-flight spans (RAM; local node only)" -TableCommentType 4294967260 0 "cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout." +TableCommentType 4294966996 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." +TableCommentType 4294966997 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." +TableCommentType 4294966998 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." +TableCommentType 4294967000 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" +TableCommentType 4294967001 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" +TableCommentType 4294967002 0 "pg_user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294967003 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" +TableCommentType 4294967004 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" +TableCommentType 4294967005 0 "pg_ts_template was created for compatibility and is currently unimplemented" +TableCommentType 4294967006 0 "pg_ts_parser was created for compatibility and is currently unimplemented" +TableCommentType 4294967007 0 "pg_ts_dict was created for compatibility and is currently unimplemented" +TableCommentType 4294967008 0 "pg_ts_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967009 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" +TableCommentType 4294967010 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" +TableCommentType 4294967011 0 "pg_transform was created for compatibility and is currently unimplemented" +TableCommentType 4294967012 0 "pg_timezone_names was created for compatibility and is currently unimplemented" +TableCommentType 4294967013 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" +TableCommentType 4294967014 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" +TableCommentType 4294967015 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" +TableCommentType 4294967016 0 "pg_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967017 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967018 0 "pg_stats was created for compatibility and is currently unimplemented" +TableCommentType 4294967019 0 "pg_stats_ext was created for compatibility and is currently unimplemented" +TableCommentType 4294967020 0 "pg_statistic was created for compatibility and is currently unimplemented" +TableCommentType 4294967021 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" +TableCommentType 4294967022 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" +TableCommentType 4294967023 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967024 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967025 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967026 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967027 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967028 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967029 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967030 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967031 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967032 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967033 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967034 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967035 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967036 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967037 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967038 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967039 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967040 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967041 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967042 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967043 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" +TableCommentType 4294967044 0 "pg_stat_slru was created for compatibility and is currently unimplemented" +TableCommentType 4294967045 0 "pg_stat_replication was created for compatibility and is currently unimplemented" +TableCommentType 4294967046 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" +TableCommentType 4294967047 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" +TableCommentType 4294967048 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" +TableCommentType 4294967049 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" +TableCommentType 4294967050 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" +TableCommentType 4294967051 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" +TableCommentType 4294967052 0 "pg_stat_database was created for compatibility and is currently unimplemented" +TableCommentType 4294967053 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" +TableCommentType 4294967054 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" +TableCommentType 4294967055 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967056 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967057 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967058 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 4294967059 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" +TableCommentType 4294967060 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" +TableCommentType 4294967061 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" +TableCommentType 4294967062 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" +TableCommentType 4294967063 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 4294967064 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" +TableCommentType 4294967065 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" +TableCommentType 4294967066 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" +TableCommentType 4294967067 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" +TableCommentType 4294967068 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" +TableCommentType 4294967069 0 "pg_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967070 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" +TableCommentType 4294967071 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 4294967072 0 "pg_replication_slots was created for compatibility and is currently unimplemented" +TableCommentType 4294967073 0 "pg_replication_origin was created for compatibility and is currently unimplemented" +TableCommentType 4294967074 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" +TableCommentType 4294967075 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" +TableCommentType 4294967076 0 "pg_publication_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967077 0 "pg_publication was created for compatibility and is currently unimplemented" +TableCommentType 4294967078 0 "pg_publication_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967079 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" +TableCommentType 4294967080 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" +TableCommentType 4294967081 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" +TableCommentType 4294967082 0 "pg_policy was created for compatibility and is currently unimplemented" +TableCommentType 4294967083 0 "pg_policies was created for compatibility and is currently unimplemented" +TableCommentType 4294967084 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" +TableCommentType 4294967085 0 "pg_opfamily was created for compatibility and is currently unimplemented" +TableCommentType 4294967086 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" +TableCommentType 4294967087 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" +TableCommentType 4294967088 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" +TableCommentType 4294967089 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" +TableCommentType 4294967090 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" +TableCommentType 4294967091 0 "pg_largeobject was created for compatibility and is currently unimplemented" +TableCommentType 4294967092 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" +TableCommentType 4294967093 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" +TableCommentType 4294967094 0 "pg_init_privs was created for compatibility and is currently unimplemented" +TableCommentType 4294967095 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" +TableCommentType 4294967096 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" +TableCommentType 4294967097 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" +TableCommentType 4294967098 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967099 0 "pg_group was created for compatibility and is currently unimplemented" +TableCommentType 4294967100 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" +TableCommentType 4294967101 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" +TableCommentType 4294967102 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" +TableCommentType 4294967103 0 "pg_file_settings was created for compatibility and is currently unimplemented" +TableCommentType 4294967104 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" +TableCommentType 4294967105 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" +TableCommentType 4294967106 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" +TableCommentType 4294967107 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" +TableCommentType 4294967108 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" +TableCommentType 4294967109 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 4294967110 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 4294967111 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" +TableCommentType 4294967112 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" +TableCommentType 4294967113 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" +TableCommentType 4294967114 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" +TableCommentType 4294967115 0 "pg_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967116 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" +TableCommentType 4294967117 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 4294967118 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" +TableCommentType 4294967119 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" +TableCommentType 4294967120 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" +TableCommentType 4294967121 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" +TableCommentType 4294967122 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 4294967123 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" +TableCommentType 4294967124 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" +TableCommentType 4294967125 0 "pg_amproc was created for compatibility and is currently unimplemented" +TableCommentType 4294967126 0 "pg_amop was created for compatibility and is currently unimplemented" +TableCommentType 4294967127 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" +TableCommentType 4294967128 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" +TableCommentType 4294967130 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" +TableCommentType 4294967131 0 "view_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967132 0 "view_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967133 0 "view_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967134 0 "grantable privileges (incomplete)" +TableCommentType 4294967135 0 "user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294967136 0 "user_mapping_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967137 0 "user_defined_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967138 0 "user_attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967139 0 "usage_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967140 0 "udt_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967141 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" +TableCommentType 4294967142 0 "triggers was created for compatibility and is currently unimplemented" +TableCommentType 4294967143 0 "triggered_update_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967144 0 "transforms was created for compatibility and is currently unimplemented" +TableCommentType 4294967145 0 "tablespaces was created for compatibility and is currently unimplemented" +TableCommentType 4294967146 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967147 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 4294967148 0 "tables_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967149 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 4294967150 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967151 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 4294967152 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" +TableCommentType 4294967153 0 "st_units_of_measure was created for compatibility and is currently unimplemented" +TableCommentType 4294967154 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" +TableCommentType 4294967155 0 "st_geometry_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967156 0 "exposes the session variables." +TableCommentType 4294967157 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" +TableCommentType 4294967158 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" +TableCommentType 4294967159 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 4294967160 0 "schemata_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967161 0 "sql_sizing was created for compatibility and is currently unimplemented" +TableCommentType 4294967162 0 "sql_parts was created for compatibility and is currently unimplemented" +TableCommentType 4294967163 0 "sql_implementation_info was created for compatibility and is currently unimplemented" +TableCommentType 4294967164 0 "sql_features was created for compatibility and is currently unimplemented" +TableCommentType 4294967165 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" +TableCommentType 4294967166 0 "routine_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967167 0 "role_usage_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967168 0 "role_udt_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967169 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 4294967170 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" +TableCommentType 4294967171 0 "role_column_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967172 0 "resource_groups was created for compatibility and is currently unimplemented" +TableCommentType 4294967173 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 4294967174 0 "profiling was created for compatibility and is currently unimplemented" +TableCommentType 4294967175 0 "processlist was created for compatibility and is currently unimplemented" +TableCommentType 4294967176 0 "plugins was created for compatibility and is currently unimplemented" +TableCommentType 4294967177 0 "partitions was created for compatibility and is currently unimplemented" +TableCommentType 4294967178 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" +TableCommentType 4294967179 0 "optimizer_trace was created for compatibility and is currently unimplemented" +TableCommentType 4294967180 0 "keywords was created for compatibility and is currently unimplemented" +TableCommentType 4294967181 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 4294967182 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" +TableCommentType 4294967183 0 "foreign_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967184 0 "foreign_table_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967185 0 "foreign_servers was created for compatibility and is currently unimplemented" +TableCommentType 4294967186 0 "foreign_server_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967187 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" +TableCommentType 4294967188 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967189 0 "files was created for compatibility and is currently unimplemented" +TableCommentType 4294967190 0 "events was created for compatibility and is currently unimplemented" +TableCommentType 4294967191 0 "engines was created for compatibility and is currently unimplemented" +TableCommentType 4294967192 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 4294967193 0 "element_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967194 0 "domains was created for compatibility and is currently unimplemented" +TableCommentType 4294967195 0 "domain_udt_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967196 0 "domain_constraints was created for compatibility and is currently unimplemented" +TableCommentType 4294967197 0 "data_type_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967198 0 "constraint_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967199 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" +TableCommentType 4294967200 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 4294967201 0 "columns_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967202 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 4294967203 0 "column_statistics was created for compatibility and is currently unimplemented" +TableCommentType 4294967204 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 4294967205 0 "column_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967206 0 "column_domain_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967207 0 "column_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967208 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" +TableCommentType 4294967209 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 4294967210 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 4294967211 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967212 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 4294967213 0 "attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967214 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 4294967215 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 4294967217 0 "list super regions of databases visible to the current user" +TableCommentType 4294967218 0 "which entries of pg_catalog are implemented in this version of CockroachDB" +TableCommentType 4294967220 0 "node-level table listing all currently running range feeds" +TableCommentType 4294967221 0 "virtual table with default privileges" +TableCommentType 4294967222 0 "available regions for the cluster" +TableCommentType 4294967223 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" +TableCommentType 4294967224 0 "virtual table with table descriptors that still have data" +TableCommentType 4294967225 0 "virtual table with cross db references" +TableCommentType 4294967226 0 "virtual table with database privileges" +TableCommentType 4294967227 0 "virtual table to validate descriptors" +TableCommentType 4294967228 0 "decoded zone configurations from system.zones (KV scan)" +TableCommentType 4294967230 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 4294967231 0 "stats for all tables accessible by current user in current database as of 10s ago" +TableCommentType 4294967232 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" +TableCommentType 4294967233 0 "key spans per SQL object" +TableCommentType 4294967234 0 "indexes accessible by current user in current database (KV scan)" +TableCommentType 4294967235 0 "details for all columns accessible by current user in current database (KV scan)" +TableCommentType 4294967237 0 "session variables (RAM)" +TableCommentType 4294967238 0 "session trace accumulated so far (RAM)" +TableCommentType 4294967239 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" +TableCommentType 4294967240 0 "server parameters, useful to construct connection URLs (RAM, local node only)" +TableCommentType 4294967242 0 "range metadata without leaseholder details (KV join; expensive!)" +TableCommentType 4294967243 0 "defined partitions for all tables/indexes accessible by the current user in the current database (KV scan)" +TableCommentType 4294967244 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 4294967245 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 4294967246 0 "current values for metrics (RAM; local node only)" +TableCommentType 4294967247 0 "running sessions visible by current user (RAM; local node only)" +TableCommentType 4294967248 0 "running user transactions visible by the current user (RAM; local node only)" +TableCommentType 4294967249 0 "running queries visible by current user (RAM; local node only)" +TableCommentType 4294967251 0 "DistSQL remote flows information (RAM; local node only)\n\nThis virtual table contains all of the remote flows of the DistSQL execution\nthat are currently running on this node. The local flows (those that are\nrunning on the same node as the query originated on) are not included." +TableCommentType 4294967252 0 "contention information (RAM; local node only)\n\nAll of the contention information internally stored in three levels:\n- on the highest, it is grouped by tableID/indexID pair\n- on the middle, it is grouped by key\n- on the lowest, it is grouped by txnID.\nEach of the levels is maintained as an LRU cache with limited size, so\nit is possible that not all of the contention information ever observed\nis contained in this table." +TableCommentType 4294967253 0 "acquired table leases (RAM; local node only)" +TableCommentType 4294967254 0 "store details and status (cluster RPC; expensive!)" +TableCommentType 4294967255 0 "node details across the entire cluster (cluster RPC; expensive!)" +TableCommentType 4294967256 0 "wrapper over system.jobs with row access control (KV scan)" +TableCommentType 4294967257 0 "decoded job metadata from crdb_internal.system_jobs (KV scan)" +TableCommentType 4294967258 0 "in-flight spans (RAM; local node only)" +TableCommentType 4294967259 0 "cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout." +TableCommentType 4294967260 0 "key spans per table index" TableCommentType 4294967261 0 "index columns for all indexes accessible by current user in current database (KV scan)" TableCommentType 4294967262 0 "cluster-wide transaction contention events. Querying this table is an\n\t\texpensive operation since it creates a cluster-wide RPC-fanout." TableCommentType 4294967263 0 "locally known edges in the gossip network (RAM; local node only)" diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index a064729ff81f..7d12b07a684e 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -27,25 +27,25 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1, # Verify data placement. query TTTI rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv WITH DETAILS] ---- -NULL /1 {1} 1 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 + …/1/1 {1} 1 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify data placement. query TTTI rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw WITH DETAILS] ---- -NULL /1 {5} 5 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 + …/1/1 {5} 5 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify execution. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_agg b/pkg/sql/logictest/testdata/logic_test/distsql_agg index 2e049aae35c8..3b7fc5d258c2 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_agg +++ b/pkg/sql/logictest/testdata/logic_test/distsql_agg @@ -22,19 +22,19 @@ INSERT INTO data SELECT a, b, c::FLOAT, d::DECIMAL FROM # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] ----- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] +---- +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 query RI SELECT sum(a), sum_int(a) FROM data @@ -323,20 +323,20 @@ statement ok INSERT INTO two VALUES (1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8), (9,9), (10,10) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE one] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE one WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /0 {5} 5 -/0 /99 {1} 1 -/99 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/0 {5} 5 +…/1/0 …/1/99 {1} 1 +…/1/99 {5} 5 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE two] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE two WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /0 {5} 5 -/0 /99 {2} 2 -/99 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/0 {5} 5 +…/1/0 …/1/99 {2} 2 +…/1/99 {5} 5 query I SELECT count(*) FROM one AS a, one AS b, two AS c @@ -701,11 +701,11 @@ ALTER TABLE table74736 EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 0), (ARRAY[2], 10 INSERT INTO table74736 SELECT x * 10000, repeat('a', 200000) FROM generate_series(1, 130) AS g(x); query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE table74736] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE table74736 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1000000 {1} 1 -/1000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1/1000000 {1} 1 +…/1/1000000 {2} 2 statement ok SET DISTSQL = OFF diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal b/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal index d61368ef83fa..4d0f22f735d6 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/distsql_crdb_internal @@ -22,19 +22,19 @@ INSERT INTO data SELECT a, b, c::FLOAT, d::DECIMAL FROM # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # As of #69273, crdb_internal.reset_sql_stats()'s behavior changes from in-memory diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_distinct_on b/pkg/sql/logictest/testdata/logic_test/distsql_distinct_on index 01ab3a68d5da..50adc30e9449 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_distinct_on +++ b/pkg/sql/logictest/testdata/logic_test/distsql_distinct_on @@ -62,25 +62,25 @@ ALTER TABLE abc EXPERIMENTAL_RELOCATE VALUES (ARRAY[5], '3', '4', '5') query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /4 {2} 2 -/4 /6 {3} 3 -/6 /7 {4} 4 -/7 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/4 {2} 2 +…/1/4 …/1/6 {3} 3 +…/1/6 …/1/7 {4} 4 +…/1/7 {5} 5 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE abc] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE abc WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /NULL/NULL/NULL {5} 5 -/NULL/NULL/NULL /"1"/"1"/"2" {1} 1 -/"1"/"1"/"2" /"1"/"2"/"2" {2} 2 -/"1"/"2"/"2" /"2"/"3"/"4" {3} 3 -/"2"/"3"/"4" /"3"/"4"/"5" {4} 4 -/"3"/"4"/"5" NULL {5} 5 +start_key end_key replicas lease_holder + …/1/NULL/NULL/NULL {5} 5 +…/1/NULL/NULL/NULL …/1/"1"/"1"/"2" {1} 1 +…/1/"1"/"1"/"2" …/1/"1"/"2"/"2" {2} 2 +…/1/"1"/"2"/"2" …/1/"2"/"3"/"4" {3} 3 +…/1/"2"/"3"/"4" …/1/"3"/"4"/"5" {4} 4 +…/1/"3"/"4"/"5" {5} 5 query III rowsort SELECT DISTINCT ON (x,y,z) x, y, z FROM xyz diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_numtables b/pkg/sql/logictest/testdata/logic_test/distsql_numtables index 5948e33cf9fd..dc13994140d0 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_numtables +++ b/pkg/sql/logictest/testdata/logic_test/distsql_numtables @@ -27,20 +27,20 @@ INSERT INTO NumToStr SELECT i, to_english(i) FROM generate_series(1, 100*100) AS # Verify data placement. query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToSquare] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToSquare WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder + {1} 1 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToStr] ----- -start_key end_key replicas lease_holder -NULL /2000 {1} 1 -/2000 /4000 {2} 2 -/4000 /6000 {3} 3 -/6000 /8000 {4} 4 -/8000 NULL {5} 5 +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToStr WITH DETAILS] +---- +start_key end_key replicas lease_holder + …/1/2000 {1} 1 +…/1/2000 …/1/4000 {2} 2 +…/1/4000 …/1/6000 {3} 3 +…/1/6000 …/1/8000 {4} 4 +…/1/8000 {5} 5 # # -- Basic tests -- diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_stats b/pkg/sql/logictest/testdata/logic_test/distsql_stats index dbe12bcc763a..647a63d616fa 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_stats +++ b/pkg/sql/logictest/testdata/logic_test/distsql_stats @@ -29,19 +29,19 @@ INSERT INTO data SELECT a, b, c::FLOAT, d::DECIMAL, (a+b+c+d) % 2 = 0 FROM # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] ----- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] +---- +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # Turn feature flag off and verify errors. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_subquery b/pkg/sql/logictest/testdata/logic_test/distsql_subquery index 95f059328846..30dfdf1780a7 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_subquery +++ b/pkg/sql/logictest/testdata/logic_test/distsql_subquery @@ -19,11 +19,11 @@ statement ok ALTER TABLE ab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 2) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE ab] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE ab WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 NULL {2} 2 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 {2} 2 query T SELECT ARRAY(SELECT a FROM ab ORDER BY b) diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_union b/pkg/sql/logictest/testdata/logic_test/distsql_union index a6960819a3ce..3cbb42ec8dfe 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_union +++ b/pkg/sql/logictest/testdata/logic_test/distsql_union @@ -28,14 +28,14 @@ ALTER TABLE xyz EXPERIMENTAL_RELOCATE VALUES (ARRAY[5], 5) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE xyz] ----- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE xyz WITH DETAILS] +---- +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 subtest Union diff --git a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node index 74c1105974cf..251f4d3ce7eb 100644 --- a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node +++ b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node @@ -19,27 +19,27 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1, # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {5} 5 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {5} 5 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 statement ok SET experimental_distsql_planning = always diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index f547d7b45fe7..4648319c9731 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -59,6 +59,7 @@ test crdb_internal gossip_liveness public test crdb_internal gossip_network public SELECT false test crdb_internal gossip_nodes public SELECT false test crdb_internal index_columns public SELECT false +test crdb_internal index_spans public SELECT false test crdb_internal index_usage_statistics public SELECT false test crdb_internal invalid_objects public SELECT false test crdb_internal jobs public SELECT false @@ -98,6 +99,7 @@ test crdb_internal system_jobs public test crdb_internal table_columns public SELECT false test crdb_internal table_indexes public SELECT false test crdb_internal table_row_statistics public SELECT false +test crdb_internal table_spans public SELECT false test crdb_internal tables public SELECT false test crdb_internal tenant_usage_details public SELECT false test crdb_internal transaction_contention_events public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index index 2f1fb018f623..11e47b3cd31c 100644 --- a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index +++ b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index @@ -810,28 +810,24 @@ CREATE TABLE t_hash_pre_split ( ); skipif config 3node-tenant-default-configs -query TITTT retry -SELECT t.name, r.table_id, r.index_name, r.start_pretty, r.end_pretty -FROM crdb_internal.tables t -JOIN crdb_internal.ranges r ON t.table_id = r.table_id -WHERE t.name = 't_hash_pre_split' -AND t.state = 'PUBLIC' -AND r.split_enforced_until IS NOT NULL; +query TITTT retry,rowsort +SELECT table_name, table_id, index_name, start_key, end_key + FROM [SHOW RANGES FROM DATABASE test WITH INDEXES] + WHERE table_name = 't_hash_pre_split' ---- +t_hash_pre_split 133 t_hash_pre_split_pkey /Table/128/3/7 /Max statement ok CREATE INDEX t_hash_pre_split_idx_b ON t_hash_pre_split (b) USING HASH WITH (bucket_count=8); skipif config 3node-tenant-default-configs -query TITTT colnames,retry -SELECT t.name, r.table_id, r.index_name, r.start_pretty, r.end_pretty -FROM crdb_internal.tables t -JOIN crdb_internal.ranges r ON t.table_id = r.table_id -WHERE t.name = 't_hash_pre_split' -AND t.state = 'PUBLIC' -AND r.split_enforced_until IS NOT NULL; +query TITTT colnames,retry,rowsort +SELECT table_name, table_id, index_name, start_key, end_key + FROM [SHOW RANGES FROM DATABASE test WITH INDEXES] + WHERE table_name = 't_hash_pre_split' ---- -name table_id index_name start_pretty end_pretty +table_name table_id index_name start_key end_key +t_hash_pre_split 133 t_hash_pre_split_pkey /Table/128/3/7 /Table/133/2 t_hash_pre_split 133 t_hash_pre_split_idx_b /Table/133/2 /Table/133/2/0 t_hash_pre_split 133 t_hash_pre_split_idx_b /Table/133/2/0 /Table/133/2/1 t_hash_pre_split 133 t_hash_pre_split_idx_b /Table/133/2/1 /Table/133/2/2 diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 97920f6dfcac..019437d95b73 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -424,6 +424,7 @@ crdb_internal gossip_liveness crdb_internal gossip_network crdb_internal gossip_nodes crdb_internal index_columns +crdb_internal index_spans crdb_internal index_usage_statistics crdb_internal invalid_objects crdb_internal jobs @@ -463,6 +464,7 @@ crdb_internal system_jobs crdb_internal table_columns crdb_internal table_indexes crdb_internal table_row_statistics +crdb_internal table_spans crdb_internal tables crdb_internal tenant_usage_details crdb_internal transaction_contention_events @@ -754,6 +756,7 @@ gossip_liveness gossip_network gossip_nodes index_columns +index_spans index_usage_statistics invalid_objects jobs @@ -793,6 +796,7 @@ system_jobs table_columns table_indexes table_row_statistics +table_spans tables tenant_usage_details transaction_contention_events @@ -1079,6 +1083,7 @@ tablespaces tables_extensions tables tables +table_spans table_row_statistics table_privileges table_indexes @@ -1123,6 +1128,7 @@ system crdb_internal gossip_liveness SYSTEM system crdb_internal gossip_network SYSTEM VIEW NO 1 system crdb_internal gossip_nodes SYSTEM VIEW NO 1 system crdb_internal index_columns SYSTEM VIEW NO 1 +system crdb_internal index_spans SYSTEM VIEW NO 1 system crdb_internal index_usage_statistics SYSTEM VIEW NO 1 system crdb_internal invalid_objects SYSTEM VIEW NO 1 system crdb_internal jobs SYSTEM VIEW NO 1 @@ -1162,6 +1168,7 @@ system crdb_internal system_jobs SYSTEM system crdb_internal table_columns SYSTEM VIEW NO 1 system crdb_internal table_indexes SYSTEM VIEW NO 1 system crdb_internal table_row_statistics SYSTEM VIEW NO 1 +system crdb_internal table_spans SYSTEM VIEW NO 1 system crdb_internal tables SYSTEM VIEW NO 1 system crdb_internal tenant_usage_details SYSTEM VIEW NO 1 system crdb_internal transaction_contention_events SYSTEM VIEW NO 1 @@ -2820,6 +2827,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NO YES NULL public system crdb_internal gossip_nodes SELECT NO YES NULL public system crdb_internal index_columns SELECT NO YES +NULL public system crdb_internal index_spans SELECT NO YES NULL public system crdb_internal index_usage_statistics SELECT NO YES NULL public system crdb_internal invalid_objects SELECT NO YES NULL public system crdb_internal jobs SELECT NO YES @@ -2859,6 +2867,7 @@ NULL public system crdb_internal system_jobs NULL public system crdb_internal table_columns SELECT NO YES NULL public system crdb_internal table_indexes SELECT NO YES NULL public system crdb_internal table_row_statistics SELECT NO YES +NULL public system crdb_internal table_spans SELECT NO YES NULL public system crdb_internal tables SELECT NO YES NULL public system crdb_internal tenant_usage_details SELECT NO YES NULL public system crdb_internal transaction_contention_events SELECT NO YES @@ -3406,6 +3415,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NO YES NULL public system crdb_internal gossip_nodes SELECT NO YES NULL public system crdb_internal index_columns SELECT NO YES +NULL public system crdb_internal index_spans SELECT NO YES NULL public system crdb_internal index_usage_statistics SELECT NO YES NULL public system crdb_internal invalid_objects SELECT NO YES NULL public system crdb_internal jobs SELECT NO YES @@ -3445,6 +3455,7 @@ NULL public system crdb_internal system_jobs NULL public system crdb_internal table_columns SELECT NO YES NULL public system crdb_internal table_indexes SELECT NO YES NULL public system crdb_internal table_row_statistics SELECT NO YES +NULL public system crdb_internal table_spans SELECT NO YES NULL public system crdb_internal tables SELECT NO YES NULL public system crdb_internal tenant_usage_details SELECT NO YES NULL public system crdb_internal transaction_contention_events SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist index d4b84a1f8622..c590c3481fb3 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist +++ b/pkg/sql/logictest/testdata/logic_test/inverted_filter_geospatial_dist @@ -49,7 +49,7 @@ statement ok ALTER INDEX geo_table@geom_index SPLIT AT VALUES (1152921574000000000) query TI colnames,rowsort -SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- replicas lease_holder {1} 1 @@ -59,11 +59,11 @@ statement ok ALTER INDEX geo_table@geom_index EXPERIMENTAL_RELOCATE VALUES (ARRAY[2], 1152921574000000000) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1152921574000000000 {1} 1 -/1152921574000000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1152921574000000000 {1} 1 +…/1152921574000000000 {2} 2 # Distributed. query I @@ -87,11 +87,11 @@ statement ok ALTER INDEX geo_table@geom_index EXPERIMENTAL_RELOCATE VALUES (ARRAY[2], 1) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1152921574000000000 {2} 2 -/1152921574000000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1152921574000000000 {2} 2 +…/1152921574000000000 {2} 2 query I SELECT k FROM geo_table@geom_index WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k @@ -115,7 +115,7 @@ statement ok CREATE INVERTED INDEX geom_index2 ON geo_table(s, geom) query TI colnames,rowsort -SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2] +SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2 WITH DETAILS] ---- replicas lease_holder {2} 2 @@ -136,7 +136,7 @@ statement ok ALTER INDEX geo_table@geom_index2 SPLIT AT VALUES ('foo', 1152921574000000000) query TI colnames,rowsort -SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2] +SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2 WITH DETAILS] ---- replicas lease_holder {2} 2 @@ -146,11 +146,11 @@ statement ok ALTER INDEX geo_table@geom_index2 EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'foo', 1152921574000000000) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /"foo"/1152921574000000000 {2} 2 -/"foo"/1152921574000000000 NULL {1} 1 +start_key end_key replicas lease_holder +…/ …/"foo"/1152921574000000000 {2} 2 +…/"foo"/1152921574000000000 {1} 1 # Distributed. query I diff --git a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist index 58df7c274998..c45b977dcfca 100644 --- a/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist +++ b/pkg/sql/logictest/testdata/logic_test/inverted_join_geospatial_dist @@ -36,12 +36,12 @@ statement ok ALTER TABLE ltable EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 2), (ARRAY[3], 3) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 {3} 3 query II SELECT lk, rk FROM ltable JOIN rtable@geom_index ON ST_Intersects(ltable.geom1, rtable.geom) ORDER BY (lk, rk) diff --git a/pkg/sql/logictest/testdata/logic_test/kv_builtin_functions_local b/pkg/sql/logictest/testdata/logic_test/kv_builtin_functions_local index 4ccd97dce827..d25e1f5ef993 100644 --- a/pkg/sql/logictest/testdata/logic_test/kv_builtin_functions_local +++ b/pkg/sql/logictest/testdata/logic_test/kv_builtin_functions_local @@ -9,7 +9,7 @@ ALTER TABLE t SPLIT AT VALUES (0) # Get the range that contains this table. let $rangeid -SELECT range_id FROM crdb_internal.ranges WHERE table_name = 't' +SELECT range_id FROM [SHOW RANGES FROM TABLE t] LIMIT 1 query B SELECT crdb_internal.kv_enqueue_replica($rangeid, 'mvccGC', true); diff --git a/pkg/sql/logictest/testdata/logic_test/merge_join_dist b/pkg/sql/logictest/testdata/logic_test/merge_join_dist index 72cd1429bbc8..3a1f5ea29ac6 100644 --- a/pkg/sql/logictest/testdata/logic_test/merge_join_dist +++ b/pkg/sql/logictest/testdata/logic_test/merge_join_dist @@ -27,20 +27,20 @@ statement ok ALTER TABLE r EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 2), (ARRAY[2], 3), (ARRAY[3], 1) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE l] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE l WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 {3} 3 query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE r] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE r WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -/2 /3 {1} 1 -/3 NULL {2} 2 -NULL /2 {3} 3 +start_key end_key replicas lease_holder +…/1/2 …/1/3 {1} 1 +…/1/3 {2} 2 + …/1/2 {3} 3 query III SELECT * FROM l LEFT OUTER JOIN r USING(a) WHERE a = 2 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index b836c5da6fae..ed8f3b68db1c 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -179,42 +179,42 @@ is_updatable b 120 2 28 is_updatable c 120 3 28 false is_updatable_view a 121 1 0 false is_updatable_view b 121 2 0 false -pg_class oid 4294967119 1 0 false -pg_class relname 4294967119 2 0 false -pg_class relnamespace 4294967119 3 0 false -pg_class reltype 4294967119 4 0 false -pg_class reloftype 4294967119 5 0 false -pg_class relowner 4294967119 6 0 false -pg_class relam 4294967119 7 0 false -pg_class relfilenode 4294967119 8 0 false -pg_class reltablespace 4294967119 9 0 false -pg_class relpages 4294967119 10 0 false -pg_class reltuples 4294967119 11 0 false -pg_class relallvisible 4294967119 12 0 false -pg_class reltoastrelid 4294967119 13 0 false -pg_class relhasindex 4294967119 14 0 false -pg_class relisshared 4294967119 15 0 false -pg_class relpersistence 4294967119 16 0 false -pg_class relistemp 4294967119 17 0 false -pg_class relkind 4294967119 18 0 false -pg_class relnatts 4294967119 19 0 false -pg_class relchecks 4294967119 20 0 false -pg_class relhasoids 4294967119 21 0 false -pg_class relhaspkey 4294967119 22 0 false -pg_class relhasrules 4294967119 23 0 false -pg_class relhastriggers 4294967119 24 0 false -pg_class relhassubclass 4294967119 25 0 false -pg_class relfrozenxid 4294967119 26 0 false -pg_class relacl 4294967119 27 0 false -pg_class reloptions 4294967119 28 0 false -pg_class relforcerowsecurity 4294967119 29 0 false -pg_class relispartition 4294967119 30 0 false -pg_class relispopulated 4294967119 31 0 false -pg_class relreplident 4294967119 32 0 false -pg_class relrewrite 4294967119 33 0 false -pg_class relrowsecurity 4294967119 34 0 false -pg_class relpartbound 4294967119 35 0 false -pg_class relminmxid 4294967119 36 0 false +pg_class oid 4294967117 1 0 false +pg_class relname 4294967117 2 0 false +pg_class relnamespace 4294967117 3 0 false +pg_class reltype 4294967117 4 0 false +pg_class reloftype 4294967117 5 0 false +pg_class relowner 4294967117 6 0 false +pg_class relam 4294967117 7 0 false +pg_class relfilenode 4294967117 8 0 false +pg_class reltablespace 4294967117 9 0 false +pg_class relpages 4294967117 10 0 false +pg_class reltuples 4294967117 11 0 false +pg_class relallvisible 4294967117 12 0 false +pg_class reltoastrelid 4294967117 13 0 false +pg_class relhasindex 4294967117 14 0 false +pg_class relisshared 4294967117 15 0 false +pg_class relpersistence 4294967117 16 0 false +pg_class relistemp 4294967117 17 0 false +pg_class relkind 4294967117 18 0 false +pg_class relnatts 4294967117 19 0 false +pg_class relchecks 4294967117 20 0 false +pg_class relhasoids 4294967117 21 0 false +pg_class relhaspkey 4294967117 22 0 false +pg_class relhasrules 4294967117 23 0 false +pg_class relhastriggers 4294967117 24 0 false +pg_class relhassubclass 4294967117 25 0 false +pg_class relfrozenxid 4294967117 26 0 false +pg_class relacl 4294967117 27 0 false +pg_class reloptions 4294967117 28 0 false +pg_class relforcerowsecurity 4294967117 29 0 false +pg_class relispartition 4294967117 30 0 false +pg_class relispopulated 4294967117 31 0 false +pg_class relreplident 4294967117 32 0 false +pg_class relrewrite 4294967117 33 0 false +pg_class relrowsecurity 4294967117 34 0 false +pg_class relpartbound 4294967117 35 0 false +pg_class relminmxid 4294967117 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 7fa15225c134..c3b7b0072089 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 -4294967218 information_schema NULL NULL -4294967131 pg_catalog NULL NULL -4294967001 pg_extension NULL NULL +4294967216 information_schema NULL NULL +4294967129 pg_catalog NULL NULL +4294966999 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 -4294967218 information_schema NULL NULL -4294967131 pg_catalog NULL NULL -4294967001 pg_extension NULL NULL +4294967216 information_schema NULL NULL +4294967129 pg_catalog NULL NULL +4294966999 pg_extension NULL NULL 105 public 2310524507 NULL user root @@ -1302,7 +1302,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967131 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967129 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1501,16 +1501,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967116 111 0 4294967119 110 14 a -4294967116 112 0 4294967119 110 15 a -4294967073 842401391 0 4294967119 110 1 n -4294967073 842401391 0 4294967119 110 2 n -4294967073 842401391 0 4294967119 110 3 n -4294967073 842401391 0 4294967119 110 4 n -4294967116 1179276562 0 4294967119 3687884464 0 n -4294967116 3935750373 0 4294967119 3687884465 0 n -4294967116 4072017905 0 4294967119 0 0 n -4294967116 4170826110 0 4294967119 0 0 n +4294967114 111 0 4294967117 110 14 a +4294967114 112 0 4294967117 110 15 a +4294967071 842401391 0 4294967117 110 1 n +4294967071 842401391 0 4294967117 110 2 n +4294967071 842401391 0 4294967117 110 3 n +4294967071 842401391 0 4294967117 110 4 n +4294967114 1179276562 0 4294967117 3687884464 0 n +4294967114 3935750373 0 4294967117 3687884465 0 n +4294967114 4072017905 0 4294967117 0 0 n +4294967114 4170826110 0 4294967117 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 @@ -1523,8 +1523,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967073 4294967119 pg_rewrite pg_class -4294967116 4294967119 pg_constraint pg_class +4294967071 4294967117 pg_rewrite pg_class +4294967114 4294967117 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 @@ -1623,88 +1623,88 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967131 NULL 1 true b -17 bytea 4294967131 NULL -1 false b -18 char 4294967131 NULL 1 true b -19 name 4294967131 NULL -1 false b -20 int8 4294967131 NULL 8 true b -21 int2 4294967131 NULL 2 true b -22 int2vector 4294967131 NULL -1 false b -23 int4 4294967131 NULL 4 true b -24 regproc 4294967131 NULL 8 true b -25 text 4294967131 NULL -1 false b -26 oid 4294967131 NULL 8 true b -30 oidvector 4294967131 NULL -1 false b -700 float4 4294967131 NULL 4 true b -701 float8 4294967131 NULL 8 true b -705 unknown 4294967131 NULL 0 true b -869 inet 4294967131 NULL 24 true b -1000 _bool 4294967131 NULL -1 false b -1001 _bytea 4294967131 NULL -1 false b -1002 _char 4294967131 NULL -1 false b -1003 _name 4294967131 NULL -1 false b -1005 _int2 4294967131 NULL -1 false b -1006 _int2vector 4294967131 NULL -1 false b -1007 _int4 4294967131 NULL -1 false b -1008 _regproc 4294967131 NULL -1 false b -1009 _text 4294967131 NULL -1 false b -1013 _oidvector 4294967131 NULL -1 false b -1014 _bpchar 4294967131 NULL -1 false b -1015 _varchar 4294967131 NULL -1 false b -1016 _int8 4294967131 NULL -1 false b -1021 _float4 4294967131 NULL -1 false b -1022 _float8 4294967131 NULL -1 false b -1028 _oid 4294967131 NULL -1 false b -1041 _inet 4294967131 NULL -1 false b -1042 bpchar 4294967131 NULL -1 false b -1043 varchar 4294967131 NULL -1 false b -1082 date 4294967131 NULL 16 true b -1083 time 4294967131 NULL 8 true b -1114 timestamp 4294967131 NULL 24 true b -1115 _timestamp 4294967131 NULL -1 false b -1182 _date 4294967131 NULL -1 false b -1183 _time 4294967131 NULL -1 false b -1184 timestamptz 4294967131 NULL 24 true b -1185 _timestamptz 4294967131 NULL -1 false b -1186 interval 4294967131 NULL 24 true b -1187 _interval 4294967131 NULL -1 false b -1231 _numeric 4294967131 NULL -1 false b -1266 timetz 4294967131 NULL 16 true b -1270 _timetz 4294967131 NULL -1 false b -1560 bit 4294967131 NULL -1 false b -1561 _bit 4294967131 NULL -1 false b -1562 varbit 4294967131 NULL -1 false b -1563 _varbit 4294967131 NULL -1 false b -1700 numeric 4294967131 NULL -1 false b -2202 regprocedure 4294967131 NULL 8 true b -2205 regclass 4294967131 NULL 8 true b -2206 regtype 4294967131 NULL 8 true b -2207 _regprocedure 4294967131 NULL -1 false b -2210 _regclass 4294967131 NULL -1 false b -2211 _regtype 4294967131 NULL -1 false b -2249 record 4294967131 NULL 0 true p -2277 anyarray 4294967131 NULL -1 false p -2278 void 4294967131 NULL 0 true p -2283 anyelement 4294967131 NULL -1 false p -2287 _record 4294967131 NULL -1 false b -2950 uuid 4294967131 NULL 16 true b -2951 _uuid 4294967131 NULL -1 false b -3614 tsvector 4294967131 NULL -1 false b -3615 tsquery 4294967131 NULL -1 false b -3643 _tsvector 4294967131 NULL -1 false b -3645 _tsquery 4294967131 NULL -1 false b -3802 jsonb 4294967131 NULL -1 false b -3807 _jsonb 4294967131 NULL -1 false b -4089 regnamespace 4294967131 NULL 8 true b -4090 _regnamespace 4294967131 NULL -1 false b -4096 regrole 4294967131 NULL 8 true b -4097 _regrole 4294967131 NULL -1 false b -90000 geometry 4294967131 NULL -1 false b -90001 _geometry 4294967131 NULL -1 false b -90002 geography 4294967131 NULL -1 false b -90003 _geography 4294967131 NULL -1 false b -90004 box2d 4294967131 NULL 32 true b -90005 _box2d 4294967131 NULL -1 false b +16 bool 4294967129 NULL 1 true b +17 bytea 4294967129 NULL -1 false b +18 char 4294967129 NULL 1 true b +19 name 4294967129 NULL -1 false b +20 int8 4294967129 NULL 8 true b +21 int2 4294967129 NULL 2 true b +22 int2vector 4294967129 NULL -1 false b +23 int4 4294967129 NULL 4 true b +24 regproc 4294967129 NULL 8 true b +25 text 4294967129 NULL -1 false b +26 oid 4294967129 NULL 8 true b +30 oidvector 4294967129 NULL -1 false b +700 float4 4294967129 NULL 4 true b +701 float8 4294967129 NULL 8 true b +705 unknown 4294967129 NULL 0 true b +869 inet 4294967129 NULL 24 true b +1000 _bool 4294967129 NULL -1 false b +1001 _bytea 4294967129 NULL -1 false b +1002 _char 4294967129 NULL -1 false b +1003 _name 4294967129 NULL -1 false b +1005 _int2 4294967129 NULL -1 false b +1006 _int2vector 4294967129 NULL -1 false b +1007 _int4 4294967129 NULL -1 false b +1008 _regproc 4294967129 NULL -1 false b +1009 _text 4294967129 NULL -1 false b +1013 _oidvector 4294967129 NULL -1 false b +1014 _bpchar 4294967129 NULL -1 false b +1015 _varchar 4294967129 NULL -1 false b +1016 _int8 4294967129 NULL -1 false b +1021 _float4 4294967129 NULL -1 false b +1022 _float8 4294967129 NULL -1 false b +1028 _oid 4294967129 NULL -1 false b +1041 _inet 4294967129 NULL -1 false b +1042 bpchar 4294967129 NULL -1 false b +1043 varchar 4294967129 NULL -1 false b +1082 date 4294967129 NULL 16 true b +1083 time 4294967129 NULL 8 true b +1114 timestamp 4294967129 NULL 24 true b +1115 _timestamp 4294967129 NULL -1 false b +1182 _date 4294967129 NULL -1 false b +1183 _time 4294967129 NULL -1 false b +1184 timestamptz 4294967129 NULL 24 true b +1185 _timestamptz 4294967129 NULL -1 false b +1186 interval 4294967129 NULL 24 true b +1187 _interval 4294967129 NULL -1 false b +1231 _numeric 4294967129 NULL -1 false b +1266 timetz 4294967129 NULL 16 true b +1270 _timetz 4294967129 NULL -1 false b +1560 bit 4294967129 NULL -1 false b +1561 _bit 4294967129 NULL -1 false b +1562 varbit 4294967129 NULL -1 false b +1563 _varbit 4294967129 NULL -1 false b +1700 numeric 4294967129 NULL -1 false b +2202 regprocedure 4294967129 NULL 8 true b +2205 regclass 4294967129 NULL 8 true b +2206 regtype 4294967129 NULL 8 true b +2207 _regprocedure 4294967129 NULL -1 false b +2210 _regclass 4294967129 NULL -1 false b +2211 _regtype 4294967129 NULL -1 false b +2249 record 4294967129 NULL 0 true p +2277 anyarray 4294967129 NULL -1 false p +2278 void 4294967129 NULL 0 true p +2283 anyelement 4294967129 NULL -1 false p +2287 _record 4294967129 NULL -1 false b +2950 uuid 4294967129 NULL 16 true b +2951 _uuid 4294967129 NULL -1 false b +3614 tsvector 4294967129 NULL -1 false b +3615 tsquery 4294967129 NULL -1 false b +3643 _tsvector 4294967129 NULL -1 false b +3645 _tsquery 4294967129 NULL -1 false b +3802 jsonb 4294967129 NULL -1 false b +3807 _jsonb 4294967129 NULL -1 false b +4089 regnamespace 4294967129 NULL 8 true b +4090 _regnamespace 4294967129 NULL -1 false b +4096 regrole 4294967129 NULL 8 true b +4097 _regrole 4294967129 NULL -1 false b +90000 geometry 4294967129 NULL -1 false b +90001 _geometry 4294967129 NULL -1 false b +90002 geography 4294967129 NULL -1 false b +90003 _geography 4294967129 NULL -1 false b +90004 box2d 4294967129 NULL 32 true b +90005 _box2d 4294967129 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 @@ -2180,7 +2180,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967131 NULL -1 false b +1000 _bool 4294967129 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2238,7 +2238,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967081 pg_proc 4294967131 2310524507 -1 false c +4294967079 pg_proc 4294967129 2310524507 -1 false c ## pg_catalog.pg_proc @@ -2248,14 +2248,14 @@ FROM pg_catalog.pg_proc WHERE proname='substring' ---- proname pronamespace proowner prolang procost prorows provariadic -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 -substring 4294967131 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 +substring 4294967129 NULL 0 NULL NULL 0 query TTBBBB colnames SELECT proname, protransform, proisagg, proiswindow, prosecdef, proleakproof @@ -2409,291 +2409,293 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294966998 4294967119 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. -4294966999 4294967119 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967000 4294967119 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967002 4294967119 0 view definitions (incomplete - see also information_schema.views) -4294967003 4294967119 0 database users -4294967004 4294967119 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967005 4294967119 0 local to remote user mapping (empty - feature does not exist) -4294967006 4294967119 0 scalar types (incomplete) -4294967007 4294967119 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967008 4294967119 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967009 4294967119 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967010 4294967119 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967011 4294967119 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967012 4294967119 0 triggers (empty - feature does not exist) -4294967013 4294967119 0 pg_transform was created for compatibility and is currently unimplemented -4294967014 4294967119 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967015 4294967119 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967016 4294967119 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967017 4294967119 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967018 4294967119 0 pg_subscription was created for compatibility and is currently unimplemented -4294967019 4294967119 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967020 4294967119 0 pg_stats was created for compatibility and is currently unimplemented -4294967021 4294967119 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967022 4294967119 0 pg_statistic was created for compatibility and is currently unimplemented -4294967023 4294967119 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967024 4294967119 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967025 4294967119 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967026 4294967119 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967027 4294967119 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967028 4294967119 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967029 4294967119 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967030 4294967119 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967031 4294967119 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967032 4294967119 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967033 4294967119 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967034 4294967119 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967035 4294967119 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967036 4294967119 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967037 4294967119 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967038 4294967119 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967039 4294967119 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967040 4294967119 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967041 4294967119 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967042 4294967119 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967043 4294967119 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967044 4294967119 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967045 4294967119 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967046 4294967119 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967047 4294967119 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967048 4294967119 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967049 4294967119 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967050 4294967119 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967051 4294967119 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967052 4294967119 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967053 4294967119 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967054 4294967119 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967055 4294967119 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967056 4294967119 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967057 4294967119 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967058 4294967119 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967059 4294967119 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967060 4294967119 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967061 4294967119 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967062 4294967119 0 Shared Dependencies (Roles depending on objects). -4294967063 4294967119 0 shared security labels (empty - feature not supported) -4294967064 4294967119 0 shared object comments -4294967065 4294967119 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967066 4294967119 0 session variables (incomplete) -4294967067 4294967119 0 pg_sequences is very similar as pg_sequence. -4294967068 4294967119 0 sequences (see also information_schema.sequences) -4294967069 4294967119 0 security labels (empty - feature does not exist) -4294967070 4294967119 0 security labels (empty) -4294967071 4294967119 0 pg_rules was created for compatibility and is currently unimplemented -4294967072 4294967119 0 database roles -4294967073 4294967119 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967074 4294967119 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967075 4294967119 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967076 4294967119 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967077 4294967119 0 range types (empty - feature does not exist) -4294967078 4294967119 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967079 4294967119 0 pg_publication was created for compatibility and is currently unimplemented -4294967080 4294967119 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967081 4294967119 0 built-in functions (incomplete) -4294967082 4294967119 0 prepared transactions (empty - feature does not exist) -4294967083 4294967119 0 prepared statements -4294967084 4294967119 0 pg_policy was created for compatibility and is currently unimplemented -4294967085 4294967119 0 pg_policies was created for compatibility and is currently unimplemented -4294967086 4294967119 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967087 4294967119 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967088 4294967119 0 operators (incomplete) -4294967089 4294967119 0 opclass (empty - Operator classes not supported yet) -4294967090 4294967119 0 available namespaces -4294967091 4294967119 0 available materialized views (empty - feature does not exist) -4294967092 4294967119 0 locks held by active processes (empty - feature does not exist) -4294967093 4294967119 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967094 4294967119 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967095 4294967119 0 available languages (empty - feature does not exist) -4294967096 4294967119 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967097 4294967119 0 table inheritance hierarchy (empty - feature does not exist) -4294967098 4294967119 0 index creation statements -4294967099 4294967119 0 indexes (incomplete) -4294967100 4294967119 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967101 4294967119 0 pg_group was created for compatibility and is currently unimplemented -4294967102 4294967119 0 foreign tables (empty - feature does not exist) -4294967103 4294967119 0 foreign servers (empty - feature does not exist) -4294967104 4294967119 0 foreign data wrappers (empty - feature does not exist) -4294967105 4294967119 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967106 4294967119 0 installed extensions (empty - feature does not exist) -4294967107 4294967119 0 event triggers (empty - feature does not exist) -4294967108 4294967119 0 enum types and labels (empty - feature does not exist) -4294967109 4294967119 0 object comments -4294967110 4294967119 0 dependency relationships (incomplete) -4294967111 4294967119 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967112 4294967119 0 contains the default values that have been configured for session variables -4294967113 4294967119 0 available databases (incomplete) -4294967114 4294967119 0 contains currently active SQL cursors created with DECLARE -4294967115 4294967119 0 encoding conversions (empty - unimplemented) -4294967116 4294967119 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967117 4294967119 0 pg_config was created for compatibility and is currently unimplemented -4294967118 4294967119 0 available collations (incomplete) -4294967119 4294967119 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967120 4294967119 0 casts (empty - needs filling out) -4294967121 4294967119 0 available extensions -4294967122 4294967119 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967123 4294967119 0 role membership -4294967124 4294967119 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967125 4294967119 0 table columns (incomplete - see also information_schema.columns) -4294967126 4294967119 0 column default values -4294967127 4294967119 0 pg_amproc was created for compatibility and is currently unimplemented -4294967128 4294967119 0 pg_amop was created for compatibility and is currently unimplemented -4294967129 4294967119 0 index access methods (incomplete) -4294967130 4294967119 0 aggregated built-in functions (incomplete) -4294967132 4294967119 0 views (incomplete) -4294967133 4294967119 0 view_table_usage was created for compatibility and is currently unimplemented -4294967134 4294967119 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967135 4294967119 0 view_column_usage was created for compatibility and is currently unimplemented -4294967136 4294967119 0 grantable privileges (incomplete) -4294967137 4294967119 0 user_mappings was created for compatibility and is currently unimplemented -4294967138 4294967119 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967139 4294967119 0 user_defined_types was created for compatibility and is currently unimplemented -4294967140 4294967119 0 user_attributes was created for compatibility and is currently unimplemented -4294967141 4294967119 0 usage_privileges was created for compatibility and is currently unimplemented -4294967142 4294967119 0 udt_privileges was created for compatibility and is currently unimplemented -4294967143 4294967119 0 type privileges (incomplete; may contain excess users or roles) -4294967144 4294967119 0 triggers was created for compatibility and is currently unimplemented -4294967145 4294967119 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967146 4294967119 0 transforms was created for compatibility and is currently unimplemented -4294967147 4294967119 0 tablespaces was created for compatibility and is currently unimplemented -4294967148 4294967119 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967149 4294967119 0 tables and views -4294967150 4294967119 0 tables_extensions was created for compatibility and is currently unimplemented -4294967151 4294967119 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967152 4294967119 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967153 4294967119 0 table constraints -4294967154 4294967119 0 index metadata and statistics (incomplete) -4294967155 4294967119 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967156 4294967119 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967157 4294967119 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967158 4294967119 0 exposes the session variables. -4294967159 4294967119 0 sequences -4294967160 4294967119 0 schema privileges (incomplete; may contain excess users or roles) -4294967161 4294967119 0 database schemas (may contain schemata without permission) -4294967162 4294967119 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967163 4294967119 0 sql_sizing was created for compatibility and is currently unimplemented -4294967164 4294967119 0 sql_parts was created for compatibility and is currently unimplemented -4294967165 4294967119 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967166 4294967119 0 sql_features was created for compatibility and is currently unimplemented -4294967167 4294967119 0 built-in functions (empty - introspection not yet supported) -4294967168 4294967119 0 routine_privileges was created for compatibility and is currently unimplemented -4294967169 4294967119 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967170 4294967119 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967171 4294967119 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967172 4294967119 0 privileges granted on functions (incomplete; only contains privileges of user-defined functions) -4294967173 4294967119 0 role_column_grants was created for compatibility and is currently unimplemented -4294967174 4294967119 0 resource_groups was created for compatibility and is currently unimplemented -4294967175 4294967119 0 foreign key constraints -4294967176 4294967119 0 profiling was created for compatibility and is currently unimplemented -4294967177 4294967119 0 processlist was created for compatibility and is currently unimplemented -4294967178 4294967119 0 plugins was created for compatibility and is currently unimplemented -4294967179 4294967119 0 partitions was created for compatibility and is currently unimplemented -4294967180 4294967119 0 built-in function parameters (empty - introspection not yet supported) -4294967181 4294967119 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967182 4294967119 0 keywords was created for compatibility and is currently unimplemented -4294967183 4294967119 0 column usage by indexes and key constraints -4294967184 4294967119 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967185 4294967119 0 foreign_tables was created for compatibility and is currently unimplemented -4294967186 4294967119 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967187 4294967119 0 foreign_servers was created for compatibility and is currently unimplemented -4294967188 4294967119 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967189 4294967119 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967190 4294967119 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967191 4294967119 0 files was created for compatibility and is currently unimplemented -4294967192 4294967119 0 events was created for compatibility and is currently unimplemented -4294967193 4294967119 0 engines was created for compatibility and is currently unimplemented -4294967194 4294967119 0 roles for the current user -4294967195 4294967119 0 element_types was created for compatibility and is currently unimplemented -4294967196 4294967119 0 domains was created for compatibility and is currently unimplemented -4294967197 4294967119 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967198 4294967119 0 domain_constraints was created for compatibility and is currently unimplemented -4294967199 4294967119 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967200 4294967119 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967201 4294967119 0 columns usage by constraints -4294967202 4294967119 0 table and view columns (incomplete) -4294967203 4294967119 0 columns_extensions was created for compatibility and is currently unimplemented -4294967204 4294967119 0 columns with user defined types -4294967205 4294967119 0 column_statistics was created for compatibility and is currently unimplemented -4294967206 4294967119 0 column privilege grants (incomplete) -4294967207 4294967119 0 column_options was created for compatibility and is currently unimplemented -4294967208 4294967119 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967209 4294967119 0 column_column_usage was created for compatibility and is currently unimplemented -4294967210 4294967119 0 shows the collations available in the current database -4294967211 4294967119 0 identifies which character set the available collations are -4294967212 4294967119 0 check constraints -4294967213 4294967119 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967214 4294967119 0 character sets available in the current database -4294967215 4294967119 0 attributes was created for compatibility and is currently unimplemented -4294967216 4294967119 0 roles available to the current user -4294967217 4294967119 0 roles for which the current user has admin option -4294967219 4294967119 0 list super regions of databases visible to the current user -4294967220 4294967119 0 which entries of pg_catalog are implemented in this version of CockroachDB -4294967222 4294967119 0 node-level table listing all currently running range feeds -4294967223 4294967119 0 virtual table with default privileges -4294967224 4294967119 0 available regions for the cluster -4294967225 4294967119 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967226 4294967119 0 virtual table with table descriptors that still have data -4294967227 4294967119 0 virtual table with cross db references -4294967228 4294967119 0 virtual table with database privileges -4294967229 4294967119 0 virtual table to validate descriptors -4294967230 4294967119 0 decoded zone configurations from system.zones (KV scan) -4294967232 4294967119 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). -4294967233 4294967119 0 stats for all tables accessible by current user in current database as of 10s ago -4294967234 4294967119 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967235 4294967119 0 indexes accessible by current user in current database (KV scan) -4294967236 4294967119 0 details for all columns accessible by current user in current database (KV scan) -4294967238 4294967119 0 session variables (RAM) -4294967239 4294967119 0 session trace accumulated so far (RAM) -4294967240 4294967119 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967241 4294967119 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967243 4294967119 0 range metadata without leaseholder details (KV join; expensive!) -4294967244 4294967119 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967245 4294967119 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967246 4294967119 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). -4294967247 4294967119 0 current values for metrics (RAM; local node only) -4294967248 4294967119 0 running sessions visible by current user (RAM; local node only) -4294967249 4294967119 0 running user transactions visible by the current user (RAM; local node only) -4294967250 4294967119 0 running queries visible by current user (RAM; local node only) -4294967252 4294967119 0 DistSQL remote flows information (RAM; local node only) -4294967253 4294967119 0 contention information (RAM; local node only) -4294967254 4294967119 0 acquired table leases (RAM; local node only) -4294967255 4294967119 0 store details and status (cluster RPC; expensive!) -4294967256 4294967119 0 node details across the entire cluster (cluster RPC; expensive!) -4294967257 4294967119 0 wrapper over system.jobs with row access control (KV scan) -4294967258 4294967119 0 decoded job metadata from crdb_internal.system_jobs (KV scan) -4294967259 4294967119 0 in-flight spans (RAM; local node only) -4294967260 4294967119 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967261 4294967119 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967262 4294967119 0 cluster-wide transaction contention events. Querying this table is an -4294967263 4294967119 0 locally known edges in the gossip network (RAM; local node only) -4294967264 4294967119 0 locally known gossiped node liveness (RAM; local node only) -4294967265 4294967119 0 locally known gossiped health alerts (RAM; local node only) -4294967266 4294967119 0 locally known gossiped node details (RAM; local node only) -4294967267 4294967119 0 node liveness status, as seen by kv -4294967268 4294967119 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967269 4294967119 0 telemetry counters (RAM; local node only) -4294967270 4294967119 0 databases accessible by the current user (KV scan) -4294967271 4294967119 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967272 4294967119 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967273 4294967119 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967274 4294967119 0 CREATE statements for all user-defined functions -4294967275 4294967119 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967276 4294967119 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967277 4294967119 0 cluster settings (RAM) -4294967278 4294967119 0 running sessions visible to current user (cluster RPC; expensive!) -4294967279 4294967119 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967280 4294967119 0 running queries visible by current user (cluster RPC; expensive!) -4294967281 4294967119 0 cluster-wide locks held in lock tables. Querying this table is an -4294967283 4294967119 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967284 4294967119 0 contention information (cluster RPC; expensive!) -4294967288 4294967119 0 like system.zones but overlaid with in-txn in-memory changes -4294967289 4294967119 0 like system.namespace but overlaid with in-txn in-memory changes -4294967290 4294967119 0 like system.descriptor but overlaid with in-txn in-memory changes and including virtual objects -4294967291 4294967119 0 like system.comments but overlaid with in-txn in-memory changes and including virtual objects -4294967292 4294967119 0 built-in functions (RAM/static) -4294967293 4294967119 0 detailed identification strings (RAM, local node only) -4294967294 4294967119 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294966996 4294967117 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294966997 4294967117 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294966998 4294967117 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967000 4294967117 0 view definitions (incomplete - see also information_schema.views) +4294967001 4294967117 0 database users +4294967002 4294967117 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967003 4294967117 0 local to remote user mapping (empty - feature does not exist) +4294967004 4294967117 0 scalar types (incomplete) +4294967005 4294967117 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967006 4294967117 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967007 4294967117 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967008 4294967117 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967009 4294967117 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967010 4294967117 0 triggers (empty - feature does not exist) +4294967011 4294967117 0 pg_transform was created for compatibility and is currently unimplemented +4294967012 4294967117 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967013 4294967117 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967014 4294967117 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967015 4294967117 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967016 4294967117 0 pg_subscription was created for compatibility and is currently unimplemented +4294967017 4294967117 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967018 4294967117 0 pg_stats was created for compatibility and is currently unimplemented +4294967019 4294967117 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967020 4294967117 0 pg_statistic was created for compatibility and is currently unimplemented +4294967021 4294967117 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967022 4294967117 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967023 4294967117 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967024 4294967117 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967025 4294967117 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967026 4294967117 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967027 4294967117 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967028 4294967117 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967029 4294967117 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967030 4294967117 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967031 4294967117 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967032 4294967117 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967033 4294967117 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967034 4294967117 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967035 4294967117 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967036 4294967117 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967037 4294967117 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967038 4294967117 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967039 4294967117 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967040 4294967117 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967041 4294967117 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967042 4294967117 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967043 4294967117 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967044 4294967117 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967045 4294967117 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967046 4294967117 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967047 4294967117 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967048 4294967117 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967049 4294967117 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967050 4294967117 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967051 4294967117 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967052 4294967117 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967053 4294967117 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967054 4294967117 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967055 4294967117 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967056 4294967117 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967057 4294967117 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967058 4294967117 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967059 4294967117 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967060 4294967117 0 Shared Dependencies (Roles depending on objects). +4294967061 4294967117 0 shared security labels (empty - feature not supported) +4294967062 4294967117 0 shared object comments +4294967063 4294967117 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967064 4294967117 0 session variables (incomplete) +4294967065 4294967117 0 pg_sequences is very similar as pg_sequence. +4294967066 4294967117 0 sequences (see also information_schema.sequences) +4294967067 4294967117 0 security labels (empty - feature does not exist) +4294967068 4294967117 0 security labels (empty) +4294967069 4294967117 0 pg_rules was created for compatibility and is currently unimplemented +4294967070 4294967117 0 database roles +4294967071 4294967117 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967072 4294967117 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967073 4294967117 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967074 4294967117 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967075 4294967117 0 range types (empty - feature does not exist) +4294967076 4294967117 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967077 4294967117 0 pg_publication was created for compatibility and is currently unimplemented +4294967078 4294967117 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967079 4294967117 0 built-in functions (incomplete) +4294967080 4294967117 0 prepared transactions (empty - feature does not exist) +4294967081 4294967117 0 prepared statements +4294967082 4294967117 0 pg_policy was created for compatibility and is currently unimplemented +4294967083 4294967117 0 pg_policies was created for compatibility and is currently unimplemented +4294967084 4294967117 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967085 4294967117 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967086 4294967117 0 operators (incomplete) +4294967087 4294967117 0 opclass (empty - Operator classes not supported yet) +4294967088 4294967117 0 available namespaces +4294967089 4294967117 0 available materialized views (empty - feature does not exist) +4294967090 4294967117 0 locks held by active processes (empty - feature does not exist) +4294967091 4294967117 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967092 4294967117 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967093 4294967117 0 available languages (empty - feature does not exist) +4294967094 4294967117 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967095 4294967117 0 table inheritance hierarchy (empty - feature does not exist) +4294967096 4294967117 0 index creation statements +4294967097 4294967117 0 indexes (incomplete) +4294967098 4294967117 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967099 4294967117 0 pg_group was created for compatibility and is currently unimplemented +4294967100 4294967117 0 foreign tables (empty - feature does not exist) +4294967101 4294967117 0 foreign servers (empty - feature does not exist) +4294967102 4294967117 0 foreign data wrappers (empty - feature does not exist) +4294967103 4294967117 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967104 4294967117 0 installed extensions (empty - feature does not exist) +4294967105 4294967117 0 event triggers (empty - feature does not exist) +4294967106 4294967117 0 enum types and labels (empty - feature does not exist) +4294967107 4294967117 0 object comments +4294967108 4294967117 0 dependency relationships (incomplete) +4294967109 4294967117 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967110 4294967117 0 contains the default values that have been configured for session variables +4294967111 4294967117 0 available databases (incomplete) +4294967112 4294967117 0 contains currently active SQL cursors created with DECLARE +4294967113 4294967117 0 encoding conversions (empty - unimplemented) +4294967114 4294967117 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967115 4294967117 0 pg_config was created for compatibility and is currently unimplemented +4294967116 4294967117 0 available collations (incomplete) +4294967117 4294967117 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967118 4294967117 0 casts (empty - needs filling out) +4294967119 4294967117 0 available extensions +4294967120 4294967117 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967121 4294967117 0 role membership +4294967122 4294967117 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967123 4294967117 0 table columns (incomplete - see also information_schema.columns) +4294967124 4294967117 0 column default values +4294967125 4294967117 0 pg_amproc was created for compatibility and is currently unimplemented +4294967126 4294967117 0 pg_amop was created for compatibility and is currently unimplemented +4294967127 4294967117 0 index access methods (incomplete) +4294967128 4294967117 0 aggregated built-in functions (incomplete) +4294967130 4294967117 0 views (incomplete) +4294967131 4294967117 0 view_table_usage was created for compatibility and is currently unimplemented +4294967132 4294967117 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967133 4294967117 0 view_column_usage was created for compatibility and is currently unimplemented +4294967134 4294967117 0 grantable privileges (incomplete) +4294967135 4294967117 0 user_mappings was created for compatibility and is currently unimplemented +4294967136 4294967117 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967137 4294967117 0 user_defined_types was created for compatibility and is currently unimplemented +4294967138 4294967117 0 user_attributes was created for compatibility and is currently unimplemented +4294967139 4294967117 0 usage_privileges was created for compatibility and is currently unimplemented +4294967140 4294967117 0 udt_privileges was created for compatibility and is currently unimplemented +4294967141 4294967117 0 type privileges (incomplete; may contain excess users or roles) +4294967142 4294967117 0 triggers was created for compatibility and is currently unimplemented +4294967143 4294967117 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967144 4294967117 0 transforms was created for compatibility and is currently unimplemented +4294967145 4294967117 0 tablespaces was created for compatibility and is currently unimplemented +4294967146 4294967117 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967147 4294967117 0 tables and views +4294967148 4294967117 0 tables_extensions was created for compatibility and is currently unimplemented +4294967149 4294967117 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967150 4294967117 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967151 4294967117 0 table constraints +4294967152 4294967117 0 index metadata and statistics (incomplete) +4294967153 4294967117 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967154 4294967117 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967155 4294967117 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967156 4294967117 0 exposes the session variables. +4294967157 4294967117 0 sequences +4294967158 4294967117 0 schema privileges (incomplete; may contain excess users or roles) +4294967159 4294967117 0 database schemas (may contain schemata without permission) +4294967160 4294967117 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967161 4294967117 0 sql_sizing was created for compatibility and is currently unimplemented +4294967162 4294967117 0 sql_parts was created for compatibility and is currently unimplemented +4294967163 4294967117 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967164 4294967117 0 sql_features was created for compatibility and is currently unimplemented +4294967165 4294967117 0 built-in functions (empty - introspection not yet supported) +4294967166 4294967117 0 routine_privileges was created for compatibility and is currently unimplemented +4294967167 4294967117 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967168 4294967117 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967169 4294967117 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967170 4294967117 0 privileges granted on functions (incomplete; only contains privileges of user-defined functions) +4294967171 4294967117 0 role_column_grants was created for compatibility and is currently unimplemented +4294967172 4294967117 0 resource_groups was created for compatibility and is currently unimplemented +4294967173 4294967117 0 foreign key constraints +4294967174 4294967117 0 profiling was created for compatibility and is currently unimplemented +4294967175 4294967117 0 processlist was created for compatibility and is currently unimplemented +4294967176 4294967117 0 plugins was created for compatibility and is currently unimplemented +4294967177 4294967117 0 partitions was created for compatibility and is currently unimplemented +4294967178 4294967117 0 built-in function parameters (empty - introspection not yet supported) +4294967179 4294967117 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967180 4294967117 0 keywords was created for compatibility and is currently unimplemented +4294967181 4294967117 0 column usage by indexes and key constraints +4294967182 4294967117 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967183 4294967117 0 foreign_tables was created for compatibility and is currently unimplemented +4294967184 4294967117 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967185 4294967117 0 foreign_servers was created for compatibility and is currently unimplemented +4294967186 4294967117 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967187 4294967117 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967188 4294967117 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967189 4294967117 0 files was created for compatibility and is currently unimplemented +4294967190 4294967117 0 events was created for compatibility and is currently unimplemented +4294967191 4294967117 0 engines was created for compatibility and is currently unimplemented +4294967192 4294967117 0 roles for the current user +4294967193 4294967117 0 element_types was created for compatibility and is currently unimplemented +4294967194 4294967117 0 domains was created for compatibility and is currently unimplemented +4294967195 4294967117 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967196 4294967117 0 domain_constraints was created for compatibility and is currently unimplemented +4294967197 4294967117 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967198 4294967117 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967199 4294967117 0 columns usage by constraints +4294967200 4294967117 0 table and view columns (incomplete) +4294967201 4294967117 0 columns_extensions was created for compatibility and is currently unimplemented +4294967202 4294967117 0 columns with user defined types +4294967203 4294967117 0 column_statistics was created for compatibility and is currently unimplemented +4294967204 4294967117 0 column privilege grants (incomplete) +4294967205 4294967117 0 column_options was created for compatibility and is currently unimplemented +4294967206 4294967117 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967207 4294967117 0 column_column_usage was created for compatibility and is currently unimplemented +4294967208 4294967117 0 shows the collations available in the current database +4294967209 4294967117 0 identifies which character set the available collations are +4294967210 4294967117 0 check constraints +4294967211 4294967117 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967212 4294967117 0 character sets available in the current database +4294967213 4294967117 0 attributes was created for compatibility and is currently unimplemented +4294967214 4294967117 0 roles available to the current user +4294967215 4294967117 0 roles for which the current user has admin option +4294967217 4294967117 0 list super regions of databases visible to the current user +4294967218 4294967117 0 which entries of pg_catalog are implemented in this version of CockroachDB +4294967220 4294967117 0 node-level table listing all currently running range feeds +4294967221 4294967117 0 virtual table with default privileges +4294967222 4294967117 0 available regions for the cluster +4294967223 4294967117 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967224 4294967117 0 virtual table with table descriptors that still have data +4294967225 4294967117 0 virtual table with cross db references +4294967226 4294967117 0 virtual table with database privileges +4294967227 4294967117 0 virtual table to validate descriptors +4294967228 4294967117 0 decoded zone configurations from system.zones (KV scan) +4294967230 4294967117 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). +4294967231 4294967117 0 stats for all tables accessible by current user in current database as of 10s ago +4294967232 4294967117 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967233 4294967117 0 key spans per SQL object +4294967234 4294967117 0 indexes accessible by current user in current database (KV scan) +4294967235 4294967117 0 details for all columns accessible by current user in current database (KV scan) +4294967237 4294967117 0 session variables (RAM) +4294967238 4294967117 0 session trace accumulated so far (RAM) +4294967239 4294967117 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967240 4294967117 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967242 4294967117 0 range metadata without leaseholder details (KV join; expensive!) +4294967243 4294967117 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967244 4294967117 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967245 4294967117 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). +4294967246 4294967117 0 current values for metrics (RAM; local node only) +4294967247 4294967117 0 running sessions visible by current user (RAM; local node only) +4294967248 4294967117 0 running user transactions visible by the current user (RAM; local node only) +4294967249 4294967117 0 running queries visible by current user (RAM; local node only) +4294967251 4294967117 0 DistSQL remote flows information (RAM; local node only) +4294967252 4294967117 0 contention information (RAM; local node only) +4294967253 4294967117 0 acquired table leases (RAM; local node only) +4294967254 4294967117 0 store details and status (cluster RPC; expensive!) +4294967255 4294967117 0 node details across the entire cluster (cluster RPC; expensive!) +4294967256 4294967117 0 wrapper over system.jobs with row access control (KV scan) +4294967257 4294967117 0 decoded job metadata from crdb_internal.system_jobs (KV scan) +4294967258 4294967117 0 in-flight spans (RAM; local node only) +4294967259 4294967117 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967260 4294967117 0 key spans per table index +4294967261 4294967117 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967262 4294967117 0 cluster-wide transaction contention events. Querying this table is an +4294967263 4294967117 0 locally known edges in the gossip network (RAM; local node only) +4294967264 4294967117 0 locally known gossiped node liveness (RAM; local node only) +4294967265 4294967117 0 locally known gossiped health alerts (RAM; local node only) +4294967266 4294967117 0 locally known gossiped node details (RAM; local node only) +4294967267 4294967117 0 node liveness status, as seen by kv +4294967268 4294967117 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967269 4294967117 0 telemetry counters (RAM; local node only) +4294967270 4294967117 0 databases accessible by the current user (KV scan) +4294967271 4294967117 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967272 4294967117 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967273 4294967117 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967274 4294967117 0 CREATE statements for all user-defined functions +4294967275 4294967117 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967276 4294967117 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967277 4294967117 0 cluster settings (RAM) +4294967278 4294967117 0 running sessions visible to current user (cluster RPC; expensive!) +4294967279 4294967117 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967280 4294967117 0 running queries visible by current user (cluster RPC; expensive!) +4294967281 4294967117 0 cluster-wide locks held in lock tables. Querying this table is an +4294967283 4294967117 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967284 4294967117 0 contention information (cluster RPC; expensive!) +4294967288 4294967117 0 like system.zones but overlaid with in-txn in-memory changes +4294967289 4294967117 0 like system.namespace but overlaid with in-txn in-memory changes +4294967290 4294967117 0 like system.descriptor but overlaid with in-txn in-memory changes and including virtual objects +4294967291 4294967117 0 like system.comments but overlaid with in-txn in-memory changes and including virtual objects +4294967292 4294967117 0 built-in functions (RAM/static) +4294967293 4294967117 0 detailed identification strings (RAM, local node only) +4294967294 4294967117 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) ## pg_catalog.pg_shdescription @@ -3216,7 +3218,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 + 4294967131 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967129 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -4062,7 +4064,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 4294967002 +test pg_views 4294967000 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 0d8dd0ff8bbb..4e3285bd1f4f 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -10,21 +10,21 @@ query error NULL value in relocation array for EXPERIMENTAL_RELOCATE ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[NULL], NULL); query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder + {1} 1 statement ok ALTER TABLE t SPLIT AT VALUES (1), (10) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /10 {1} 1 -/10 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/10 {1} 1 +…/1/10 {1} 1 statement ok ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[4], 1, 12) @@ -33,26 +33,26 @@ statement ok ALTER TABLE t EXPERIMENTAL_RELOCATE LEASE VALUES (4, 1, 12) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /10 {4} 4 -/10 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/10 {4} 4 +…/1/10 {1} 1 statement ok ALTER TABLE t SPLIT AT VALUES (5,1), (5,2), (5,3) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /5/1 {4} 4 -/5/1 /5/2 {4} 4 -/5/2 /5/3 {4} 4 -/5/3 /10 {4} 4 -/10 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/5/1 {4} 4 +…/1/5/1 …/1/5/2 {4} 4 +…/1/5/2 …/1/5/3 {4} 4 +…/1/5/3 …/1/10 {4} 4 +…/1/10 {1} 1 statement ok ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 5, 1), (ARRAY[5,2,3], 5, 2), (ARRAY[4,2,1], 5, 3) @@ -64,61 +64,61 @@ statement ok ALTER TABLE t EXPERIMENTAL_RELOCATE LEASE VALUES (1, 5, 1), (5, 5, 2) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /5/1 {3,4} 3 -/5/1 /5/2 {1,2,3} 1 -/5/2 /5/3 {2,3,5} 5 -/5/3 /10 {1,2,4} 4 -/10 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/5/1 {3,4} 3 +…/1/5/1 …/1/5/2 {1,2,3} 1 +…/1/5/2 …/1/5/3 {2,3,5} 5 +…/1/5/3 …/1/10 {1,2,4} 4 +…/1/10 {1} 1 statement ok CREATE INDEX idx ON t(v, w) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder +…/ {1} 1 statement ok ALTER INDEX t@idx SPLIT AT VALUES (100,1), (100,50) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /100/1 {1} 1 -/100/1 /100/50 {1} 1 -/100/50 NULL {1} 1 +start_key end_key replicas lease_holder +…/ …/100/1 {1} 1 +…/100/1 …/100/50 {1} 1 +…/100/50 {1} 1 statement ok ALTER INDEX t@idx SPLIT AT VALUES (8), (9) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /8 {1} 1 -/8 /9 {1} 1 -/9 /100/1 {1} 1 -/100/1 /100/50 {1} 1 -/100/50 NULL {1} 1 +start_key end_key replicas lease_holder +…/ …/8 {1} 1 +…/8 …/9 {1} 1 +…/9 …/100/1 {1} 1 +…/100/1 …/100/50 {1} 1 +…/100/50 {1} 1 statement ok ALTER INDEX t@idx EXPERIMENTAL_RELOCATE VALUES (ARRAY[5], 100, 10), (ARRAY[3], 100, 11) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /8 {1} 1 -/8 /9 {1} 1 -/9 /100/1 {1} 1 -/100/1 /100/50 {3} 3 -/100/50 NULL {1} 1 +start_key end_key replicas lease_holder +…/ …/8 {1} 1 +…/8 …/9 {1} 1 +…/9 …/100/1 {1} 1 +…/100/1 …/100/50 {3} 3 +…/100/50 {1} 1 # Create and drop things to produce interesting data for crdb_internal.ranges. @@ -154,151 +154,151 @@ ALTER INDEX d.c@c_i_idx SPLIT AT VALUES (0) # Hex encode start_key, end_key so that text is UTF-8 compatible. # Non-UTF-8 characters cause git to treat this as a binary file and prevent diff from working. -query TTTTTTTTI colnames -SELECT encode(start_key, 'hex'), start_pretty, encode(end_key, 'hex'), end_pretty, database_name, table_name, index_name, replicas, crdb_internal.lease_holder(start_key) FROM crdb_internal.ranges_no_leases; ----- -encode start_pretty encode end_pretty database_name table_name index_name replicas crdb_internal.lease_holder -· /Min 04006c6976656e6573732d /System/NodeLiveness · · · {1} 1 -04006c6976656e6573732d /System/NodeLiveness 04006c6976656e6573732e /System/NodeLivenessMax · · · {1} 1 -04006c6976656e6573732e /System/NodeLivenessMax 04747364 /System/tsd · · · {1} 1 -04747364 /System/tsd 04747365 /System/"tse" · · · {1} 1 -04747365 /System/"tse" 88 /Table/0 · · · {1} 1 -88 /Table/0 8b /Table/3 · · · {1} 1 -8b /Table/3 8c /Table/4 system descriptor · {1} 1 -8c /Table/4 8d /Table/5 system users · {1} 1 -8d /Table/5 8e /Table/6 system zones · {1} 1 -8e /Table/6 8f /Table/7 system settings · {1} 1 -8f /Table/7 90 /Table/8 system descriptor_id_seq · {1} 1 -90 /Table/8 93 /Table/11 system tenants · {1} 1 -93 /Table/11 94 /Table/12 system lease · {1} 1 -94 /Table/12 95 /Table/13 system eventlog · {1} 1 -95 /Table/13 96 /Table/14 system rangelog · {1} 1 -96 /Table/14 97 /Table/15 system ui · {1} 1 -97 /Table/15 98 /Table/16 system jobs · {1} 1 -98 /Table/16 99 /Table/17 · · · {1} 1 -99 /Table/17 9a /Table/18 · · · {1} 1 -9a /Table/18 9b /Table/19 · · · {1} 1 -9b /Table/19 9c /Table/20 system web_sessions · {1} 1 -9c /Table/20 9d /Table/21 system table_statistics · {1} 1 -9d /Table/21 9e /Table/22 system locations · {1} 1 -9e /Table/22 9f /Table/23 · · · {1} 1 -9f /Table/23 a0 /Table/24 system role_members · {1} 1 -a0 /Table/24 a1 /Table/25 system comments · {1} 1 -a1 /Table/25 a2 /Table/26 system replication_constraint_stats · {1} 1 -a2 /Table/26 a3 /Table/27 system replication_critical_localities · {1} 1 -a3 /Table/27 a4 /Table/28 system replication_stats · {1} 1 -a4 /Table/28 a5 /Table/29 system reports_meta · {1} 1 -a5 /Table/29 a6 /NamespaceTable/30 · · · {1} 1 -a6 /NamespaceTable/30 a7 /NamespaceTable/Max system namespace · {1} 1 -a7 /NamespaceTable/Max a8 /Table/32 system protected_ts_meta · {1} 1 -a8 /Table/32 a9 /Table/33 system protected_ts_records · {1} 1 -a9 /Table/33 aa /Table/34 system role_options · {1} 1 -aa /Table/34 ab /Table/35 system statement_bundle_chunks · {1} 1 -ab /Table/35 ac /Table/36 system statement_diagnostics_requests · {1} 1 -ac /Table/36 ad /Table/37 system statement_diagnostics · {1} 1 -ad /Table/37 ae /Table/38 system scheduled_jobs · {1} 1 -ae /Table/38 af /Table/39 · · · {1} 1 -af /Table/39 b0 /Table/40 system sqlliveness · {1} 1 -b0 /Table/40 b1 /Table/41 system migrations · {1} 1 -b1 /Table/41 b2 /Table/42 system join_tokens · {1} 1 -b2 /Table/42 b3 /Table/43 system statement_statistics · {1} 1 -b3 /Table/43 b4 /Table/44 system transaction_statistics · {1} 1 -b4 /Table/44 b5 /Table/45 system database_role_settings · {1} 1 -b5 /Table/45 b6 /Table/46 system tenant_usage · {1} 1 -b6 /Table/46 b7 /Table/47 system sql_instances · {1} 1 -b7 /Table/47 b8 /Table/48 system span_configurations · {1} 1 -b8 /Table/48 ba /Table/50 system role_id_seq · {1} 1 -ba /Table/50 bb /Table/51 system tenant_settings · {1} 1 -bb /Table/51 bc /Table/52 system privileges · {1} 1 -bc /Table/52 bd /Table/53 system external_connections · {1} 1 -bd /Table/53 f28989 /Table/106/1/1 system job_info · {1} 1 -f28989 /Table/106/1/1 f2898d89 /Table/106/1/5/1 test t · {3,4} 3 -f2898d89 /Table/106/1/5/1 f2898d8a /Table/106/1/5/2 test t · {1,2,3} 1 -f2898d8a /Table/106/1/5/2 f2898d8b /Table/106/1/5/3 test t · {2,3,5} 5 -f2898d8b /Table/106/1/5/3 f28992 /Table/106/1/10 test t · {1,2,4} 4 -f28992 /Table/106/1/10 f28a /Table/106/2 test t · {1} 1 -f28a /Table/106/2 f28a90 /Table/106/2/8 test t idx {1} 1 -f28a90 /Table/106/2/8 f28a91 /Table/106/2/9 test t idx {1} 1 -f28a91 /Table/106/2/9 f28aec89 /Table/106/2/100/1 test t idx {1} 1 -f28aec89 /Table/106/2/100/1 f28aecba /Table/106/2/100/50 test t idx {3} 3 -f28aecba /Table/106/2/100/50 f6708988 /Table/112/1/0 test t idx {1} 1 -f6708988 /Table/112/1/0 f67189f67b /Table/113/1/123 · b · {1} 1 -f67189f67b /Table/113/1/123 f6718a /Table/113/2 d c · {1} 1 -f6718a /Table/113/2 f6718a88 /Table/113/2/0 d c c_i_idx {1} 1 -f6718a88 /Table/113/2/0 ffff /Max d c c_i_idx {1} 1 - -query TTTTTTTTI colnames -SELECT encode(start_key, 'hex'), start_pretty, encode(end_key, 'hex'), end_pretty, database_name, table_name, index_name, replicas, lease_holder FROM crdb_internal.ranges ----- -encode start_pretty encode end_pretty database_name table_name index_name replicas lease_holder -· /Min 04006c6976656e6573732d /System/NodeLiveness · · · {1} 1 -04006c6976656e6573732d /System/NodeLiveness 04006c6976656e6573732e /System/NodeLivenessMax · · · {1} 1 -04006c6976656e6573732e /System/NodeLivenessMax 04747364 /System/tsd · · · {1} 1 -04747364 /System/tsd 04747365 /System/"tse" · · · {1} 1 -04747365 /System/"tse" 88 /Table/0 · · · {1} 1 -88 /Table/0 8b /Table/3 · · · {1} 1 -8b /Table/3 8c /Table/4 system descriptor · {1} 1 -8c /Table/4 8d /Table/5 system users · {1} 1 -8d /Table/5 8e /Table/6 system zones · {1} 1 -8e /Table/6 8f /Table/7 system settings · {1} 1 -8f /Table/7 90 /Table/8 system descriptor_id_seq · {1} 1 -90 /Table/8 93 /Table/11 system tenants · {1} 1 -93 /Table/11 94 /Table/12 system lease · {1} 1 -94 /Table/12 95 /Table/13 system eventlog · {1} 1 -95 /Table/13 96 /Table/14 system rangelog · {1} 1 -96 /Table/14 97 /Table/15 system ui · {1} 1 -97 /Table/15 98 /Table/16 system jobs · {1} 1 -98 /Table/16 99 /Table/17 · · · {1} 1 -99 /Table/17 9a /Table/18 · · · {1} 1 -9a /Table/18 9b /Table/19 · · · {1} 1 -9b /Table/19 9c /Table/20 system web_sessions · {1} 1 -9c /Table/20 9d /Table/21 system table_statistics · {1} 1 -9d /Table/21 9e /Table/22 system locations · {1} 1 -9e /Table/22 9f /Table/23 · · · {1} 1 -9f /Table/23 a0 /Table/24 system role_members · {1} 1 -a0 /Table/24 a1 /Table/25 system comments · {1} 1 -a1 /Table/25 a2 /Table/26 system replication_constraint_stats · {1} 1 -a2 /Table/26 a3 /Table/27 system replication_critical_localities · {1} 1 -a3 /Table/27 a4 /Table/28 system replication_stats · {1} 1 -a4 /Table/28 a5 /Table/29 system reports_meta · {1} 1 -a5 /Table/29 a6 /NamespaceTable/30 · · · {1} 1 -a6 /NamespaceTable/30 a7 /NamespaceTable/Max system namespace · {1} 1 -a7 /NamespaceTable/Max a8 /Table/32 system protected_ts_meta · {1} 1 -a8 /Table/32 a9 /Table/33 system protected_ts_records · {1} 1 -a9 /Table/33 aa /Table/34 system role_options · {1} 1 -aa /Table/34 ab /Table/35 system statement_bundle_chunks · {1} 1 -ab /Table/35 ac /Table/36 system statement_diagnostics_requests · {1} 1 -ac /Table/36 ad /Table/37 system statement_diagnostics · {1} 1 -ad /Table/37 ae /Table/38 system scheduled_jobs · {1} 1 -ae /Table/38 af /Table/39 · · · {1} 1 -af /Table/39 b0 /Table/40 system sqlliveness · {1} 1 -b0 /Table/40 b1 /Table/41 system migrations · {1} 1 -b1 /Table/41 b2 /Table/42 system join_tokens · {1} 1 -b2 /Table/42 b3 /Table/43 system statement_statistics · {1} 1 -b3 /Table/43 b4 /Table/44 system transaction_statistics · {1} 1 -b4 /Table/44 b5 /Table/45 system database_role_settings · {1} 1 -b5 /Table/45 b6 /Table/46 system tenant_usage · {1} 1 -b6 /Table/46 b7 /Table/47 system sql_instances · {1} 1 -b7 /Table/47 b8 /Table/48 system span_configurations · {1} 1 -b8 /Table/48 ba /Table/50 system role_id_seq · {1} 1 -ba /Table/50 bb /Table/51 system tenant_settings · {1} 1 -bb /Table/51 bc /Table/52 system privileges · {1} 1 -bc /Table/52 bd /Table/53 system external_connections · {1} 1 -bd /Table/53 f28989 /Table/106/1/1 system job_info · {1} 1 -f28989 /Table/106/1/1 f2898d89 /Table/106/1/5/1 test t · {3,4} 3 -f2898d89 /Table/106/1/5/1 f2898d8a /Table/106/1/5/2 test t · {1,2,3} 1 -f2898d8a /Table/106/1/5/2 f2898d8b /Table/106/1/5/3 test t · {2,3,5} 5 -f2898d8b /Table/106/1/5/3 f28992 /Table/106/1/10 test t · {1,2,4} 4 -f28992 /Table/106/1/10 f28a /Table/106/2 test t · {1} 1 -f28a /Table/106/2 f28a90 /Table/106/2/8 test t idx {1} 1 -f28a90 /Table/106/2/8 f28a91 /Table/106/2/9 test t idx {1} 1 -f28a91 /Table/106/2/9 f28aec89 /Table/106/2/100/1 test t idx {1} 1 -f28aec89 /Table/106/2/100/1 f28aecba /Table/106/2/100/50 test t idx {3} 3 -f28aecba /Table/106/2/100/50 f6708988 /Table/112/1/0 test t idx {1} 1 -f6708988 /Table/112/1/0 f67189f67b /Table/113/1/123 · b · {1} 1 -f67189f67b /Table/113/1/123 f6718a /Table/113/2 d c · {1} 1 -f6718a /Table/113/2 f6718a88 /Table/113/2/0 d c c_i_idx {1} 1 -f6718a88 /Table/113/2/0 ffff /Max d c c_i_idx {1} 1 +query TTTTTI colnames +SELECT encode(start_key, 'hex'), start_pretty, encode(end_key, 'hex'), end_pretty, replicas, crdb_internal.lease_holder(start_key) FROM crdb_internal.ranges_no_leases; +---- +encode start_pretty encode end_pretty replicas crdb_internal.lease_holder +· /Min 04006c6976656e6573732d /System/NodeLiveness {1} 1 +04006c6976656e6573732d /System/NodeLiveness 04006c6976656e6573732e /System/NodeLivenessMax {1} 1 +04006c6976656e6573732e /System/NodeLivenessMax 04747364 /System/tsd {1} 1 +04747364 /System/tsd 04747365 /System/"tse" {1} 1 +04747365 /System/"tse" 88 /Table/0 {1} 1 +88 /Table/0 8b /Table/3 {1} 1 +8b /Table/3 8c /Table/4 {1} 1 +8c /Table/4 8d /Table/5 {1} 1 +8d /Table/5 8e /Table/6 {1} 1 +8e /Table/6 8f /Table/7 {1} 1 +8f /Table/7 90 /Table/8 {1} 1 +90 /Table/8 93 /Table/11 {1} 1 +93 /Table/11 94 /Table/12 {1} 1 +94 /Table/12 95 /Table/13 {1} 1 +95 /Table/13 96 /Table/14 {1} 1 +96 /Table/14 97 /Table/15 {1} 1 +97 /Table/15 98 /Table/16 {1} 1 +98 /Table/16 99 /Table/17 {1} 1 +99 /Table/17 9a /Table/18 {1} 1 +9a /Table/18 9b /Table/19 {1} 1 +9b /Table/19 9c /Table/20 {1} 1 +9c /Table/20 9d /Table/21 {1} 1 +9d /Table/21 9e /Table/22 {1} 1 +9e /Table/22 9f /Table/23 {1} 1 +9f /Table/23 a0 /Table/24 {1} 1 +a0 /Table/24 a1 /Table/25 {1} 1 +a1 /Table/25 a2 /Table/26 {1} 1 +a2 /Table/26 a3 /Table/27 {1} 1 +a3 /Table/27 a4 /Table/28 {1} 1 +a4 /Table/28 a5 /Table/29 {1} 1 +a5 /Table/29 a6 /NamespaceTable/30 {1} 1 +a6 /NamespaceTable/30 a7 /NamespaceTable/Max {1} 1 +a7 /NamespaceTable/Max a8 /Table/32 {1} 1 +a8 /Table/32 a9 /Table/33 {1} 1 +a9 /Table/33 aa /Table/34 {1} 1 +aa /Table/34 ab /Table/35 {1} 1 +ab /Table/35 ac /Table/36 {1} 1 +ac /Table/36 ad /Table/37 {1} 1 +ad /Table/37 ae /Table/38 {1} 1 +ae /Table/38 af /Table/39 {1} 1 +af /Table/39 b0 /Table/40 {1} 1 +b0 /Table/40 b1 /Table/41 {1} 1 +b1 /Table/41 b2 /Table/42 {1} 1 +b2 /Table/42 b3 /Table/43 {1} 1 +b3 /Table/43 b4 /Table/44 {1} 1 +b4 /Table/44 b5 /Table/45 {1} 1 +b5 /Table/45 b6 /Table/46 {1} 1 +b6 /Table/46 b7 /Table/47 {1} 1 +b7 /Table/47 b8 /Table/48 {1} 1 +b8 /Table/48 ba /Table/50 {1} 1 +ba /Table/50 bb /Table/51 {1} 1 +bb /Table/51 bc /Table/52 {1} 1 +bc /Table/52 bd /Table/53 {1} 1 +bd /Table/53 f28989 /Table/106/1/1 {1} 1 +f28989 /Table/106/1/1 f2898d89 /Table/106/1/5/1 {3,4} 3 +f2898d89 /Table/106/1/5/1 f2898d8a /Table/106/1/5/2 {1,2,3} 1 +f2898d8a /Table/106/1/5/2 f2898d8b /Table/106/1/5/3 {2,3,5} 5 +f2898d8b /Table/106/1/5/3 f28992 /Table/106/1/10 {1,2,4} 4 +f28992 /Table/106/1/10 f28a /Table/106/2 {1} 1 +f28a /Table/106/2 f28a90 /Table/106/2/8 {1} 1 +f28a90 /Table/106/2/8 f28a91 /Table/106/2/9 {1} 1 +f28a91 /Table/106/2/9 f28aec89 /Table/106/2/100/1 {1} 1 +f28aec89 /Table/106/2/100/1 f28aecba /Table/106/2/100/50 {3} 3 +f28aecba /Table/106/2/100/50 f6708988 /Table/112/1/0 {1} 1 +f6708988 /Table/112/1/0 f67189f67b /Table/113/1/123 {1} 1 +f67189f67b /Table/113/1/123 f6718a /Table/113/2 {1} 1 +f6718a /Table/113/2 f6718a88 /Table/113/2/0 {1} 1 +f6718a88 /Table/113/2/0 ffff /Max {1} 1 + +query TTTTTI colnames +SELECT encode(start_key, 'hex'), start_pretty, encode(end_key, 'hex'), end_pretty, replicas, lease_holder FROM crdb_internal.ranges +---- +encode start_pretty encode end_pretty replicas lease_holder +· /Min 04006c6976656e6573732d /System/NodeLiveness {1} 1 +04006c6976656e6573732d /System/NodeLiveness 04006c6976656e6573732e /System/NodeLivenessMax {1} 1 +04006c6976656e6573732e /System/NodeLivenessMax 04747364 /System/tsd {1} 1 +04747364 /System/tsd 04747365 /System/"tse" {1} 1 +04747365 /System/"tse" 88 /Table/0 {1} 1 +88 /Table/0 8b /Table/3 {1} 1 +8b /Table/3 8c /Table/4 {1} 1 +8c /Table/4 8d /Table/5 {1} 1 +8d /Table/5 8e /Table/6 {1} 1 +8e /Table/6 8f /Table/7 {1} 1 +8f /Table/7 90 /Table/8 {1} 1 +90 /Table/8 93 /Table/11 {1} 1 +93 /Table/11 94 /Table/12 {1} 1 +94 /Table/12 95 /Table/13 {1} 1 +95 /Table/13 96 /Table/14 {1} 1 +96 /Table/14 97 /Table/15 {1} 1 +97 /Table/15 98 /Table/16 {1} 1 +98 /Table/16 99 /Table/17 {1} 1 +99 /Table/17 9a /Table/18 {1} 1 +9a /Table/18 9b /Table/19 {1} 1 +9b /Table/19 9c /Table/20 {1} 1 +9c /Table/20 9d /Table/21 {1} 1 +9d /Table/21 9e /Table/22 {1} 1 +9e /Table/22 9f /Table/23 {1} 1 +9f /Table/23 a0 /Table/24 {1} 1 +a0 /Table/24 a1 /Table/25 {1} 1 +a1 /Table/25 a2 /Table/26 {1} 1 +a2 /Table/26 a3 /Table/27 {1} 1 +a3 /Table/27 a4 /Table/28 {1} 1 +a4 /Table/28 a5 /Table/29 {1} 1 +a5 /Table/29 a6 /NamespaceTable/30 {1} 1 +a6 /NamespaceTable/30 a7 /NamespaceTable/Max {1} 1 +a7 /NamespaceTable/Max a8 /Table/32 {1} 1 +a8 /Table/32 a9 /Table/33 {1} 1 +a9 /Table/33 aa /Table/34 {1} 1 +aa /Table/34 ab /Table/35 {1} 1 +ab /Table/35 ac /Table/36 {1} 1 +ac /Table/36 ad /Table/37 {1} 1 +ad /Table/37 ae /Table/38 {1} 1 +ae /Table/38 af /Table/39 {1} 1 +af /Table/39 b0 /Table/40 {1} 1 +b0 /Table/40 b1 /Table/41 {1} 1 +b1 /Table/41 b2 /Table/42 {1} 1 +b2 /Table/42 b3 /Table/43 {1} 1 +b3 /Table/43 b4 /Table/44 {1} 1 +b4 /Table/44 b5 /Table/45 {1} 1 +b5 /Table/45 b6 /Table/46 {1} 1 +b6 /Table/46 b7 /Table/47 {1} 1 +b7 /Table/47 b8 /Table/48 {1} 1 +b8 /Table/48 ba /Table/50 {1} 1 +ba /Table/50 bb /Table/51 {1} 1 +bb /Table/51 bc /Table/52 {1} 1 +bc /Table/52 bd /Table/53 {1} 1 +bd /Table/53 f28989 /Table/106/1/1 {1} 1 +f28989 /Table/106/1/1 f2898d89 /Table/106/1/5/1 {3,4} 3 +f2898d89 /Table/106/1/5/1 f2898d8a /Table/106/1/5/2 {1,2,3} 1 +f2898d8a /Table/106/1/5/2 f2898d8b /Table/106/1/5/3 {2,3,5} 5 +f2898d8b /Table/106/1/5/3 f28992 /Table/106/1/10 {1,2,4} 4 +f28992 /Table/106/1/10 f28a /Table/106/2 {1} 1 +f28a /Table/106/2 f28a90 /Table/106/2/8 {1} 1 +f28a90 /Table/106/2/8 f28a91 /Table/106/2/9 {1} 1 +f28a91 /Table/106/2/9 f28aec89 /Table/106/2/100/1 {1} 1 +f28aec89 /Table/106/2/100/1 f28aecba /Table/106/2/100/50 {3} 3 +f28aecba /Table/106/2/100/50 f6708988 /Table/112/1/0 {1} 1 +f6708988 /Table/112/1/0 f67189f67b /Table/113/1/123 {1} 1 +f67189f67b /Table/113/1/123 f6718a /Table/113/2 {1} 1 +f6718a /Table/113/2 f6718a88 /Table/113/2/0 {1} 1 +f6718a88 /Table/113/2/0 ffff /Max {1} 1 # Due to asynchronous splitting of ranges, we cannot guarantee the output # of the show ranges from database command. The test below just ensures that @@ -317,12 +317,12 @@ SHOW RANGES FROM DATABASE "show ranges" query TT SELECT start_key, end_key FROM [SHOW RANGES FROM TABLE "show ranges".t] ---- -NULL NULL + query TT SELECT start_key, end_key FROM [SHOW RANGES FROM INDEX "show ranges".t@t_pkey] ---- -NULL NULL + statement ok CREATE DATABASE """" @@ -336,12 +336,12 @@ SHOW RANGES FROM DATABASE """" query TT SELECT start_key, end_key FROM [SHOW RANGES FROM TABLE """".t] ---- -NULL NULL + query TT SELECT start_key, end_key FROM [SHOW RANGES FROM INDEX """".t@t_pkey] ---- -NULL NULL + query T SELECT feature_name FROM crdb_internal.feature_usage WHERE feature_name='sql.show.ranges' AND usage_count > 0 @@ -359,7 +359,7 @@ ALTER TABLE simple_range_for_row SPLIT AT VALUES (1), (2) query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE simple_range_for_row FOR ROW (1)] ---- -/1 /2 +…/1 statement ok CREATE TABLE range_for_row(x INT, y INT, z INT, w INT, PRIMARY KEY (x, y), INDEX i (z, w)) @@ -373,32 +373,32 @@ ALTER INDEX range_for_row@i SPLIT AT VALUES (3, 4), (3, 5) query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 2)] ---- -/1/2 /1/3 +…/1/2 query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 3)] ---- -/1/3 NULL +…/1/3 query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 1)] ---- -NULL /1/2 + query TT SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (1, 2, 1, 2)] ---- -NULL /3/4 +…/1/3 query TT SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (3, 4, 1, 2)] ---- -/3/4 /3/5 +…/3/4 query TT SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (3, 5, 1, 2)] ---- -/3/5 NULL +…/3/5 … statement ok CREATE TABLE range_for_row_string(x STRING PRIMARY KEY) @@ -409,7 +409,7 @@ ALTER TABLE range_for_row_string SPLIT AT VALUES ('hello') query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row_string FOR ROW ('he')] ---- -NULL /"hello" + statement ok CREATE TABLE range_for_row_decimal(x DECIMAL PRIMARY KEY) @@ -420,7 +420,7 @@ ALTER TABLE range_for_row_decimal SPLIT AT VALUES (1), (2) query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row_decimal FOR ROW (1)] ---- -/1 /2 +…/1 statement ok CREATE TABLE range_for_row_nulls(x INT PRIMARY KEY, y INT, INDEX i (y)) @@ -431,7 +431,7 @@ ALTER INDEX range_for_row_nulls@i SPLIT AT VALUES (NULL) query TT SELECT start_key, end_key from [SHOW RANGE FROM INDEX range_for_row_nulls@i FOR ROW (NULL, 1)] ---- -/NULL NULL +…/NULL … subtest end @@ -521,12 +521,12 @@ ALTER TABLE t63646 SPLIT AT VALUES ('a'), ('b') query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE t63646 FOR ROW ('a')] ---- -/"@" /"\x80" +…/"@" query TT SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE t63646 FOR ROW ('b')] ---- -/"\x80" NULL +…/"\x80" … # Test permissions for showing ranges with ZONECONFIG privilege @@ -537,10 +537,10 @@ GRANT SELECT ON TABLE t to testuser user testuser -statement error only users with the ZONECONFIG privilege or the admin role can use SHOW RANGES on t +statement error only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases SHOW RANGES FROM TABLE t -statement error only users with the ZONECONFIG privilege or the admin role can use SHOW RANGES on t +statement error only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases SHOW RANGES FROM INDEX t@idx user root @@ -572,44 +572,6 @@ SELECT crdb_internal.range_stats(k) user root -subtest show_ranges_database_columns - -statement ok -CREATE DATABASE db - -query TTTT_TTTTTT colnames -SHOW RANGES FROM DATABASE db ----- -table_name start_key end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas - -subtest end - -subtest show_ranges_table_columns - -statement ok -CREATE TABLE tbl(); - -query TTTI_ITTTTT colnames -SHOW RANGES FROM TABLE tbl; ----- -start_key end_key index_name range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas -NULL NULL · 90 _ 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} - -subtest end - -subtest show_ranges_index_columns - -statement ok -CREATE TABLE tbl_with_idx(i INT, INDEX idx (i)); - -query TTTI_ITTTTT colnames -SHOW RANGES FROM INDEX tbl_with_idx@idx; ----- -start_key end_key index_name range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas -NULL NULL · 90 _ 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} - -subtest end - subtest show_range_table_for_row_columns statement ok @@ -618,8 +580,8 @@ CREATE TABLE tbl_for_row(i INT PRIMARY KEY); query TTIITTTTT colnames SHOW RANGE FROM TABLE tbl_for_row FOR ROW (0) ---- -start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas -NULL NULL 90 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} +start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas + … 90 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} subtest end @@ -631,7 +593,7 @@ CREATE TABLE tbl_with_idx_for_row(i INT, INDEX idx (i)); query TTIITTTTT colnames SHOW RANGE FROM INDEX tbl_with_idx_for_row@idx FOR ROW (NULL, 0) ---- -start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas -NULL NULL 90 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} +start_key end_key range_id lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas + … 90 1 region=test,dc=dc1 {1} {"region=test,dc=dc1"} {1} {} subtest end diff --git a/pkg/sql/logictest/testdata/logic_test/show_ranges b/pkg/sql/logictest/testdata/logic_test/show_ranges new file mode 100644 index 000000000000..b8f5d2808cc1 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/show_ranges @@ -0,0 +1,448 @@ +# LogicTest: local + +subtest syntax_equivalence + +# SHOW RANGES is an alias for SHOW RANGES FROM CURRENT_CATALOG. +query B +SELECT (SELECT * FROM [SHOW RANGES WITH EXPLAIN]) = (SELECT * FROM [SHOW RANGES FROM CURRENT_CATALOG WITH EXPLAIN]) +---- +true + +# SHOW RANGES FROM CURRENT_CATALOG is an alias for SHOW RANGES FROM DATABASE . +query B +SELECT (SELECT * FROM [SHOW RANGES FROM CURRENT_CATALOG WITH EXPLAIN]) = (SELECT * FROM [SHOW RANGES FROM DATABASE test WITH EXPLAIN]) +---- +true + +subtest incompatible_options + +statement error cannot use WITH TABLES with SHOW RANGES FROM +SHOW RANGES FROM TABLE t WITH TABLES + +statement error cannot use WITH INDEXES with SHOW RANGES FROM INDEX +SHOW RANGES FROM INDEX t@idx WITH INDEXES + +statement error cannot use WITH TABLES with SHOW RANGES FROM +SHOW RANGES FROM INDEX t@idx WITH TABLES + +subtest show_cluster_ranges + +# SHOW CLUSTER RANGES: all ranges across all databases. +# First get all column names, to assert the schema. +query TTITTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES] LIMIT 0 +---- +start_key end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +# Then also show the range keys. +query TTI colnames +SELECT start_key, end_key, range_id FROM [SHOW CLUSTER RANGES] +ORDER BY range_id LIMIT 10 +---- +start_key end_key range_id +/Min /System/NodeLiveness 1 +/System/NodeLiveness /System/NodeLivenessMax 2 +/System/NodeLivenessMax /System/tsd 3 +/System/tsd /System/"tse" 4 +/System/"tse" /Table/0 5 +/Table/0 /Table/3 6 +/Table/3 /Table/4 7 +/Table/4 /Table/5 8 +/Table/5 /Table/6 9 +/Table/6 /Table/7 10 + +# Ditto, verbose form. +query TTIFITTTTTTTI colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH DETAILS] LIMIT 0 +---- +start_key end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Ditto, with keys. +query TTTTITTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +# Ditto, verbose + keys. +query TTTTIFITTTTTTTI colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH DETAILS, KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTII colnames +SELECT start_key, to_hex(raw_start_key), end_key, to_hex(raw_end_key), range_id, lease_holder FROM [SHOW CLUSTER RANGES WITH DETAILS, KEYS] +ORDER BY range_id LIMIT 10 +---- +start_key to_hex end_key to_hex range_id lease_holder +/Min · /System/NodeLiveness 04006c6976656e6573732d 1 1 +/System/NodeLiveness 04006c6976656e6573732d /System/NodeLivenessMax 04006c6976656e6573732e 2 1 +/System/NodeLivenessMax 04006c6976656e6573732e /System/tsd 04747364 3 1 +/System/tsd 04747364 /System/"tse" 04747365 4 1 +/System/"tse" 04747365 /Table/0 88 5 1 +/Table/0 88 /Table/3 8b 6 1 +/Table/3 8b /Table/4 8c 7 1 +/Table/4 8c /Table/5 8d 8 1 +/Table/5 8d /Table/6 8e 9 1 +/Table/6 8e /Table/7 8f 10 1 + + +subtest show_cluster_ranges/with_tables + +# Assert the schema. +query TTITTTITTTTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH TABLES] LIMIT 0 +---- +start_key end_key range_id database_name schema_name table_name table_id table_start_key table_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTITTTITTFITTTTTTTI colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH DETAILS, TABLES] LIMIT 0 +---- +start_key end_key range_id database_name schema_name table_name table_id table_start_key table_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTITTTITTTTTTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH KEYS, TABLES] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id database_name schema_name table_name table_id table_start_key table_end_key raw_table_start_key raw_table_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +# Show some rows. +# This also demonstrates how the table information remains NULL if there's no table contained. +query TTITTTITTTT colnames +SELECT start_key, end_key, range_id, database_name, schema_name, table_name, table_id, table_start_key, to_hex(raw_table_start_key), table_end_key, to_hex(raw_table_end_key) +FROM [SHOW CLUSTER RANGES WITH TABLES, KEYS] +WHERE table_name LIKE 'repl%' + OR start_key LIKE '/System%' +ORDER BY range_id +---- +start_key end_key range_id database_name schema_name table_name table_id table_start_key to_hex table_end_key to_hex +/System/NodeLiveness /System/NodeLivenessMax 2 NULL NULL NULL NULL NULL NULL NULL NULL +/System/NodeLivenessMax /System/tsd 3 NULL NULL NULL NULL NULL NULL NULL NULL +/System/tsd /System/"tse" 4 NULL NULL NULL NULL NULL NULL NULL NULL +/System/"tse" /Table/0 5 NULL NULL NULL NULL NULL NULL NULL NULL +/Table/25 /Table/26 27 system public replication_constraint_stats 25 /Table/25 a1 /Table/26 a2 +/Table/26 /Table/27 28 system public replication_critical_localities 26 /Table/26 a2 /Table/27 a3 +/Table/27 /Table/28 29 system public replication_stats 27 /Table/27 a3 /Table/28 a4 + +subtest show_cluster_ranges/with_indexes + +# Assert the schema. +query TTITTTITITTTTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH INDEXES] LIMIT 0 +---- +start_key end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTITTTITITTFITTTTTTTI colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH DETAILS, INDEXES] LIMIT 0 +---- +start_key end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTITTTITITTTTTTTTTT colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH KEYS, INDEXES] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTTITITTTTFITTTTTTTI colnames +SELECT * FROM [SHOW CLUSTER RANGES WITH DETAILS, KEYS, INDEXES] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Show some rows. +# This also demonstrates how the index information remains NULL if there's no index contained. +query TTITTTITITTTT colnames +SELECT start_key, end_key, range_id, database_name, schema_name, table_name, table_id, index_name, index_id, index_start_key, to_hex(raw_index_start_key), index_end_key, to_hex(raw_index_end_key) +FROM [SHOW CLUSTER RANGES WITH INDEXES, KEYS] +WHERE table_name LIKE 'repl%' + OR start_key LIKE '/System%' +ORDER BY range_id +---- +start_key end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key to_hex index_end_key to_hex +/System/NodeLiveness /System/NodeLivenessMax 2 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +/System/NodeLivenessMax /System/tsd 3 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +/System/tsd /System/"tse" 4 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +/System/"tse" /Table/0 5 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +/Table/25 /Table/26 27 system public replication_constraint_stats 25 primary 1 /Table/25/1 a189 /Table/25/2 a18a +/Table/26 /Table/27 28 system public replication_critical_localities 26 primary 1 /Table/26/1 a289 /Table/26/2 a28a +/Table/27 /Table/28 29 system public replication_stats 27 primary 1 /Table/27/1 a389 /Table/27/2 a38a + + +subtest show_ranges_from_database + +# SHOW RANGES FROM DATABASE: starts empty because the test db does not contain tables yet. +# We use the "SHOW RANGES" shorcut syntax for convenience; we've already asserted above +# that it is equivalent to the full form. +query TTITTTTTT colnames +SHOW RANGES +---- +start_key end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTTTTT colnames +SHOW RANGES WITH KEYS +---- +start_key end_key raw_start_key raw_end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTIFITTTTTTTI colnames +SHOW RANGES WITH DETAILS +---- +start_key end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Add some tables. This will force at least one range to pop up. +# also split them to make the test more interesting. +statement ok +CREATE TABLE t(x INT PRIMARY KEY, y INT); ALTER TABLE t SPLIT AT VALUES (10); +CREATE INDEX idx ON t(y); ALTER INDEX t@idx SPLIT AT VALUES (20), (30); +CREATE TABLE u(x INT PRIMARY KEY); ALTER TABLE u SPLIT AT VALUES (42); + +# List ranges again. Expect data this time. +query TTIT colnames +SELECT start_key, end_key, range_id, split_enforced_until FROM [SHOW RANGES] +ORDER BY range_id +---- +start_key end_key range_id split_enforced_until +/Table/53 /Table/106/1/10 54 NULL +/Table/106/1/10 /Table/106/2/20 55 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/106/2/20 /Table/106/2/30 56 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/106/2/30 /Table/107/1/42 57 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/107/1/42 /Max 58 2262-04-11 23:47:16.854776 +0000 +0000 + +# Ditto, verbose form. +query TTIIT colnames +SELECT start_key, end_key, range_id, lease_holder, split_enforced_until FROM [SHOW RANGES WITH DETAILS] +ORDER BY range_id +---- +start_key end_key range_id lease_holder split_enforced_until +/Table/53 /Table/106/1/10 54 1 NULL +/Table/106/1/10 /Table/106/2/20 55 1 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/106/2/20 /Table/106/2/30 56 1 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/106/2/30 /Table/107/1/42 57 1 2262-04-11 23:47:16.854776 +0000 +0000 +/Table/107/1/42 /Max 58 1 2262-04-11 23:47:16.854776 +0000 +0000 + +# Show that the new tables shows up in the full range list. +query TTITTTITITT colnames +SELECT start_key, end_key, range_id, database_name, schema_name, table_name, table_id, index_name, index_id, index_start_key, index_end_key +FROM [SHOW CLUSTER RANGES WITH INDEXES] +WHERE (database_name = 'system' AND table_name LIKE 'repl%') + OR (database_name = 'test') +ORDER BY range_id, table_id, index_id +---- +start_key end_key range_id database_name schema_name table_name table_id index_name index_id index_start_key index_end_key +/Table/25 /Table/26 27 system public replication_constraint_stats 25 primary 1 /Table/25/1 /Table/25/2 +/Table/26 /Table/27 28 system public replication_critical_localities 26 primary 1 /Table/26/1 /Table/26/2 +/Table/27 /Table/28 29 system public replication_stats 27 primary 1 /Table/27/1 /Table/27/2 +/Table/53 /Table/106/1/10 54 test public t 106 t_pkey 1 /Table/106/1 /Table/106/2 +/Table/106/1/10 /Table/106/2/20 55 test public t 106 t_pkey 1 /Table/106/1 /Table/106/2 +/Table/106/1/10 /Table/106/2/20 55 test public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/20 /Table/106/2/30 56 test public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/30 /Table/107/1/42 57 test public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/30 /Table/107/1/42 57 test public u 107 u_pkey 1 /Table/107/1 /Table/107/2 +/Table/107/1/42 /Max 58 test public u 107 u_pkey 1 /Table/107/1 /Table/107/2 + +subtest show_ranges_from_database/with_tables + +# Assert the schema. +query TTITTITTTTTTTT colnames +SELECT * FROM [SHOW RANGES WITH TABLES] LIMIT 0 +---- +start_key end_key range_id schema_name table_name table_id table_start_key table_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTITTTTTTTTTT colnames +SELECT * FROM [SHOW RANGES WITH TABLES, KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id schema_name table_name table_id table_start_key table_end_key raw_table_start_key raw_table_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTITTITTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES WITH DETAILS, TABLES] LIMIT 0 +---- +start_key end_key range_id schema_name table_name table_id table_start_key table_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTITTITTTTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES WITH DETAILS, KEYS, TABLES] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id schema_name table_name table_id table_start_key table_end_key raw_table_start_key raw_table_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Show the data. +query TTITTITT colnames +SELECT start_key, end_key, range_id, schema_name, table_name, table_id, table_start_key, table_end_key +FROM [SHOW RANGES WITH TABLES] +ORDER BY range_id +---- +start_key end_key range_id schema_name table_name table_id table_start_key table_end_key +/Table/53 /Table/106/1/10 54 public t 106 /Table/106 /Table/107 +/Table/106/1/10 /Table/106/2/20 55 public t 106 /Table/106 /Table/107 +/Table/106/2/20 /Table/106/2/30 56 public t 106 /Table/106 /Table/107 +/Table/106/2/30 /Table/107/1/42 57 public t 106 /Table/106 /Table/107 +/Table/106/2/30 /Table/107/1/42 57 public u 107 /Table/107 /Table/108 +/Table/107/1/42 /Max 58 public u 107 /Table/107 /Table/108 + + +subtest show_ranges_from_database/with_indexes + +# Assert the schema. +query TTITTITITTTTTTTT colnames +SELECT * FROM [SHOW RANGES WITH INDEXES] LIMIT 0 +---- +start_key end_key range_id schema_name table_name table_id index_name index_id index_start_key index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTITITTTTTTTTTT colnames +SELECT * FROM [SHOW RANGES WITH INDEXES, KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id schema_name table_name table_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTITTITITTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES WITH DETAILS, INDEXES] LIMIT 0 +---- +start_key end_key range_id schema_name table_name table_id index_name index_id index_start_key index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTITTITITTTTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES WITH DETAILS, KEYS, INDEXES] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id schema_name table_name table_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Show some rows. +# This also demonstrates how the index information remains NULL if there's no index contained. +query TTITTITITT colnames +SELECT start_key, end_key, range_id, schema_name, table_name, table_id, index_name, index_id, index_start_key, index_end_key +FROM [SHOW RANGES WITH INDEXES] +ORDER BY range_id, table_id, index_id +---- +start_key end_key range_id schema_name table_name table_id index_name index_id index_start_key index_end_key +/Table/53 /Table/106/1/10 54 public t 106 t_pkey 1 /Table/106/1 /Table/106/2 +/Table/106/1/10 /Table/106/2/20 55 public t 106 t_pkey 1 /Table/106/1 /Table/106/2 +/Table/106/1/10 /Table/106/2/20 55 public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/20 /Table/106/2/30 56 public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/30 /Table/107/1/42 57 public t 106 idx 2 /Table/106/2 /Table/106/3 +/Table/106/2/30 /Table/107/1/42 57 public u 107 u_pkey 1 /Table/107/1 /Table/107/2 +/Table/107/1/42 /Max 58 public u 107 u_pkey 1 /Table/107/1 /Table/107/2 + + +subtest show_ranges_from_table + +# Assert the schema. +query TTITTTTTT colnames +SELECT * FROM [SHOW RANGES FROM TABLE t] LIMIT 0 +---- +start_key end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTTTTT colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTIFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH DETAILS] LIMIT 0 +---- +start_key end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTIFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH DETAILS,KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Now also look at the output. +query TTIT colnames +SELECT start_key, end_key, range_id, split_enforced_until FROM [SHOW RANGES FROM TABLE t] +ORDER BY range_id +---- +start_key end_key range_id split_enforced_until + …/1/10 54 NULL +…/1/10 …/2/20 55 2262-04-11 23:47:16.854776 +0000 +0000 +…/2/20 …/2/30 56 2262-04-11 23:47:16.854776 +0000 +0000 +…/2/30 57 2262-04-11 23:47:16.854776 +0000 +0000 + +# Ditto, verbose form. +query TTIIT colnames +SELECT start_key, end_key, range_id, lease_holder, split_enforced_until FROM [SHOW RANGES FROM TABLE t WITH DETAILS] +ORDER BY range_id +---- +start_key end_key range_id lease_holder split_enforced_until + …/1/10 54 1 NULL +…/1/10 …/2/20 55 1 2262-04-11 23:47:16.854776 +0000 +0000 +…/2/20 …/2/30 56 1 2262-04-11 23:47:16.854776 +0000 +0000 +…/2/30 57 1 2262-04-11 23:47:16.854776 +0000 +0000 + +# Let's inspect the other table for comparison. +query TTIT colnames +SELECT start_key, end_key, range_id, split_enforced_until FROM [SHOW RANGES FROM TABLE u] +ORDER BY range_id +---- +start_key end_key range_id split_enforced_until + …/1/42 57 2262-04-11 23:47:16.854776 +0000 +0000 +…/1/42 58 2262-04-11 23:47:16.854776 +0000 +0000 + + + +subtest show_ranges_from_table/with_indexes + +# Assert the schema. +query TTITITTTTTTTT colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH INDEXES] LIMIT 0 +---- +start_key end_key range_id index_name index_id index_start_key index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITITTTTTTTTTT colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH INDEXES,KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTITITTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH INDEXES, DETAILS] LIMIT 0 +---- +start_key end_key range_id index_name index_id index_start_key index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTITITTTTFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM TABLE t WITH INDEXES, KEYS, DETAILS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id index_name index_id index_start_key index_end_key raw_index_start_key raw_index_end_key range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Now also look at the output. +query TTITITT colnames +SELECT start_key, end_key, range_id, index_name, index_id, index_start_key, index_end_key FROM [SHOW RANGES FROM TABLE t WITH INDEXES] +ORDER BY range_id, index_id +---- +start_key end_key range_id index_name index_id index_start_key index_end_key + …/1/10 54 t_pkey 1 …/1 …/2 +…/1/10 …/2/20 55 t_pkey 1 …/1 …/2 +…/1/10 …/2/20 55 idx 2 …/2 …/3 +…/2/20 …/2/30 56 idx 2 …/2 …/3 +…/2/30 57 idx 2 …/2 …/3 + + + +subtest show_ranges_from_index + +# Assert the schema. +query TTITTTTTT colnames +SELECT * FROM [SHOW RANGES FROM INDEX t@idx] LIMIT 0 +---- +start_key end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTTTITTTTTT colnames +SELECT * FROM [SHOW RANGES FROM INDEX t@idx WITH KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until + +query TTIFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] LIMIT 0 +---- +start_key end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +query TTTTIFITTTTTTTI colnames +SELECT * FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS, KEYS] LIMIT 0 +---- +start_key end_key raw_start_key raw_end_key range_id range_size_mb lease_holder lease_holder_locality replicas replica_localities voting_replicas non_voting_replicas learner_replicas split_enforced_until range_size + +# Now also look at the output. +query TTIT colnames +SELECT start_key, end_key, range_id, split_enforced_until FROM [SHOW RANGES FROM INDEX t@idx] +---- +start_key end_key range_id split_enforced_until + …/20 55 2262-04-11 23:47:16.854776 +0000 +0000 +…/20 …/30 56 2262-04-11 23:47:16.854776 +0000 +0000 +…/30 57 2262-04-11 23:47:16.854776 +0000 +0000 + +# Ditto, verbose form. +query TTIIT colnames +SELECT start_key, end_key, range_id, lease_holder, split_enforced_until FROM [SHOW RANGES FROM INDEX t@idx WITH DETAILS] +---- +start_key end_key range_id lease_holder split_enforced_until + …/20 55 1 2262-04-11 23:47:16.854776 +0000 +0000 +…/20 …/30 56 1 2262-04-11 23:47:16.854776 +0000 +0000 +…/30 57 1 2262-04-11 23:47:16.854776 +0000 +0000 diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 07757fb88f0f..02e78a4226f3 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -451,19 +451,25 @@ testuser · {} query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE system.descriptor] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM CURRENT_CATALOG WITH DETAILS] ---- start_key end_key replicas lease_holder -NULL NULL {1} 1 +/Table/53 /Max {1} 1 + +query TTTI colnames +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE system.descriptor WITH DETAILS] +---- +start_key end_key replicas lease_holder +…/ …/ {1} 1 statement ok CREATE INDEX ix ON foo(x) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX foo@ix] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX foo@ix WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder +…/ {1} 1 query TTTTTTT colnames SELECT * FROM [SHOW TRACE FOR SESSION] LIMIT 0 diff --git a/pkg/sql/logictest/testdata/logic_test/split_at b/pkg/sql/logictest/testdata/logic_test/split_at index abaea65db4b6..a0b297fa6822 100644 --- a/pkg/sql/logictest/testdata/logic_test/split_at +++ b/pkg/sql/logictest/testdata/logic_test/split_at @@ -6,10 +6,10 @@ statement ok CREATE TABLE t (a INT PRIMARY KEY) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder + {1} 1 query TTT colnames ALTER TABLE t SPLIT AT VALUES (1), (10) @@ -19,12 +19,12 @@ key pretty split_enforced_until [242 137 146] /10 2262-04-11 23:47:16.854776 +0000 +0000 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE t WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /10 {1} 1 -/10 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/10 {1} 1 +…/1/10 {1} 1 statement ok DROP TABLE t diff --git a/pkg/sql/logictest/testdata/logic_test/sql_keys b/pkg/sql/logictest/testdata/logic_test/sql_keys index e9a39db25ed1..ba153b214dc3 100644 --- a/pkg/sql/logictest/testdata/logic_test/sql_keys +++ b/pkg/sql/logictest/testdata/logic_test/sql_keys @@ -10,7 +10,7 @@ ALTER TABLE t SPLIT AT VALUES (0) # Get the range that contains this table. let $rangeid -SELECT range_id FROM crdb_internal.ranges WHERE table_name = 't' +SELECT range_id FROM [SHOW RANGES FROM TABLE t] OFFSET 1 LIMIT 1 let $tableid SELECT id FROM system.namespace WHERE name = 't' @@ -52,7 +52,7 @@ INSERT INTO t2 (SELECT * FROM generate_series(1, 4096)); ALTER TABLE t2 SPLIT AT VALUES (0) let $rangeid -SELECT range_id FROM crdb_internal.ranges WHERE table_name = 't2' +SELECT range_id FROM [SHOW RANGES FROM TABLE t2] OFFSET 1 LIMIT 1 let $tableid SELECT id FROM system.namespace WHERE name = 't2' diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 23121d67a6b9..9b7cd4f5eda4 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -584,6 +584,7 @@ gossip_liveness NULL gossip_network NULL gossip_nodes NULL index_columns NULL +index_spans NULL index_usage_statistics NULL invalid_objects NULL jobs NULL @@ -623,6 +624,7 @@ system_jobs NULL table_columns NULL table_indexes NULL table_row_statistics NULL +table_spans NULL tables NULL tenant_usage_details NULL transaction_contention_events NULL diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index 0b0af3634f79..11ca91a36fad 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -1857,6 +1857,13 @@ func TestLogic_show_indexes( runLogicTest(t, "show_indexes") } +func TestLogic_show_ranges( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "show_ranges") +} + func TestLogic_show_source( t *testing.T, ) { diff --git a/pkg/sql/multitenant_admin_function_test.go b/pkg/sql/multitenant_admin_function_test.go index 1e1bb5f9a29d..de9d86386288 100644 --- a/pkg/sql/multitenant_admin_function_test.go +++ b/pkg/sql/multitenant_admin_function_test.go @@ -167,7 +167,6 @@ func getReplicaState( expectedNumNonVotingReplicas int, message string, ) replicaState { - checkCount := func(kind string, numExpected int, actual []roachpb.NodeID) error { numActual := len(actual) if numExpected != numActual { @@ -185,7 +184,9 @@ func getReplicaState( ) // Actual #replicas can be < expected #replicas even after WaitForFullReplication. testutils.SucceedsSoon(t, func() error { - rows, err := db.QueryContext(ctx, "SELECT lease_holder, replicas, voting_replicas, non_voting_replicas FROM [SHOW RANGES FROM INDEX t@primary];") + rows, err := db.QueryContext(ctx, ` +SELECT lease_holder, replicas, voting_replicas, non_voting_replicas +FROM [SHOW RANGES FROM INDEX t@primary WITH DETAILS]`) require.NoErrorf(t, err, message) rowNestedStrings, err := sqlutils.RowsToStrMatrix(rows) require.NoErrorf(t, err, message) @@ -460,7 +461,9 @@ func TestTruncateTable(t *testing.T) { _, testServer, cleanup := createTestCluster(t, cfg) defer cleanup() db := createSystemTenantDB(t, testServer) - execQueries(db, "system", [][]string{{"NULL", "/1"}, {"/1", "NULL"}}) + execQueries(db, "system", [][]string{ + {"", "…/1"}, + {"…/1", ""}}) }() // Test secondary tenant. @@ -473,7 +476,10 @@ func TestTruncateTable(t *testing.T) { true, /* allowSplitAndScatter */ false, /* skipSQLSystemTenantCheck */ ) - execQueries(db, "secondary", [][]string{{"NULL", "/104/2/1"}, {"/104/2/1", "NULL"}}) + execQueries(db, "secondary", [][]string{ + {"", "…/104/2/1"}, + {"…/104/2/1", ""}, + }) }() } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize b/pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize index 0490d665616c..72771dcec895 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize +++ b/pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize @@ -27,25 +27,25 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1, # Verify data placement. query TTTI rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv WITH DETAILS] ---- -NULL /1 {1} 1 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 + …/1/1 {1} 1 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify data placement. query TTTI rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw WITH DETAILS] ---- -NULL /1 {5} 5 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 + …/1/1 {5} 5 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify that EXPLAIN ANALYZE (DISTSQL) works in a distributed setting. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg index 42d7e54f975b..523aebad579d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg @@ -14,19 +14,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # Verify that all aggregate functions that we expect to have multi-stage # execution do, in fact, get planned with 2 stages. @@ -811,19 +811,19 @@ ALTER TABLE sorted_data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE sorted_data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE sorted_data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 query T EXPLAIN (DISTSQL) SELECT a, max(b) FROM sorted_data GROUP BY a @@ -1001,19 +1001,19 @@ ALTER TABLE data2 EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data2] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data2 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1/1 {2} 2 -/1/1 /1/2 {5} 5 -/1/2 /1/3 {5} 5 -/1/3 /1/4 {5} 5 -/1/4 /1/5 {5} 5 -/1/5 /1/6 {5} 5 -/1/6 /1/7 {5} 5 -/1/7 /1/8 {5} 5 -/1/8 /1/9 {5} 5 -/1/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1/1 {2} 2 +…/1/1/1 …/1/1/2 {5} 5 +…/1/1/2 …/1/1/3 {5} 5 +…/1/1/3 …/1/1/4 {5} 5 +…/1/1/4 …/1/1/5 {5} 5 +…/1/1/5 …/1/1/6 {5} 5 +…/1/1/6 …/1/1/7 {5} 5 +…/1/1/7 …/1/1/8 {5} 5 +…/1/1/8 …/1/1/9 {5} 5 +…/1/1/9 {5} 5 query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_distinct_on b/pkg/sql/opt/exec/execbuilder/testdata/distsql_distinct_on index 58fdfe2a73dd..9f45dfcb0c52 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_distinct_on +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_distinct_on @@ -44,25 +44,25 @@ ALTER TABLE abc EXPERIMENTAL_RELOCATE VALUES (ARRAY[5], '3', '4', '5') query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /4 {2} 2 -/4 /6 {3} 3 -/6 /7 {4} 4 -/7 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/4 {2} 2 +…/1/4 …/1/6 {3} 3 +…/1/6 …/1/7 {4} 4 +…/1/7 {5} 5 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE abc] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE abc WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /NULL/NULL/NULL {5} 5 -/NULL/NULL/NULL /"1"/"1"/"2" {1} 1 -/"1"/"1"/"2" /"1"/"2"/"2" {2} 2 -/"1"/"2"/"2" /"2"/"3"/"4" {3} 3 -/"2"/"3"/"4" /"3"/"4"/"5" {4} 4 -/"3"/"4"/"5" NULL {5} 5 +start_key end_key replicas lease_holder + …/1/NULL/NULL/NULL {5} 5 +…/1/NULL/NULL/NULL …/1/"1"/"1"/"2" {1} 1 +…/1/"1"/"1"/"2" …/1/"1"/"2"/"2" {2} 2 +…/1/"1"/"2"/"2" …/1/"2"/"3"/"4" {3} 3 +…/1/"2"/"3"/"4" …/1/"3"/"4"/"5" {4} 4 +…/1/"3"/"4"/"5" {5} 5 query T EXPLAIN (VERBOSE) SELECT DISTINCT ON (x,y,z) x, y, z FROM xyz diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_group_join b/pkg/sql/opt/exec/execbuilder/testdata/distsql_group_join index f8e40f9a0193..809ab5b5a020 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_group_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_group_join @@ -14,19 +14,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 statement ok SET experimental_hash_group_join_enabled = true diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_indexjoin b/pkg/sql/opt/exec/execbuilder/testdata/distsql_indexjoin index 847600fa0b27..9d4b543d4694 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_indexjoin +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_indexjoin @@ -13,14 +13,14 @@ ALTER INDEX t@v EXPERIMENTAL_RELOCATE SELECT ARRAY[i+1], (i * 10)::int FROM generate_series(0, 4) AS g(i) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM INDEX t@v] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM INDEX t@v WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -/10 /20 {2} 2 -/20 /30 {3} 3 -/30 /40 {4} 4 -/40 NULL {5} 5 +start_key end_key replicas lease_holder + …/10 {1} 1 +…/10 …/20 {2} 2 +…/20 …/30 {3} 3 +…/30 …/40 {4} 4 +…/40 {5} 5 query T EXPLAIN (DISTSQL) SELECT * FROM t WHERE v > 10 AND v < 50 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_inverted_index b/pkg/sql/opt/exec/execbuilder/testdata/distsql_inverted_index index 23a101cbf392..d3e17a1f20fd 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_inverted_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_inverted_index @@ -63,22 +63,22 @@ ALTER TABLE json_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM INDEX json_tab@primary] ORDER BY lease_holder, start_key +FROM [SHOW RANGES FROM INDEX json_tab@primary WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -/10 /20 {2} 2 -/20 NULL {3} 3 +start_key end_key replicas lease_holder + …/10 {1} 1 +…/10 …/20 {2} 2 +…/20 …/ {3} 3 query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM TABLE json_tab] ORDER BY lease_holder, start_key +FROM [SHOW RANGES FROM TABLE json_tab WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -· NULL {1} 1 -/10 /20 {2} 2 -/20 · {3} 3 +start_key end_key replicas lease_holder + …/1/10 {1} 1 +…/2 {1} 1 +…/1/10 …/1/20 {2} 2 +…/1/20 …/2 {3} 3 # Filter with a fully-specified array. This should use a zigzag join. query T @@ -203,20 +203,20 @@ ALTER TABLE array_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 3), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM INDEX array_tab@primary] ORDER BY lease_holder, start_key +FROM [SHOW RANGES FROM INDEX array_tab@primary WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /3 {1} 1 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/3 {1} 1 +…/3 …/ {3} 3 query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM TABLE array_tab] ORDER BY lease_holder +FROM [SHOW RANGES FROM TABLE array_tab WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /3 {1} 1 -· NULL {1} 1 -/3 · {3} 3 +start_key end_key replicas lease_holder + …/1/3 {1} 1 +…/2 {1} 1 +…/1/3 …/2 {3} 3 # This should use a zigzag join. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_join b/pkg/sql/opt/exec/execbuilder/testdata/distsql_join index d45e6f8e4af5..1b2745d01a9f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_join @@ -14,19 +14,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # ensure merge joins are planned when there's orderings. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_merge_join b/pkg/sql/opt/exec/execbuilder/testdata/distsql_merge_join index 275fa2d98fef..05d3289c54ea 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_merge_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_merge_join @@ -103,31 +103,31 @@ ALTER TABLE grandchild2 EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE parent1] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE parent1 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /8 {1} 1 -/8 /16 {1} 1 -/16 /24 {1} 1 -/24 /32 {1} 1 -/32 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/8 {1} 1 +…/1/8 …/1/16 {1} 1 +…/1/16 …/1/24 {1} 1 +…/1/24 …/1/32 {1} 1 +…/1/32 {1} 1 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE child1] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE child1 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /4/44 {1} 1 -/4/44 /12/52 {1} 1 -/12/52 /20/60 {1} 1 -/20/60 /28/68 {1} 1 -/28/68 /36/76 {1} 1 -/36/76 NULL {1} 1 +start_key end_key replicas lease_holder + …/1/4/44 {1} 1 +…/1/4/44 …/1/12/52 {1} 1 +…/1/12/52 …/1/20/60 {1} 1 +…/1/20/60 …/1/28/68 {1} 1 +…/1/28/68 …/1/36/76 {1} 1 +…/1/36/76 {1} 1 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE grandchild1] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE grandchild1 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder + {1} 1 ############### # Merge joins # @@ -750,19 +750,19 @@ ALTER TABLE outer_p1 EXPERIMENTAL_RELOCATE SELECT ARRAY[(((i-3)/5)%4)::INT + 1], i FROM generate_series(3, 40, 5) AS g(i) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE outer_p1] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE outer_p1 WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /0 {5} 5 -/0 /5 {1} 1 -/5 /10 {2} 2 -/10 /15 {3} 3 -/15 /20 {4} 4 -/20 /25 {1} 1 -/25 /30 {2} 2 -/30 /35 {3} 3 -/35 /40 {4} 4 -/40 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/0 {5} 5 +…/1/0 …/1/5 {1} 1 +…/1/5 …/1/10 {2} 2 +…/1/10 …/1/15 {3} 3 +…/1/15 …/1/20 {4} 4 +…/1/20 …/1/25 {1} 1 +…/1/25 …/1/30 {2} 2 +…/1/30 …/1/35 {3} 3 +…/1/35 …/1/40 {4} 4 +…/1/40 {5} 5 ### Begin OUTER queries diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_misc b/pkg/sql/opt/exec/execbuilder/testdata/distsql_misc index eebaf51c9c40..f545e6be7105 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_misc +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_misc @@ -69,19 +69,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 query T EXPLAIN (DISTSQL) CREATE STATISTICS s1 ON a FROM data diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables b/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables index 633cd17bf74f..48fedd3f8190 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_numtables @@ -24,20 +24,20 @@ ALTER TABLE NumToStr EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToSquare] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToSquare WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {1} 1 +start_key end_key replicas lease_holder + {1} 1 query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToStr] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE NumToStr WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2000 {1} 1 -/2000 /4000 {2} 2 -/4000 /6000 {3} 3 -/6000 /8000 {4} 4 -/8000 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2000 {1} 1 +…/1/2000 …/1/4000 {2} 2 +…/1/4000 …/1/6000 {3} 3 +…/1/6000 …/1/8000 {4} 4 +…/1/8000 {5} 5 # # -- Basic tests -- diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_ordinality b/pkg/sql/opt/exec/execbuilder/testdata/distsql_ordinality index 82f9d5bceaa2..9ade818f3ea6 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_ordinality +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_ordinality @@ -21,14 +21,14 @@ ALTER TABLE xyz EXPERIMENTAL_RELOCATE VALUES query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /4 {2} 2 -/4 /6 {3} 3 -/6 /7 {4} 4 -/7 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/4 {2} 2 +…/1/4 …/1/6 {3} 3 +…/1/6 …/1/7 {4} 4 +…/1/7 {5} 5 query T EXPLAIN (VERBOSE) SELECT x, y, z, ordinality FROM xyz WITH ORDINALITY diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_tighten_spans b/pkg/sql/opt/exec/execbuilder/testdata/distsql_tighten_spans index dd88a602011f..1d9598602b4b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_tighten_spans +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_tighten_spans @@ -93,36 +93,36 @@ ALTER TABLE p2 EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1,3 ############################# query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p1] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p1 WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /0 {3} 3 -/0 /2 {4} 4 -/2 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/2/0 {3} 3 +…/2/0 …/2/2 {4} 4 +…/2/2 {5} 5 # Indexes query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM INDEX b] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM INDEX b WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /0 {3} 3 -/0 /2 {4} 4 -/2 NULL {5} 5 +start_key end_key replicas lease_holder + …/0 {3} 3 +…/0 …/2 {4} 4 +…/2 {5} 5 query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p2] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE p2 WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /0 {1} 1 -· /2 {1} 1 -/0 /2 {1} 1 -/2 NULL {1} 1 -/2 · {3} 3 +start_key end_key replicas lease_holder + …/1/0 {1} 1 +…/1/0 …/1/2 {1} 1 +…/2 …/2/2 {1} 1 +…/2/2 {1} 1 +…/1/2 …/2 {3} 3 ############### # Query tests # diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_union b/pkg/sql/opt/exec/execbuilder/testdata/distsql_union index 18ace61e686a..33f4049123ec 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_union +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_union @@ -19,14 +19,14 @@ ALTER TABLE xyz EXPERIMENTAL_RELOCATE VALUES (ARRAY[5], 5) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE xyz WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 subtest Union diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_window b/pkg/sql/opt/exec/execbuilder/testdata/distsql_window index 6e357910dee9..6f70fc1c6a0a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_window +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_window @@ -14,19 +14,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # Verify that the window functions with the same PARTITION BY clause are # evaluated as a single distributed windower stage followed by a couple of diff --git a/pkg/sql/opt/exec/execbuilder/testdata/experimental_distsql_planning_5node b/pkg/sql/opt/exec/execbuilder/testdata/experimental_distsql_planning_5node index a9bd33d6fa29..14ae060fb05e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/experimental_distsql_planning_5node +++ b/pkg/sql/opt/exec/execbuilder/testdata/experimental_distsql_planning_5node @@ -19,27 +19,27 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1, # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {5} 5 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {5} 5 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 statement ok SET experimental_distsql_planning = always diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze_plans b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze_plans index b604a4506fe6..f5e3af706518 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze_plans +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze_plans @@ -33,27 +33,27 @@ ALTER TABLE kw EXPERIMENTAL_RELOCATE SELECT ARRAY[i], i FROM generate_series(1, # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kv WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE kw WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {5} 5 -/1 /2 {1} 1 -/2 /3 {2} 2 -/3 /4 {3} 3 -/4 /5 {4} 4 -/5 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {5} 5 +…/1/1 …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 …/1/4 {3} 3 +…/1/4 …/1/5 {4} 4 +…/1/5 {5} 5 # This query verifies stat collection for the tableReader, mergeJoiner, and # aggregator. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist index 3c6bbe376cd9..9aa51c2e73f7 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_geospatial_dist @@ -61,7 +61,7 @@ statement ok ALTER INDEX geo_table@geom_index SPLIT AT VALUES (1152921574000000000) query TI colnames,rowsort -SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- replicas lease_holder {1} 1 @@ -100,11 +100,11 @@ statement ok ALTER INDEX geo_table@geom_index EXPERIMENTAL_RELOCATE VALUES (ARRAY[2], 1152921574000000000) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1152921574000000000 {1} 1 -/1152921574000000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1152921574000000000 {1} 1 +…/1152921574000000000 {2} 2 # Distributed. TODO(treilly): This claims to be distributed, but it isn't. What gives? query T @@ -169,11 +169,11 @@ statement ok ALTER INDEX geo_table@geom_index EXPERIMENTAL_RELOCATE VALUES (ARRAY[2], 1) query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1152921574000000000 {2} 2 -/1152921574000000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1152921574000000000 {2} 2 +…/1152921574000000000 {2} 2 # Filtering is placed at node 2. We need a retry here to account for possibly # stale dist sender caches. @@ -240,18 +240,18 @@ statement ok SET CLUSTER SETTING sql.spatial.experimental_box2d_comparison_operators.enabled = on query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1152921574000000000 {2} 2 -/1152921574000000000 NULL {2} 2 +start_key end_key replicas lease_holder + …/1152921574000000000 {2} 2 +…/1152921574000000000 {2} 2 query ITTTI colnames,rowsort -SELECT range_id, start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@primary] +SELECT range_id, start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@primary WITH DETAILS] ORDER BY lease_holder, start_key ---- -range_id start_key end_key replicas lease_holder -54 NULL NULL {2} 2 +range_id start_key end_key replicas lease_holder +54 {2} 2 # We should see a distributed execution (though need to retry to purge possibly # stale dist sender caches). diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_json_array_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_json_array_dist index 0803a95708d7..ed773cc404e3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_json_array_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_filter_json_array_dist @@ -66,12 +66,12 @@ ALTER INDEX json_inv EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW EXPERIMENTAL_RANGES FROM INDEX json_inv] ORDER BY lease_holder +FROM [SHOW EXPERIMENTAL_RANGES FROM INDEX json_inv WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -/10 /20 {2} 2 -/20 NULL {3} 3 +start_key end_key replicas lease_holder +…/ …/10 {1} 1 +…/10 …/20 {2} 2 +…/20 {3} 3 statement ok ALTER TABLE json_tab VALIDATE CONSTRAINT check_c @@ -352,12 +352,12 @@ ALTER INDEX arr_inv EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW EXPERIMENTAL_RANGES FROM INDEX arr_inv] ORDER BY lease_holder +FROM [SHOW EXPERIMENTAL_RANGES FROM INDEX arr_inv WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -/10 /20 {2} 2 -/20 NULL {3} 3 +start_key end_key replicas lease_holder +…/ …/10 {1} 1 +…/10 …/20 {2} 2 +…/20 {3} 3 query T EXPLAIN (DISTSQL) diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist index fa355c202d3e..868bb50c621a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist @@ -36,12 +36,12 @@ statement ok ALTER TABLE ltable EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 2), (ARRAY[3], 3) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 {3} 3 query T EXPLAIN (DISTSQL) SELECT lk, rk FROM ltable JOIN rtable@geom_index diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist_vec b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist_vec index 6edbe8a6c3b3..9f439d9b4e7f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist_vec +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_geospatial_dist_vec @@ -36,12 +36,12 @@ statement ok ALTER TABLE ltable EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 2), (ARRAY[3], 3) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE ltable WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 {3} 3 query T EXPLAIN (VEC) SELECT lk, rk FROM ltable JOIN rtable@geom_index diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_json_array_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_json_array_dist index 5476ff2aa377..97142898ab89 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_json_array_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_json_array_dist @@ -62,12 +62,12 @@ ALTER TABLE json_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 10), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM INDEX json_tab@primary] ORDER BY lease_holder, start_key +FROM [SHOW RANGES FROM INDEX json_tab@primary WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /10 {1} 1 -/10 /20 {2} 2 -/20 NULL {3} 3 +start_key end_key replicas lease_holder + …/10 {1} 1 +…/10 …/20 {2} 2 +…/20 …/ {3} 3 # This query performs an inverted join. query T @@ -287,11 +287,11 @@ ALTER TABLE array_tab EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 3), query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW RANGES FROM INDEX array_tab@primary] ORDER BY lease_holder, start_key +FROM [SHOW RANGES FROM INDEX array_tab@primary WITH DETAILS] ORDER BY lease_holder, start_key ---- -start_key end_key replicas lease_holder -NULL /3 {1} 1 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/3 {1} 1 +…/3 …/ {3} 3 # This query performs an inverted join. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_multi_column_dist b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_multi_column_dist index a19a80a2bdc7..dd96c22eb025 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_multi_column_dist +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_join_multi_column_dist @@ -78,12 +78,12 @@ ALTER TABLE j2 EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 44), (ARRA query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE j2] ORDER BY lease_holder +FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE j2 WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /44 {1} 1 -/44 /88 {2} 2 -/88 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/44 {1} 1 +…/1/44 …/1/88 {2} 2 +…/1/88 {3} 3 # This query performs an inverted join. query T @@ -158,12 +158,12 @@ ALTER TABLE a2 EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1), (ARRAY[2], 8), (ARRAY query TTTI colnames SELECT start_key, end_key, replicas, lease_holder -FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE a2] ORDER BY lease_holder +FROM [SHOW EXPERIMENTAL_RANGES FROM TABLE a2 WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /8 {1} 1 -/8 /16 {2} 2 -/16 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/8 {1} 1 +…/1/8 …/1/16 {2} 2 +…/1/16 {3} 3 # This query performs an inverted join. query T diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join index 1d60e9f969ad..3088ad90241d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join @@ -425,19 +425,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 query T EXPLAIN (VERBOSE) @@ -607,10 +607,10 @@ vectorized: true # Verify data placement. query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE books] +SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE books WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL NULL {5} 5 +start_key end_key replicas lease_holder + {5} 5 query T EXPLAIN (DISTSQL) SELECT DISTINCT authors.name FROM books AS b1, books2 AS b2, authors WHERE b1.title = b2.title AND authors.book = b1.title AND b1.shelf <> b2.shelf diff --git a/pkg/sql/opt/exec/execbuilder/testdata/merge_join_dist_vec b/pkg/sql/opt/exec/execbuilder/testdata/merge_join_dist_vec index 1c7d6dfd106b..558638d37b61 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/merge_join_dist_vec +++ b/pkg/sql/opt/exec/execbuilder/testdata/merge_join_dist_vec @@ -27,20 +27,20 @@ statement ok ALTER TABLE r EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 2), (ARRAY[2], 3), (ARRAY[3], 1) query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE l] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE l WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -NULL /2 {1} 1 -/2 /3 {2} 2 -/3 NULL {3} 3 +start_key end_key replicas lease_holder + …/1/2 {1} 1 +…/1/2 …/1/3 {2} 2 +…/1/3 {3} 3 query TTTI colnames -SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE r] ORDER BY lease_holder +SELECT start_key, end_key, replicas, lease_holder from [SHOW EXPERIMENTAL_RANGES FROM TABLE r WITH DETAILS] ORDER BY lease_holder ---- -start_key end_key replicas lease_holder -/2 /3 {1} 1 -/3 NULL {2} 2 -NULL /2 {3} 3 +start_key end_key replicas lease_holder +…/1/2 …/1/3 {1} 1 +…/1/3 {2} 2 + …/1/2 {3} 3 query T EXPLAIN (DISTSQL) SELECT * FROM l LEFT OUTER JOIN r USING(a) WHERE a = 2 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/scan_parallel b/pkg/sql/opt/exec/execbuilder/testdata/scan_parallel index 946ae042557f..6c5bd3203bbf 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/scan_parallel +++ b/pkg/sql/opt/exec/execbuilder/testdata/scan_parallel @@ -14,19 +14,19 @@ ALTER TABLE data EXPERIMENTAL_RELOCATE # Verify data placement. query TTTI colnames,rowsort -SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data] +SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM TABLE data WITH DETAILS] ---- -start_key end_key replicas lease_holder -NULL /1 {1} 1 -/1 /2 {2} 2 -/2 /3 {3} 3 -/3 /4 {4} 4 -/4 /5 {5} 5 -/5 /6 {1} 1 -/6 /7 {2} 2 -/7 /8 {3} 3 -/8 /9 {4} 4 -/9 NULL {5} 5 +start_key end_key replicas lease_holder + …/1/1 {1} 1 +…/1/1 …/1/2 {2} 2 +…/1/2 …/1/3 {3} 3 +…/1/3 …/1/4 {4} 4 +…/1/4 …/1/5 {5} 5 +…/1/5 …/1/6 {1} 1 +…/1/6 …/1/7 {2} 2 +…/1/7 …/1/8 {3} 3 +…/1/8 …/1/9 {4} 4 +…/1/9 {5} 5 # Populate the range cache. statement ok diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f683b5d3ce75..4956bfeceb9d 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -831,6 +831,9 @@ func (u *sqlSymUnion) functionObjs() tree.FuncObjs { func (u *sqlSymUnion) tenantReplicationOptions() *tree.TenantReplicationOptions { return u.val.(*tree.TenantReplicationOptions) } +func (u *sqlSymUnion) showRangesOpts() *tree.ShowRangesOptions { + return u.val.(*tree.ShowRangesOptions) +} %} // NB: the %token definitions must come before the %type definitions in this @@ -869,7 +872,7 @@ func (u *sqlSymUnion) tenantReplicationOptions() *tree.TenantReplicationOptions %token CURRENT_USER CURSOR CYCLE %token DATA DATABASE DATABASES DATE DAY DEBUG_PAUSE_ON DEC DECIMAL DEFAULT DEFAULTS DEFINER -%token DEALLOCATE DECLARE DEFERRABLE DEFERRED DELETE DELIMITER DEPENDS DESC DESTINATION DETACHED +%token DEALLOCATE DECLARE DEFERRABLE DEFERRED DELETE DELIMITER DEPENDS DESC DESTINATION DETACHED DETAILS %token DISCARD DISTINCT DO DOMAIN DOUBLE DROP %token ELSE ENCODING ENCRYPTED ENCRYPTION_PASSPHRASE END ENUM ENUMS ESCAPE EXCEPT EXCLUDE EXCLUDING @@ -1602,6 +1605,7 @@ func (u *sqlSymUnion) tenantReplicationOptions() *tree.TenantReplicationOptions %type <*tree.LabelSpec> label_spec +%type <*tree.ShowRangesOptions> opt_show_ranges_options show_ranges_options // Precedence: lowest to highest %nonassoc VALUES // see value_clause @@ -7815,23 +7819,91 @@ show_range_for_row_stmt: // %Help: SHOW RANGES - list ranges // %Category: Misc // %Text: -// SHOW RANGES FROM TABLE -// SHOW RANGES FROM INDEX [ @ ] +// SHOW CLUSTER RANGES [ WITH ] +// SHOW RANGES FROM DATABASE [ WITH ] +// SHOW RANGES FROM CURRENT_CATALOG [ WITH ] +// SHOW RANGES FROM TABLE [ WITH ] +// SHOW RANGES FROM INDEX [ @ ] [ WITH ] +// +// Options: +// INDEXES +// TABLES +// DETAILS +// EXPLAIN show_ranges_stmt: - SHOW RANGES FROM TABLE table_name + SHOW RANGES FROM INDEX table_index_name opt_show_ranges_options + { + $$.val = &tree.ShowRanges{Source: tree.ShowRangesIndex, TableOrIndex: $5.tableIndexName(), Options: $6.showRangesOpts()} + } +| SHOW RANGES FROM TABLE table_name opt_show_ranges_options { name := $5.unresolvedObjectName().ToTableName() - $$.val = &tree.ShowRanges{TableOrIndex: tree.TableIndexName{Table: name}} + $$.val = &tree.ShowRanges{Source: tree.ShowRangesTable, TableOrIndex: tree.TableIndexName{Table: name}, Options: $6.showRangesOpts()} + } +| SHOW RANGES FROM DATABASE database_name opt_show_ranges_options + { + $$.val = &tree.ShowRanges{Source: tree.ShowRangesDatabase, DatabaseName: tree.Name($5), Options: $6.showRangesOpts()} } -| SHOW RANGES FROM INDEX table_index_name +| SHOW RANGES FROM CURRENT_CATALOG opt_show_ranges_options { - $$.val = &tree.ShowRanges{TableOrIndex: $5.tableIndexName()} + $$.val = &tree.ShowRanges{Source: tree.ShowRangesCurrentDatabase, Options: $5.showRangesOpts()} } -| SHOW RANGES FROM DATABASE database_name +| SHOW RANGES opt_show_ranges_options { - $$.val = &tree.ShowRanges{DatabaseName: tree.Name($5)} + $$.val = &tree.ShowRanges{Source: tree.ShowRangesCurrentDatabase, Options: $3.showRangesOpts()} } | SHOW RANGES error // SHOW HELP: SHOW RANGES +| SHOW CLUSTER RANGES opt_show_ranges_options + { + $$.val = &tree.ShowRanges{Source: tree.ShowRangesCluster, Options: $4.showRangesOpts()} + } +| SHOW CLUSTER RANGES error // SHOW HELP: SHOW RANGES + +opt_show_ranges_options: + /* EMPTY */ + { $$.val = &tree.ShowRangesOptions{} } +| WITH show_ranges_options + { $$.val = $2.showRangesOpts() } + +show_ranges_options: + TABLES { $$.val = &tree.ShowRangesOptions{Mode: tree.ExpandTables} } +| INDEXES { $$.val = &tree.ShowRangesOptions{Mode: tree.ExpandIndexes} } +| DETAILS { $$.val = &tree.ShowRangesOptions{Details: true} } +| KEYS { $$.val = &tree.ShowRangesOptions{Keys: true} } +| EXPLAIN { $$.val = &tree.ShowRangesOptions{Explain: true} } +| show_ranges_options ',' TABLES + { + o := $1.showRangesOpts() + if o.Mode != 0 { return setErr(sqllex, errors.New("conflicting modes")) } + o.Mode = tree.ExpandTables + $$.val = o + } +| show_ranges_options ',' INDEXES + { + o := $1.showRangesOpts() + if o.Mode != 0 { return setErr(sqllex, errors.New("conflicting modes")) } + o.Mode = tree.ExpandIndexes + $$.val = o + } +| show_ranges_options ',' DETAILS + { + o := $1.showRangesOpts() + o.Details = true + $$.val = o + } +| show_ranges_options ',' EXPLAIN + { + o := $1.showRangesOpts() + o.Explain = true + $$.val = o + } +| show_ranges_options ',' KEYS + { + o := $1.showRangesOpts() + o.Keys = true + $$.val = o + } + // %Help: SHOW SURVIVAL GOAL - list survival goals // %Category: DDL @@ -15621,6 +15693,7 @@ unreserved_keyword: | DEPENDS | DESTINATION | DETACHED +| DETAILS | DISCARD | DOMAIN | DOUBLE diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index 47ac07e0a6fa..937ef5c7fdfd 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -1018,6 +1018,127 @@ SHOW RANGE FROM INDEX i FOR ROW ((1), (2)) -- fully parenthesized SHOW RANGE FROM INDEX i FOR ROW (_, _) -- literals removed SHOW RANGE FROM INDEX _ FOR ROW (1, 2) -- identifiers removed +parse +SHOW CLUSTER RANGES +---- +SHOW CLUSTER RANGES +SHOW CLUSTER RANGES -- fully parenthesized +SHOW CLUSTER RANGES -- literals removed +SHOW CLUSTER RANGES -- identifiers removed + +parse +SHOW CLUSTER RANGES WITH DETAILS +---- +SHOW CLUSTER RANGES WITH DETAILS +SHOW CLUSTER RANGES WITH DETAILS -- fully parenthesized +SHOW CLUSTER RANGES WITH DETAILS -- literals removed +SHOW CLUSTER RANGES WITH DETAILS -- identifiers removed + +parse +SHOW CLUSTER RANGES WITH INDEXES +---- +SHOW CLUSTER RANGES WITH INDEXES +SHOW CLUSTER RANGES WITH INDEXES -- fully parenthesized +SHOW CLUSTER RANGES WITH INDEXES -- literals removed +SHOW CLUSTER RANGES WITH INDEXES -- identifiers removed + +parse +SHOW CLUSTER RANGES WITH TABLES +---- +SHOW CLUSTER RANGES WITH TABLES +SHOW CLUSTER RANGES WITH TABLES -- fully parenthesized +SHOW CLUSTER RANGES WITH TABLES -- literals removed +SHOW CLUSTER RANGES WITH TABLES -- identifiers removed + +parse +SHOW CLUSTER RANGES WITH INDEXES, DETAILS, KEYS +---- +SHOW CLUSTER RANGES WITH DETAILS, KEYS, INDEXES -- normalized! +SHOW CLUSTER RANGES WITH DETAILS, KEYS, INDEXES -- fully parenthesized +SHOW CLUSTER RANGES WITH DETAILS, KEYS, INDEXES -- literals removed +SHOW CLUSTER RANGES WITH DETAILS, KEYS, INDEXES -- identifiers removed + +parse +SHOW RANGES FROM DATABASE d +---- +SHOW RANGES FROM DATABASE d +SHOW RANGES FROM DATABASE d -- fully parenthesized +SHOW RANGES FROM DATABASE d -- literals removed +SHOW RANGES FROM DATABASE _ -- identifiers removed + +parse +SHOW RANGES FROM CURRENT_CATALOG +---- +SHOW RANGES FROM CURRENT_CATALOG +SHOW RANGES FROM CURRENT_CATALOG -- fully parenthesized +SHOW RANGES FROM CURRENT_CATALOG -- literals removed +SHOW RANGES FROM CURRENT_CATALOG -- identifiers removed + +parse +SHOW RANGES FROM DATABASE d WITH TABLES +---- +SHOW RANGES FROM DATABASE d WITH TABLES +SHOW RANGES FROM DATABASE d WITH TABLES -- fully parenthesized +SHOW RANGES FROM DATABASE d WITH TABLES -- literals removed +SHOW RANGES FROM DATABASE _ WITH TABLES -- identifiers removed + +parse +SHOW RANGES FROM DATABASE d WITH TABLES, DETAILS +---- +SHOW RANGES FROM DATABASE d WITH DETAILS, TABLES -- normalized! +SHOW RANGES FROM DATABASE d WITH DETAILS, TABLES -- fully parenthesized +SHOW RANGES FROM DATABASE d WITH DETAILS, TABLES -- literals removed +SHOW RANGES FROM DATABASE _ WITH DETAILS, TABLES -- identifiers removed + +parse +SHOW RANGES FROM DATABASE d WITH DETAILS +---- +SHOW RANGES FROM DATABASE d WITH DETAILS +SHOW RANGES FROM DATABASE d WITH DETAILS -- fully parenthesized +SHOW RANGES FROM DATABASE d WITH DETAILS -- literals removed +SHOW RANGES FROM DATABASE _ WITH DETAILS -- identifiers removed + +parse +SHOW RANGES FROM CURRENT_CATALOG WITH TABLES +---- +SHOW RANGES FROM CURRENT_CATALOG WITH TABLES +SHOW RANGES FROM CURRENT_CATALOG WITH TABLES -- fully parenthesized +SHOW RANGES FROM CURRENT_CATALOG WITH TABLES -- literals removed +SHOW RANGES FROM CURRENT_CATALOG WITH TABLES -- identifiers removed + +parse +SHOW RANGES FROM DATABASE d WITH INDEXES +---- +SHOW RANGES FROM DATABASE d WITH INDEXES +SHOW RANGES FROM DATABASE d WITH INDEXES -- fully parenthesized +SHOW RANGES FROM DATABASE d WITH INDEXES -- literals removed +SHOW RANGES FROM DATABASE _ WITH INDEXES -- identifiers removed + +parse +SHOW RANGES +---- +SHOW RANGES FROM CURRENT_CATALOG -- normalized! +SHOW RANGES FROM CURRENT_CATALOG -- fully parenthesized +SHOW RANGES FROM CURRENT_CATALOG -- literals removed +SHOW RANGES FROM CURRENT_CATALOG -- identifiers removed + +parse +SHOW RANGES WITH INDEXES +---- +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- normalized! +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- fully parenthesized +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- literals removed +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- identifiers removed + + +parse +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES +---- +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- fully parenthesized +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- literals removed +SHOW RANGES FROM CURRENT_CATALOG WITH INDEXES -- identifiers removed + parse SHOW RANGES FROM TABLE d.t ---- @@ -1042,6 +1163,30 @@ SHOW RANGES FROM TABLE t -- fully parenthesized SHOW RANGES FROM TABLE t -- literals removed SHOW RANGES FROM TABLE _ -- identifiers removed +parse +SHOW RANGES FROM TABLE t WITH DETAILS +---- +SHOW RANGES FROM TABLE t WITH DETAILS +SHOW RANGES FROM TABLE t WITH DETAILS -- fully parenthesized +SHOW RANGES FROM TABLE t WITH DETAILS -- literals removed +SHOW RANGES FROM TABLE _ WITH DETAILS -- identifiers removed + +parse +SHOW RANGES FROM TABLE t WITH DETAILS, INDEXES +---- +SHOW RANGES FROM TABLE t WITH DETAILS, INDEXES +SHOW RANGES FROM TABLE t WITH DETAILS, INDEXES -- fully parenthesized +SHOW RANGES FROM TABLE t WITH DETAILS, INDEXES -- literals removed +SHOW RANGES FROM TABLE _ WITH DETAILS, INDEXES -- identifiers removed + +parse +SHOW RANGES FROM TABLE t WITH INDEXES +---- +SHOW RANGES FROM TABLE t WITH INDEXES +SHOW RANGES FROM TABLE t WITH INDEXES -- fully parenthesized +SHOW RANGES FROM TABLE t WITH INDEXES -- literals removed +SHOW RANGES FROM TABLE _ WITH INDEXES -- identifiers removed + parse SHOW RANGES FROM INDEX d.t@i ---- diff --git a/pkg/sql/scatter_test.go b/pkg/sql/scatter_test.go index 714fe86f552b..edcdf16687f2 100644 --- a/pkg/sql/scatter_test.go +++ b/pkg/sql/scatter_test.go @@ -59,7 +59,7 @@ func TestScatterRandomizeLeases(t *testing.T) { r.Exec(t, "ALTER TABLE test.t SPLIT AT (SELECT i*10 FROM generate_series(1, 99) AS g(i))") getLeaseholders := func() (map[int]int, error) { - rows := r.Query(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test.t]`) + rows := r.Query(t, `SELECT range_id, lease_holder FROM [SHOW RANGES FROM TABLE test.t WITH DETAILS]`) leaseholders := make(map[int]int) numRows := 0 for ; rows.Next(); numRows++ { @@ -132,8 +132,7 @@ func TestScatterResponse(t *testing.T) { // the actual split boundaries. Wait until the table itself is split off // into its own range. testutils.SucceedsSoon(t, func() error { - row := r.QueryRow(t, `SELECT count(*) FROM crdb_internal.ranges_no_leases WHERE table_id = $1`, - tableDesc.GetID()) + row := r.QueryRow(t, `SELECT count(*) FROM [SHOW RANGES FROM TABLE test.t] WHERE start_key LIKE '%TableMin%'`) var nRanges int row.Scan(&nRanges) if nRanges != 1 { diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index c4839cc4bf2d..b85d51c1ad45 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -127,6 +127,7 @@ const ( CrdbInternalGossipNetworkTableID CrdbInternalTransactionContentionEvents CrdbInternalIndexColumnsTableID + CrdbInternalIndexSpansTableID CrdbInternalIndexUsageStatisticsTableID CrdbInternalInflightTraceSpanTableID CrdbInternalJobsTableID @@ -153,6 +154,7 @@ const ( CrdbInternalStmtStatsTableID CrdbInternalTableColumnsTableID CrdbInternalTableIndexesTableID + CrdbInternalTableSpansTableID CrdbInternalTablesTableID CrdbInternalTablesTableLastStatsID CrdbInternalTransactionStatsTableID diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 3133b495b355..ba2062e1a7a9 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -704,22 +704,107 @@ func (node *ShowRoles) Format(ctx *FmtCtx) { // ShowRanges represents a SHOW RANGES statement. type ShowRanges struct { - TableOrIndex TableIndexName DatabaseName Name + TableOrIndex TableIndexName + Options *ShowRangesOptions + Source ShowRangesSource } +// ShowRangesSource represents the source of a SHOW RANGES statement. +type ShowRangesSource int8 + +const ( + // SHOW RANGES FROM CURRENT_CATALOG + ShowRangesCurrentDatabase ShowRangesSource = iota + // SHOW RANGES FROM DATABASE + ShowRangesDatabase + // SHOW RANGES FROM TABLE + ShowRangesTable + // SHOW RANGES FROM INDEX + ShowRangesIndex + // SHOW CLUSTER RANGES + ShowRangesCluster +) + +// ShowRangesOptions represents the WITH clause in SHOW RANGES. +type ShowRangesOptions struct { + Details bool + Explain bool + Keys bool + Mode ShowRangesMode +} + +// ShowRangesMode represents the WITH clause in SHOW RANGES. +type ShowRangesMode int8 + +const ( + // UniqueRanges tells to use just 1 row per range in the output. + // + // Note: The UniqueRanges constant must have value 0; otherwise, + // the parsing logic would become incorrect. + UniqueRanges ShowRangesMode = iota + // ExpandTables requests one row per table in the output. + ExpandTables + // ExpandIndexes requests one row per index in the output. + ExpandIndexes +) + // Format implements the NodeFormatter interface. func (node *ShowRanges) Format(ctx *FmtCtx) { - ctx.WriteString("SHOW RANGES FROM ") - if node.DatabaseName != "" { - ctx.WriteString("DATABASE ") - ctx.FormatNode(&node.DatabaseName) - } else if node.TableOrIndex.Index != "" { - ctx.WriteString("INDEX ") - ctx.FormatNode(&node.TableOrIndex) - } else { - ctx.WriteString("TABLE ") - ctx.FormatNode(&node.TableOrIndex) + ctx.WriteString("SHOW ") + if node.Source == ShowRangesCluster { + ctx.WriteString("CLUSTER ") + } + ctx.WriteString("RANGES") + if node.Source != ShowRangesCluster { + ctx.WriteString(" FROM ") + switch node.Source { + case ShowRangesCurrentDatabase: + ctx.WriteString("CURRENT_CATALOG") + case ShowRangesDatabase: + ctx.WriteString("DATABASE ") + ctx.FormatNode(&node.DatabaseName) + case ShowRangesIndex: + ctx.WriteString("INDEX ") + ctx.FormatNode(&node.TableOrIndex) + case ShowRangesTable: + ctx.WriteString("TABLE ") + ctx.FormatNode(&node.TableOrIndex) + } + } + ctx.FormatNode(node.Options) +} + +// Format implements the NodeFormatter interface. +func (node *ShowRangesOptions) Format(ctx *FmtCtx) { + noOpts := ShowRangesOptions{} + if *node == noOpts { + return + } + ctx.WriteString(" WITH ") + comma := "" + if node.Details { + ctx.WriteString("DETAILS") + comma = ", " + } + if node.Keys { + ctx.WriteString(comma) + ctx.WriteString("KEYS") + comma = ", " + } + if node.Explain { + ctx.WriteString(comma) + ctx.WriteString("EXPLAIN") + comma = ", " + } + if node.Mode != UniqueRanges { + ctx.WriteString(comma) + switch node.Mode { + case ExpandTables: + ctx.WriteString("TABLES") + case ExpandIndexes: + ctx.WriteString("INDEXES") + } } } diff --git a/pkg/sql/show_ranges_test.go b/pkg/sql/show_ranges_test.go index a6ad00ef907b..640ab103313c 100644 --- a/pkg/sql/show_ranges_test.go +++ b/pkg/sql/show_ranges_test.go @@ -52,7 +52,8 @@ func TestShowRangesWithLocality(t *testing.T) { replicas := make([]int, 3) // TestClusters get some localities by default. - q := `SELECT lease_holder, lease_holder_locality, replicas, replica_localities, voting_replicas, non_voting_replicas FROM [SHOW RANGES FROM TABLE t]` + q := `SELECT lease_holder, lease_holder_locality, replicas, replica_localities, voting_replicas, non_voting_replicas +FROM [SHOW RANGES FROM TABLE t WITH DETAILS]` result := sqlDB.QueryStr(t, q) for _, row := range result { // Verify the leaseholder localities. @@ -125,9 +126,9 @@ func TestShowRangesMultipleStores(t *testing.T) { sqlDB.Exec(t, "ALTER TABLE system.jobs SCATTER") // Ensure that the localities line up. for _, q := range []string{ - "SHOW RANGES FROM DATABASE system", - "SHOW RANGES FROM TABLE system.jobs", - "SHOW RANGES FROM INDEX system.jobs@jobs_status_created_idx", + "SHOW RANGES FROM DATABASE system WITH DETAILS", + "SHOW RANGES FROM TABLE system.jobs WITH DETAILS", + "SHOW RANGES FROM INDEX system.jobs@jobs_status_created_idx WITH DETAILS", "SHOW RANGE FROM TABLE system.jobs FOR ROW (0)", "SHOW RANGE FROM INDEX system.jobs@jobs_status_created_idx FOR ROW ('running', now(), 0)", } { diff --git a/pkg/sql/tests/truncate_test.go b/pkg/sql/tests/truncate_test.go index 25af91e9ce4f..85fa56037467 100644 --- a/pkg/sql/tests/truncate_test.go +++ b/pkg/sql/tests/truncate_test.go @@ -421,7 +421,7 @@ ALTER INDEX a_b_idx SPLIT AT VALUES(1000), (2000), (3000), (4000), (5000), (6000 // succeeds-soon block here and below. testutils.SucceedsSoon(t, func() error { row := tc.Conns[0].QueryRowContext(ctx, ` -SELECT count(*) FROM crdb_internal.ranges_no_leases WHERE table_id = 'a'::regclass`) +SELECT count(*) FROM [SHOW RANGES FROM TABLE a]`) assert.NoError(t, row.Err()) var nRanges int @@ -443,7 +443,7 @@ SELECT count(*) FROM crdb_internal.ranges_no_leases WHERE table_id = 'a'::regcla testutils.SucceedsSoon(t, func() error { row := tc.Conns[0].QueryRowContext(ctx, ` -SELECT count(*) FROM crdb_internal.ranges_no_leases WHERE table_id = 'a'::regclass`) +SELECT count(*) FROM [SHOW RANGES FROM TABLE a]`) assert.NoError(t, row.Err()) var nRanges int diff --git a/pkg/sql/ttl/ttljob/ttljob_test.go b/pkg/sql/ttl/ttljob/ttljob_test.go index e894e48195a2..92a6b91d6367 100644 --- a/pkg/sql/ttl/ttljob/ttljob_test.go +++ b/pkg/sql/ttl/ttljob/ttljob_test.go @@ -470,7 +470,7 @@ func TestRowLevelTTLJobMultipleNodes(t *testing.T) { // Split table ranges := sqlDB.QueryStr(t, fmt.Sprintf( - `SELECT lease_holder FROM [SHOW RANGES FROM INDEX %s@primary]`, + `SELECT lease_holder FROM [SHOW RANGES FROM INDEX %s@primary WITH DETAILS]`, tableName, )) require.Equal(t, 1, len(ranges)) diff --git a/pkg/sql/unsplit.go b/pkg/sql/unsplit.go index eb360a231c52..9ae124421bb2 100644 --- a/pkg/sql/unsplit.go +++ b/pkg/sql/unsplit.go @@ -88,31 +88,21 @@ type unsplitAllRun struct { func (n *unsplitAllNode) startExec(params runParams) error { // Use the internal executor to retrieve the split keys. - statement := ` - SELECT - start_key - FROM - crdb_internal.ranges_no_leases - WHERE - database_name=$1 AND table_name=$2 AND index_name=$3 AND split_enforced_until IS NOT NULL - ` - _, dbDesc, err := params.p.Descriptors().GetImmutableDatabaseByID( - params.ctx, params.p.txn, n.tableDesc.GetParentID(), tree.DatabaseLookupFlags{AvoidLeased: true}, - ) - if err != nil { - return err - } - indexName := "" - if n.index.GetID() != n.tableDesc.GetPrimaryIndexID() { - indexName = n.index.GetName() - } + const statement = `SELECT r.start_key +FROM crdb_internal.ranges_no_leases r, + crdb_internal.index_spans s +WHERE s.descriptor_id = $1 + AND s.index_id = $2 + AND s.start_key < r.end_key + AND s.end_key > r.start_key + AND r.start_key >= s.start_key -- only consider split points inside the table keyspace. + AND split_enforced_until IS NOT NULL` ie := params.p.ExecCfg().InternalExecutorFactory.NewInternalExecutor(params.SessionData()) it, err := ie.QueryIteratorEx( params.ctx, "split points query", params.p.txn, sessiondata.NoSessionDataOverride, statement, - dbDesc.GetName(), - n.tableDesc.GetName(), - indexName, + n.tableDesc.GetID(), + n.index.GetID(), ) if err != nil { return err diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 968aa077577a..4e372b9acd89 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -132,6 +132,10 @@ type virtualSchemaTable struct { // will be queryable but return no rows. Otherwise querying the table will // return an unimplemented error. unimplemented bool + + // resultColumns is optional; if present, it will be checked for coherency + // with schema. + resultColumns colinfo.ResultColumns } // virtualSchemaView represents a view within a virtualSchema @@ -212,6 +216,25 @@ func (t virtualSchemaTable) initVirtualTableDesc( return descpb.TableDescriptor{}, err } + if t.resultColumns != nil { + if len(mutDesc.Columns) != len(t.resultColumns) { + return descpb.TableDescriptor{}, errors.AssertionFailedf( + "virtual table %s.%s declares incorrect number of columns: %d vs %d", + sc.GetName(), mutDesc.GetName(), + len(mutDesc.Columns), len(t.resultColumns)) + } + for i := range mutDesc.Columns { + if mutDesc.Columns[i].Name != t.resultColumns[i].Name || + mutDesc.Columns[i].Hidden != t.resultColumns[i].Hidden || + mutDesc.Columns[i].Type.String() != t.resultColumns[i].Typ.String() { + return descpb.TableDescriptor{}, errors.AssertionFailedf( + "virtual table %s.%s declares incorrect column metadata: %#v vs %#v", + sc.GetName(), mutDesc.GetName(), + mutDesc.Columns[i], t.resultColumns[i]) + } + } + } + if t.generator != nil { for _, idx := range t.indexes { if idx.incomplete {