diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index f38e3cbff479..4cbd4d564279 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -958,7 +958,7 @@ func backupPlanHook( // Include all tenants. // TODO(tbg): make conditional on cluster setting. - tenantRows, err = p.ExecCfg().InternalExecutor.Query( + tenantRows, err = p.ExecCfg().InternalExecutor.QueryBuffered( ctx, "backup-lookup-tenant", p.ExtendedEvalContext().Txn, `SELECT id, active, info FROM system.tenants`, ) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row index 6215cfc015b4..7fdcffc1c9bf 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row +++ b/pkg/ccl/logictestccl/testdata/logic_test/regional_by_row @@ -1176,6 +1176,17 @@ INSERT INTO regional_by_row_table_as (pk, a, b) VALUES (30, 1, 1) statement error pq: duplicate key value violates unique constraint "regional_by_row_table_as_b_key"\nDETAIL: Key \(b\)=\(1\) already exists\. INSERT INTO regional_by_row_table_as (pk, a, b) VALUES (2, 1, 1) +# Verify that we plan single-region scans for REGIONAL BY ROW tables with a computed region. +query T +EXPLAIN SELECT * FROM regional_by_row_table_as WHERE pk = 10 +---- +distribution: local +vectorized: true +· +• scan + missing stats + table: regional_by_row_table_as@primary + spans: [/'us-east-1'/10 - /'us-east-1'/10] # Tests for altering the survivability of a REGIONAL BY ROW table. statement ok diff --git a/pkg/cli/initial_sql.go b/pkg/cli/initial_sql.go index 2ef5be0908c9..bcc7c65d3983 100644 --- a/pkg/cli/initial_sql.go +++ b/pkg/cli/initial_sql.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" ) // runInitialSQL concerns itself with running "initial SQL" code when @@ -80,21 +81,24 @@ func createAdminUser(ctx context.Context, s *server.Server, adminUser, adminPass // given server object. func cliDisableReplication(ctx context.Context, s *server.Server) error { return s.RunLocalSQL(ctx, - func(ctx context.Context, ie *sql.InternalExecutor) error { - rows, err := ie.Query(ctx, "get-zones", nil, + func(ctx context.Context, ie *sql.InternalExecutor) (retErr error) { + it, err := ie.QueryIterator(ctx, "get-zones", nil, "SELECT target FROM crdb_internal.zones") if err != nil { return 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() { retErr = errors.CombineErrors(retErr, it.Close()) }() - for _, row := range rows { - zone := string(*row[0].(*tree.DString)) + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + zone := string(*it.Cur()[0].(*tree.DString)) if _, err := ie.Exec(ctx, "set-zone", nil, fmt.Sprintf("ALTER %s CONFIGURE ZONE USING num_replicas = 1", zone)); err != nil { return err } } - - return nil + return err }) } diff --git a/pkg/cmd/generate-pg-catalog/BUILD.bazel b/pkg/cmd/generate-postgres-metadata-tables/BUILD.bazel similarity index 65% rename from pkg/cmd/generate-pg-catalog/BUILD.bazel rename to pkg/cmd/generate-postgres-metadata-tables/BUILD.bazel index 5448c856b231..729b9728d96e 100644 --- a/pkg/cmd/generate-pg-catalog/BUILD.bazel +++ b/pkg/cmd/generate-postgres-metadata-tables/BUILD.bazel @@ -1,9 +1,9 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library") go_library( - name = "generate-pg-catalog_lib", + name = "generate-postgres-metadata-tables_lib", srcs = ["main.go"], - importpath = "github.com/cockroachdb/cockroach/pkg/cmd/generate-pg-catalog", + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/generate-postgres-metadata-tables", visibility = ["//visibility:private"], deps = [ "//pkg/sql", @@ -12,7 +12,7 @@ go_library( ) go_binary( - name = "generate-pg-catalog", - embed = [":generate-pg-catalog_lib"], + name = "generate-postgres-metadata-tables", + embed = [":generate-postgres-metadata-tables_lib"], visibility = ["//visibility:public"], ) diff --git a/pkg/cmd/generate-pg-catalog/main.go b/pkg/cmd/generate-postgres-metadata-tables/main.go similarity index 79% rename from pkg/cmd/generate-pg-catalog/main.go rename to pkg/cmd/generate-postgres-metadata-tables/main.go index f46a2fc467b9..2c6fd124c16d 100644 --- a/pkg/cmd/generate-pg-catalog/main.go +++ b/pkg/cmd/generate-postgres-metadata-tables/main.go @@ -34,17 +34,18 @@ import ( const getServerVersion = `SELECT current_setting('server_version');` var ( - postgresAddr = flag.String("addr", "localhost:5432", "Postgres server address") - postgresUser = flag.String("user", "postgres", "Postgres user") + postgresAddr = flag.String("addr", "localhost:5432", "Postgres server address") + postgresUser = flag.String("user", "postgres", "Postgres user") + postgresSchema = flag.String("catalog", "pg_catalog", "Catalog or namespace, default: pg_catalog") ) func main() { flag.Parse() db := connect() defer closeDB(db) - pgCatalogFile := &sql.PGCatalogFile{ - PgVersion: getPGVersion(db), - PgCatalog: sql.PGCatalogTables{}, + pgCatalogFile := &sql.PGMetadataFile{ + PGVersion: getPGVersion(db), + PGMetadata: sql.PGMetadataTables{}, } rows := describePgCatalog(db) @@ -55,14 +56,14 @@ func main() { if err := rows.Scan(&table, &column, &dataType, &dataTypeOid); err != nil { panic(err) } - pgCatalogFile.PgCatalog.AddColumnMetadata(table, column, dataType, dataTypeOid) + pgCatalogFile.PGMetadata.AddColumnMetadata(table, column, dataType, dataTypeOid) } pgCatalogFile.Save(os.Stdout) } func describePgCatalog(conn *pgx.Conn) *pgx.Rows { - rows, err := conn.Query(sql.GetPGCatalogSQL) + rows, err := conn.Query(sql.GetPGMetadataSQL, *postgresSchema) if err != nil { panic(err) } diff --git a/pkg/cmd/roachtest/pgx_blocklist.go b/pkg/cmd/roachtest/pgx_blocklist.go index 042c229fcb07..b37cea2f0a11 100644 --- a/pkg/cmd/roachtest/pgx_blocklist.go +++ b/pkg/cmd/roachtest/pgx_blocklist.go @@ -20,7 +20,42 @@ var pgxBlocklists = blocklistsForVersion{ // Please keep these lists alphabetized for easy diffing. // After a failed run, an updated version of this blocklist should be available // in the test log. -var pgxBlocklist21_1 = pgxBlocklist20_2 +var pgxBlocklist21_1 = blocklist{ + "v4.Example_CustomType": "27796", + "v4.TestConnBeginBatchDeferredError": "31632", + "v4.TestConnCopyFromLarge": "52722", + "v4.TestConnQueryDeferredError": "31632", + "v4.TestConnQueryErrorWhileReturningRows": "26925", + "v4.TestConnQueryReadRowMultipleTimes": "26925", + "v4.TestConnQueryValues": "26925", + "v4.TestConnSendBatch": "44712", + "v4.TestConnSimpleProtocol": "21286", + "v4.TestConnSimpleProtocolRefusesNonStandardConformingStrings": "36215", + "v4.TestConnSimpleProtocolRefusesNonUTF8ClientEncoding": "37129", + "v4.TestDomainType": "27796", + "v4.TestFatalRxError": "35897", + "v4.TestFatalTxError": "35897", + "v4.TestInetCIDRArrayTranscodeIP": "18846", + "v4.TestInetCIDRArrayTranscodeIPNet": "18846", + "v4.TestInetCIDRTranscodeIP": "18846", + "v4.TestInetCIDRTranscodeIPNet": "18846", + "v4.TestInetCIDRTranscodeWithJustIP": "18846", + "v4.TestLargeObjects": "26725", + "v4.TestLargeObjectsMultipleTransactions": "26725", + "v4.TestLargeObjectsPreferSimpleProtocol": "26725", + "v4.TestListenNotify": "41522", + "v4.TestListenNotifySelfNotification": "41522", + "v4.TestListenNotifyWhileBusyIsSafe": "41522", + "v4.TestQueryContextErrorWhileReceivingRows": "26925", + "v4.TestRowDecode": "26925", + "v4.TestTransactionSuccessfulCommit": "31632", + "v4.TestTransactionSuccessfulRollback": "31632", + "v4.TestTxCommitSerializationFailure": "12701", + "v4.TestTxCommitWhenTxBroken": "31632", + "v4.TestTxNestedTransactionCommit": "31632", + "v4.TestTxNestedTransactionRollback": "31632", + "v4.TestUnregisteredTypeUsableAsStringArgumentAndBaseResult": "27796", +} var pgxBlocklist20_2 = blocklist{ "v4.Example_CustomType": "27796", diff --git a/pkg/jobs/deprecated.go b/pkg/jobs/deprecated.go index 86c69466dfe3..fdce695fd047 100644 --- a/pkg/jobs/deprecated.go +++ b/pkg/jobs/deprecated.go @@ -65,7 +65,7 @@ func (r *Registry) deprecatedMaybeAdoptJob( SELECT id, payload, progress IS NULL, status FROM system.jobs WHERE status IN ($1, $2, $3, $4, $5) ORDER BY created DESC` - it, err := r.ex.QueryIterator( + rows, err := r.ex.QueryBuffered( ctx, "adopt-job", nil /* txn */, stmt, StatusPending, StatusRunning, StatusCancelRequested, StatusPauseRequested, StatusReverting, ) @@ -73,17 +73,6 @@ WHERE status IN ($1, $2, $3, $4, $5) ORDER BY created DESC` return errors.Wrap(err, "failed querying for jobs") } - // TODO(yuzefovich): use QueryBuffered method once it is added to - // sqlutil.InternalExecutor interface. - var rows []tree.Datums - var ok bool - for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - rows = append(rows, it.Cur()) - } - if err != nil { - return errors.Wrap(err, "failed querying for jobs") - } - if randomizeJobOrder { rand.Seed(timeutil.Now().UnixNano()) rand.Shuffle(len(rows), func(i, j int) { rows[i], rows[j] = rows[j], rows[i] }) diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index dfbe6e7cd296..c60b9b7c7b98 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -295,12 +295,7 @@ func (s *jobScheduler) executeSchedules( // We have to make sure to close the iterator since we might return from the // for loop early (before Next() returns false). - defer func() { - closeErr := it.Close() - if retErr == nil { - retErr = closeErr - } - }() + defer func() { retErr = errors.CombineErrors(retErr, it.Close()) }() // The loop below might encounter an error after some schedules have been // executed (i.e. previous iterations succeeded), and this is ok. diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 3827b28b9aac..101cf0a3fb6e 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -919,12 +919,7 @@ func (r *Registry) cleanupOldJobsPage( } // We have to make sure to close the iterator since we might return from the // for loop early (before Next() returns false). - defer func() { - closeErr := it.Close() - if retErr == nil { - retErr = closeErr - } - }() + defer func() { retErr = errors.CombineErrors(retErr, it.Close()) }() toDelete := tree.NewDArray(types.Int) oldMicros := timeutil.ToUnixMicros(olderThan) diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index 9b7d81b724a5..ec6d83c9c91d 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -704,6 +704,12 @@ func (ie *wrappedInternalExecutor) QueryRowExWithCols( panic("not implemented") } +func (ie *wrappedInternalExecutor) QueryBuffered( + ctx context.Context, opName string, txn *kv.Txn, stmt string, qargs ...interface{}, +) ([]tree.Datums, error) { + panic("not implemented") +} + func (ie *wrappedInternalExecutor) QueryBufferedEx( ctx context.Context, opName string, diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index a5d8ff40f7db..b4b752657abd 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -195,6 +195,7 @@ func TestAddReplicaViaLearner(t *testing.T) { func TestAddRemoveNonVotingReplicasBasic(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) ctx := context.Background() knobs, ltk := makeReplicationTestKnobs() diff --git a/pkg/migration/migrationmanager/manager.go b/pkg/migration/migrationmanager/manager.go index e0f7a745b09e..8fa668226be2 100644 --- a/pkg/migration/migrationmanager/manager.go +++ b/pkg/migration/migrationmanager/manager.go @@ -310,17 +310,7 @@ SELECT id, status if err != nil { return false, 0, errors.Wrap(err, "failed to marshal version to JSON") } - // TODO(yuzefovich): use QueryBuffered method once it is added to - // sqlutil.InternalExecutor interface. - it, err := m.ie.QueryIterator(ctx, "migration-manager-find-jobs", txn, query, jsonMsg.String()) - if err != nil { - return false, 0, err - } - var rows []tree.Datums - var ok bool - for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - rows = append(rows, it.Cur()) - } + rows, err := m.ie.QueryBuffered(ctx, "migration-manager-find-jobs", txn, query, jsonMsg.String()) if err != nil { return false, 0, err } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index f163ec33ac28..77b29aad2e41 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -124,8 +124,8 @@ go_library( "partition.go", "partition_utils.go", "pg_catalog.go", - "pg_catalog_diff.go", "pg_extension.go", + "pg_metadata_diff.go", "plan.go", "plan_batch.go", "plan_columns.go", @@ -449,7 +449,7 @@ go_test( "namespace_test.go", "old_foreign_key_desc_test.go", "partition_test.go", - "pg_catalog_test.go", + "pg_metadata_test.go", "pg_oid_test.go", "pgwire_internal_test.go", "plan_opt_test.go", diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index 98d57109420e..bab3091ffe35 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -385,14 +385,16 @@ func (p *planner) resolveMemberOfWithAdminOption( } visited[m] = struct{}{} - rows, err := p.ExecCfg().InternalExecutor.Query( + it, err := p.ExecCfg().InternalExecutor.QueryIterator( ctx, "expand-roles", txn, lookupRolesStmt, m.Normalized(), ) if err != nil { return nil, err } - for _, row := range rows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() roleName := tree.MustBeDString(row[0]) isAdmin := row[1].(*tree.DBool) @@ -403,6 +405,9 @@ func (p *planner) resolveMemberOfWithAdminOption( // We need to expand this role. Let the "pop" worry about already-visited elements. toVisit = append(toVisit, role) } + if err != nil { + return nil, err + } } return ret, nil diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 3d823551df1f..db40f7af8f1b 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -2081,19 +2081,10 @@ SELECT "descID", version, expiration FROM system.public.lease AS OF SYSTEM TIME retryOptions.Closer = m.stopper.ShouldQuiesce() // The retry is required because of errors caused by node restarts. Retry 30 times. if err := retry.WithMaxAttempts(ctx, retryOptions, 30, func() error { - it, err := m.storage.internalExecutor.QueryIterator( + var err error + rows, err = m.storage.internalExecutor.QueryBuffered( ctx, "read orphaned leases", nil /*txn*/, sqlQuery, ) - if err != nil { - return err - } - rows = rows[:0] - // TODO(yuzefovich): use QueryBuffered method once it is added to - // sqlutil.InternalExecutor interface. - var ok bool - for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - rows = append(rows, it.Cur()) - } return err }); err != nil { log.Warningf(ctx, "unable to read orphaned leases: %+v", err) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 005c299825d7..792ce0b51f37 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -2840,14 +2840,16 @@ func getAllNames( ) (map[descpb.ID]NamespaceKey, error) { namespace := map[descpb.ID]NamespaceKey{} if executor.s.cfg.Settings.Version.IsActive(ctx, clusterversion.NamespaceTableWithSchemas) { - rows, err := executor.Query( + it, err := executor.QueryIterator( ctx, "get-all-names", txn, `SELECT id, "parentID", "parentSchemaID", name FROM system.namespace`, ) if err != nil { return nil, err } - for _, r := range rows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + r := it.Cur() id, parentID, parentSchemaID, name := tree.MustBeDInt(r[0]), tree.MustBeDInt(r[1]), tree.MustBeDInt(r[2]), tree.MustBeDString(r[3]) namespace[descpb.ID(id)] = NamespaceKey{ ParentID: descpb.ID(parentID), @@ -2855,20 +2857,25 @@ func getAllNames( Name: string(name), } } + if err != nil { + return nil, err + } } // Also get all rows from namespace_deprecated, and add to the namespace map // if it is not already there yet. // If a row exists in both here and namespace, only use the one from namespace. // TODO(sqlexec): In 20.2, this can be removed. - deprecatedRows, err := executor.Query( + it, err := executor.QueryIterator( ctx, "get-all-names-deprecated-namespace", txn, fmt.Sprintf(`SELECT id, "parentID", name FROM [%d as namespace]`, keys.DeprecatedNamespaceTableID), ) if err != nil { return nil, err } - for _, r := range deprecatedRows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + r := it.Cur() id, parentID, name := tree.MustBeDInt(r[0]), tree.MustBeDInt(r[1]), tree.MustBeDString(r[2]) if _, ok := namespace[descpb.ID(id)]; !ok { namespace[descpb.ID(id)] = NamespaceKey{ @@ -2877,6 +2884,9 @@ func getAllNames( } } } + if err != nil { + return nil, err + } return namespace, nil } @@ -2932,7 +2942,9 @@ CREATE TABLE crdb_internal.zones ( return kv.Value, nil } - rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( + // For some reason, if we use the iterator API here, "concurrent txn use + // detected" error might occur, so we buffer up all zones first. + rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryBuffered( ctx, "crdb-internal-zones-table", p.txn, `SELECT id, config FROM system.zones`) if err != nil { return err diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index a16aad7192f7..f000cfe4f460 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -599,14 +599,14 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er // pending, running, or paused status that started earlier than this one. If // there are, checkRunningJobs returns an error. If job is nil, checkRunningJobs // just checks if there are any pending, running, or paused CreateStats jobs. -func checkRunningJobs(ctx context.Context, job *jobs.Job, p JobExecContext) error { +func checkRunningJobs(ctx context.Context, job *jobs.Job, p JobExecContext) (retErr error) { var jobID jobspb.JobID if job != nil { jobID = job.ID() } const stmt = `SELECT id, payload FROM system.jobs WHERE status IN ($1, $2, $3) ORDER BY created` - rows, err := p.ExecCfg().InternalExecutor.Query( + it, err := p.ExecCfg().InternalExecutor.QueryIterator( ctx, "get-jobs", nil, /* txn */ @@ -618,8 +618,13 @@ func checkRunningJobs(ctx context.Context, job *jobs.Job, p JobExecContext) erro if err != nil { return 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() { retErr = errors.CombineErrors(retErr, it.Close()) }() - for _, row := range rows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() payload, err := jobs.UnmarshalPayload(row[1]) if err != nil { return err @@ -636,7 +641,7 @@ func checkRunningJobs(ctx context.Context, job *jobs.Job, p JobExecContext) erro return stats.ConcurrentCreateStatsError } } - return nil + return err } // OnFailOrCancel is part of the jobs.Resumer interface. diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 854ca435563b..a95134ab0aee 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -434,7 +434,9 @@ type DistSQLReceiver struct { rangeCache *rangecache.RangeCache tracing *SessionTracing - cleanup func() + // cleanup will be called when the DistSQLReceiver is Release()'d back to + // its sync.Pool. + cleanup func() // The transaction in which the flow producing data for this // receiver runs. The DistSQLReceiver updates the transaction in @@ -572,6 +574,7 @@ func MakeDistSQLReceiver( // Release releases this DistSQLReceiver back to the pool. func (r *DistSQLReceiver) Release() { + r.cleanup() *r = DistSQLReceiver{} receiverSyncPool.Put(r) } @@ -782,7 +785,6 @@ func (r *DistSQLReceiver) ProducerDone() { panic("double close") } r.closed = true - r.cleanup() } // Types is part of the RowReceiver interface. diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index d4bf2f9bb372..abe967efa83b 100755 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -497,13 +497,14 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`, dbNameStr, // udt_catalog udtSchema, // udt_schema tree.NewDString(column.GetType().PGName()), // udt_name - tree.DNull, // scope_catalog - tree.DNull, // scope_schema - tree.DNull, // scope_name - tree.DNull, // maximum_cardinality - tree.DNull, // dtd_identifier - tree.DNull, // is_self_referencing - tree.DNull, // is_identity + tree.DNull, // scope_catalog + tree.DNull, // scope_schema + tree.DNull, // scope_name + tree.DNull, // maximum_cardinality + tree.DNull, // dtd_identifier + tree.DNull, // is_self_referencing + //TODO: Need to update when supporting identiy columns (Issue #48532) + noString, // is_identity tree.DNull, // identity_generation tree.DNull, // identity_start tree.DNull, // identity_increment @@ -2223,10 +2224,13 @@ FROM ro.username = u.username AND option = 'VALID UNTIL'; ` - rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( + // For some reason, using the iterator API here causes privilege_builtins + // logic test fail in 3node-tenant config with 'txn already encountered an + // error' (because of the context cancellation), so we buffer all roles + // first. + rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryBuffered( ctx, "read-roles", p.txn, query, ) - if err != nil { return err } @@ -2259,16 +2263,21 @@ FROM func forEachRoleMembership( ctx context.Context, p *planner, fn func(role, member security.SQLUsername, isAdmin bool) error, -) error { +) (retErr error) { query := `SELECT "role", "member", "isAdmin" FROM system.role_members` - rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( + it, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryIterator( ctx, "read-members", p.txn, query, ) if err != nil { return 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() { retErr = errors.CombineErrors(retErr, it.Close()) }() - for _, row := range rows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() roleName := tree.MustBeDString(row[0]) memberName := tree.MustBeDString(row[1]) isAdmin := row[2].(*tree.DBool) @@ -2281,7 +2290,7 @@ func forEachRoleMembership( return err } } - return nil + return err } func userCanSeeDescriptor( diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index d60e4d2a798a..4e3d3304acdb 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -361,15 +361,16 @@ func (r *rowsIterator) Types() colinfo.ResultColumns { return r.resultCols } -// Query executes the supplied SQL statement and returns the resulting rows. -// If no user has been previously set through SetSessionData, the statement is -// executed as the root user. +// QueryBuffered executes the supplied SQL statement and returns the resulting +// rows (meaning all of them are buffered at once). If no user has been +// previously set through SetSessionData, the statement is executed as the root +// user. // // If txn is not nil, the statement will be executed in the respective txn. // -// Query is deprecated because it may transparently execute a query as root. Use -// QueryEx instead. -func (ie *InternalExecutor) Query( +// QueryBuffered is deprecated because it may transparently execute a query as +// root. Use QueryBufferedEx instead. +func (ie *InternalExecutor) QueryBuffered( ctx context.Context, opName string, txn *kv.Txn, stmt string, qargs ...interface{}, ) ([]tree.Datums, error) { return ie.QueryBufferedEx(ctx, opName, txn, ie.maybeRootSessionDataOverride(opName), stmt, qargs...) diff --git a/pkg/sql/internal_test.go b/pkg/sql/internal_test.go index 799b2e77bfa6..1fe5f4d2f3ac 100644 --- a/pkg/sql/internal_test.go +++ b/pkg/sql/internal_test.go @@ -168,7 +168,7 @@ func TestInternalFullTableScan(t *testing.T) { // Internal queries that perform full table scans shouldn't fail because of // the setting above. - _, err = ie.Query(ctx, "full-table-scan-select", nil, "SELECT * FROM db.t") + _, err = ie.Exec(ctx, "full-table-scan-select", nil, "SELECT * FROM db.t") require.NoError(t, err) } @@ -383,9 +383,9 @@ func TestInternalExecAppNameInitialization(t *testing.T) { } type testInternalExecutor interface { - Query( + QueryRow( ctx context.Context, opName string, txn *kv.Txn, stmt string, qargs ...interface{}, - ) ([]tree.Datums, error) + ) (tree.Datums, error) Exec( ctx context.Context, opName string, txn *kv.Txn, stmt string, qargs ...interface{}, ) (int, error) @@ -398,12 +398,12 @@ func testInternalExecutorAppNameInitialization( ie testInternalExecutor, ) { // Check that the application_name is set properly in the executor. - if rows, err := ie.Query(context.Background(), "test-query", nil, + if row, err := ie.QueryRow(context.Background(), "test-query", nil, "SHOW application_name"); err != nil { t.Fatal(err) - } else if len(rows) != 1 { - t.Fatalf("expected 1 row, got: %+v", rows) - } else if appName := string(*rows[0][0].(*tree.DString)); appName != expectedAppName { + } else if row == nil { + t.Fatalf("expected 1 row, got 0") + } else if appName := string(*row[0].(*tree.DString)); appName != expectedAppName { t.Fatalf("unexpected app name: expected %q, got %q", expectedAppName, appName) } @@ -411,7 +411,7 @@ func testInternalExecutorAppNameInitialization( // have this keep running until we cancel it below. errChan := make(chan error) go func() { - _, err := ie.Query(context.Background(), + _, err := ie.Exec(context.Background(), "test-query", nil, /* txn */ "SELECT pg_sleep(1337666)") @@ -427,7 +427,7 @@ func testInternalExecutorAppNameInitialization( // When it does, we capture the query ID. var queryID string testutils.SucceedsSoon(t, func() error { - rows, err := ie.Query(context.Background(), + row, err := ie.QueryRow(context.Background(), "find-query", nil, /* txn */ // We need to assemble the magic string so that this SELECT @@ -436,32 +436,29 @@ func testInternalExecutorAppNameInitialization( if err != nil { return err } - switch len(rows) { - case 0: + if row == nil { // The SucceedsSoon test may find this a couple of times before // this succeeds. return fmt.Errorf("query not started yet") - case 1: - appName := string(*rows[0][1].(*tree.DString)) + } else { + appName := string(*row[1].(*tree.DString)) if appName != expectedAppName { return fmt.Errorf("unexpected app name: expected %q, got %q", expectedAppName, appName) } // Good app name, retrieve query ID for later cancellation. - queryID = string(*rows[0][0].(*tree.DString)) + queryID = string(*row[0].(*tree.DString)) return nil - default: - return fmt.Errorf("unexpected results: %+v", rows) } }) // Check that the query shows up in the internal tables without error. - if rows, err := ie.Query(context.Background(), "find-query", nil, + if row, err := ie.QueryRow(context.Background(), "find-query", nil, "SELECT application_name FROM crdb_internal.node_queries WHERE query LIKE '%337' || '666%'"); err != nil { t.Fatal(err) - } else if len(rows) != 1 { - t.Fatalf("expected 1 query, got: %+v", rows) - } else if appName := string(*rows[0][0].(*tree.DString)); appName != expectedAppName { + } else if row == nil { + t.Fatalf("expected 1 query, got 0") + } else if appName := string(*row[0].(*tree.DString)); appName != expectedAppName { t.Fatalf("unexpected app name: expected %q, got %q", expectedAppName, appName) } @@ -481,12 +478,12 @@ func testInternalExecutorAppNameInitialization( } // Now check that it was properly registered in statistics. - if rows, err := ie.Query(context.Background(), "find-query", nil, + if row, err := ie.QueryRow(context.Background(), "find-query", nil, "SELECT application_name FROM crdb_internal.node_statement_statistics WHERE key LIKE 'SELECT' || ' pg_sleep(%'"); err != nil { t.Fatal(err) - } else if len(rows) != 1 { - t.Fatalf("expected 1 query, got: %+v", rows) - } else if appName := string(*rows[0][0].(*tree.DString)); appName != expectedAppNameInStats { + } else if row == nil { + t.Fatalf("expected 1 query, got 0") + } else if appName := string(*row[0].(*tree.DString)); appName != expectedAppNameInStats { t.Fatalf("unexpected app name: expected %q, got %q", expectedAppNameInStats, appName) } } diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 0256aaa4b9d2..41f00ba80ece 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -4301,3 +4301,13 @@ udt_schema udt_name sh d public e pg_catalog int8 + +# Testing information_schema.columns.is_identity which for now is False for every column +statement ok +SET DATABASE = ""; + +query T colnames +SELECT distinct is_identity FROM information_schema.columns +---- +is_identity +NO diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic index 33570a162d9f..0a016b360620 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit_nonmetamorphic @@ -668,6 +668,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' ---- dist sender send r36: sending batch 1 DelRng to (n1,s1):1 dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 Scan to (n1,s1):1 dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B @@ -692,6 +693,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' dist sender send r36: sending batch 1 DelRng to (n1,s1):1 dist sender send r36: sending batch 1 Scan to (n1,s1):1 dist sender send r36: sending batch 1 Put to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B @@ -717,6 +719,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' dist sender send r36: sending batch 1 DelRng to (n1,s1):1 dist sender send r36: sending batch 1 Scan to (n1,s1):1 dist sender send r36: sending batch 1 Del to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. @@ -743,6 +746,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' ---- dist sender send r36: sending batch 1 DelRng to (n1,s1):1 dist sender send r36: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 dist sender send r36: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # ----------------------- diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic index f4e618e0e849..e45575fce18a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index_nonmetamorphic @@ -905,6 +905,7 @@ query T kvtrace DELETE FROM t57085_p WHERE p = 1; ---- DelRange /Table/56/1/1 - /Table/56/1/1/# +Scan /Table/57/{1-2} Del /Table/57/2/1/10/0 Del /Table/57/1/10/0 Del /Table/57/1/20/0 diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index becc31f6c2e7..bd7cb3206e9e 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1532,7 +1532,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-depend.html`, // as a datum row, containing object id, sub id (column id in the case of // columns), comment text, and comment type (keys.FooCommentType). func getComments(ctx context.Context, p *planner) ([]tree.Datums, error) { - return p.extendedEvalCtx.ExecCfg.InternalExecutor.Query( + return p.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( ctx, "select-comments", p.EvalContext().Txn, diff --git a/pkg/sql/pg_catalog_diff.go b/pkg/sql/pg_metadata_diff.go similarity index 61% rename from pkg/sql/pg_catalog_diff.go rename to pkg/sql/pg_metadata_diff.go index a48c7a5413dc..a0ee13f2a289 100644 --- a/pkg/sql/pg_catalog_diff.go +++ b/pkg/sql/pg_metadata_diff.go @@ -9,7 +9,7 @@ // licenses/APL.txt. // This file contains constants and types used by pg_catalog diff tool -// that are also re-used in /pkg/cmd/generate-pg-catalog +// that are also re-used in /pkg/cmd/generate-postgres-metadata-tables package sql @@ -19,9 +19,9 @@ import ( "os" ) -// GetPGCatalogSQL is a query uses udt_name::regtype instead of data_type column because +// GetPGMetadataSQL is a query uses udt_name::regtype instead of data_type column because // data_type only says "ARRAY" but does not say which kind of array it is. -const GetPGCatalogSQL = ` +const GetPGMetadataSQL = ` SELECT c.relname AS table_name, a.attname AS column_name, @@ -31,47 +31,47 @@ const GetPGCatalogSQL = ` JOIN pg_attribute a ON a.attrelid = c.oid JOIN pg_type t ON t.oid = a.atttypid JOIN pg_namespace n ON n.oid = c.relnamespace - WHERE n.nspname = 'pg_catalog' + WHERE n.nspname = $1 AND a.attnum > 0 ORDER BY 1, 2; ` -// PGCatalogColumn is a structure which contains a small description about the datatype of a column, but this can also be +// PGMetadataColumnType is a structure which contains a small description about the datatype of a column, but this can also be // used as a diff information if populating ExpectedOid. Fields are exported for Marshaling purposes. -type PGCatalogColumn struct { +type PGMetadataColumnType struct { Oid uint32 `json:"oid"` DataType string `json:"dataType"` ExpectedOid *uint32 `json:"expectedOid"` ExpectedDataType *string `json:"expectedDataType"` } -// PGCatalogColumns maps column names to datatype description -type PGCatalogColumns map[string]*PGCatalogColumn +// PGMetadataColumns maps column names to datatype description +type PGMetadataColumns map[string]*PGMetadataColumnType -// PGCatalogTables have 2 use cases: +// PGMetadataTables have 2 use cases: // First: This is used to model pg_schema for postgres and cockroach db for comparison purposes by mapping tableNames // to columns. // Second: This is used to store and load expected diffs: -// - Using it this way, a table name pointing to a zero length PGCatalogColumns means that we expect this table to be missing +// - Using it this way, a table name pointing to a zero length PGMetadataColumns means that we expect this table to be missing // in cockroach db -// - If PGCatalogColumns is not empty but columnName points to null, we expect that column to be missing in that table in +// - If PGMetadataColumns is not empty but columnName points to null, we expect that column to be missing in that table in // cockroach db -// - If column Name points to a not null PGCatalogColumn, the test column describes how we expect that data type to be +// - If column Name points to a not null PGMetadataColumnType, the test column describes how we expect that data type to be // different between cockroach db and postgres -type PGCatalogTables map[string]PGCatalogColumns +type PGMetadataTables map[string]PGMetadataColumns -// PGCatalogFile is used to export pg_catalog from postgres and store the representation of this structure as a +// PGMetadataFile is used to export pg_catalog from postgres and store the representation of this structure as a // json file -type PGCatalogFile struct { - PgVersion string `json:"pgVersion"` - PgCatalog PGCatalogTables `json:"pgCatalog"` +type PGMetadataFile struct { + PGVersion string `json:"pgVersion"` + PGMetadata PGMetadataTables `json:"pgMetadata"` } -func (p PGCatalogTables) addColumn(tableName, columnName string, column *PGCatalogColumn) { +func (p PGMetadataTables) addColumn(tableName, columnName string, column *PGMetadataColumnType) { columns, ok := p[tableName] if !ok { - columns = make(PGCatalogColumns) + columns = make(PGMetadataColumns) p[tableName] = columns } @@ -79,10 +79,10 @@ func (p PGCatalogTables) addColumn(tableName, columnName string, column *PGCatal } // AddColumnMetadata is used to load data from postgres or cockroach pg_catalog schema -func (p PGCatalogTables) AddColumnMetadata( +func (p PGMetadataTables) AddColumnMetadata( tableName string, columnName string, dataType string, dataTypeOid uint32, ) { - p.addColumn(tableName, columnName, &PGCatalogColumn{ + p.addColumn(tableName, columnName, &PGMetadataColumnType{ dataTypeOid, dataType, nil, @@ -91,10 +91,10 @@ func (p PGCatalogTables) AddColumnMetadata( } // addDiff is for the second use case for pgTables which objective is create a datatype diff -func (p PGCatalogTables) addDiff( - tableName string, columnName string, expected *PGCatalogColumn, actual *PGCatalogColumn, +func (p PGMetadataTables) addDiff( + tableName string, columnName string, expected *PGMetadataColumnType, actual *PGMetadataColumnType, ) { - p.addColumn(tableName, columnName, &PGCatalogColumn{ + p.addColumn(tableName, columnName, &PGMetadataColumnType{ actual.Oid, actual.DataType, &expected.Oid, @@ -103,8 +103,8 @@ func (p PGCatalogTables) addDiff( } // isDiffOid verifies if there is a datatype mismatch or if the diff is an expected diff -func (p PGCatalogTables) isDiffOid( - tableName string, columnName string, expected *PGCatalogColumn, actual *PGCatalogColumn, +func (p PGMetadataTables) isDiffOid( + tableName string, columnName string, expected *PGMetadataColumnType, actual *PGMetadataColumnType, ) bool { if expected.Oid == actual.Oid { return false @@ -125,9 +125,9 @@ func (p PGCatalogTables) isDiffOid( return !(diff.Oid == actual.Oid && *diff.ExpectedOid == expected.Oid) } -// isExpectedMissingTable is used by the diff PGCatalogTables to verify whether missing a table in cockroach is expected +// isExpectedMissingTable is used by the diff PGMetadataTables to verify whether missing a table in cockroach is expected // or not -func (p PGCatalogTables) isExpectedMissingTable(tableName string) bool { +func (p PGMetadataTables) isExpectedMissingTable(tableName string) bool { if columns, ok := p[tableName]; !ok || len(columns) > 0 { return false } @@ -136,7 +136,7 @@ func (p PGCatalogTables) isExpectedMissingTable(tableName string) bool { } // isExpectedMissingColumn is similar to isExpectedMissingTable to verify column expected misses -func (p PGCatalogTables) isExpectedMissingColumn(tableName string, columnName string) bool { +func (p PGMetadataTables) isExpectedMissingColumn(tableName string, columnName string) bool { columns, ok := p[tableName] if !ok { return false @@ -151,16 +151,16 @@ func (p PGCatalogTables) isExpectedMissingColumn(tableName string, columnName st } // addMissingTable adds a tablename when it is not found in cockroach db -func (p PGCatalogTables) addMissingTable(tableName string) { - p[tableName] = make(PGCatalogColumns) +func (p PGMetadataTables) addMissingTable(tableName string) { + p[tableName] = make(PGMetadataColumns) } // addMissingColumn adds a column when it is not found in cockroach db -func (p PGCatalogTables) addMissingColumn(tableName string, columnName string) { +func (p PGMetadataTables) addMissingColumn(tableName string, columnName string) { columns, ok := p[tableName] if !ok { - columns = make(PGCatalogColumns) + columns = make(PGMetadataColumns) p[tableName] = columns } @@ -168,7 +168,7 @@ func (p PGCatalogTables) addMissingColumn(tableName string, columnName string) { } // rewriteDiffs creates pg_catalog_test-diffs.json -func (p PGCatalogTables) rewriteDiffs(diffFile string) error { +func (p PGMetadataTables) rewriteDiffs(diffFile string) error { f, err := os.OpenFile(diffFile, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) if err != nil { return err @@ -188,7 +188,7 @@ func (p PGCatalogTables) rewriteDiffs(diffFile string) error { } // Save have the purpose of storing all the data retrieved from postgres and useful information as postgres version -func (f *PGCatalogFile) Save(writer io.Writer) { +func (f *PGMetadataFile) Save(writer io.Writer) { byteArray, err := json.MarshalIndent(f, "", " ") if err != nil { panic(err) diff --git a/pkg/sql/pg_catalog_test.go b/pkg/sql/pg_metadata_test.go similarity index 81% rename from pkg/sql/pg_catalog_test.go rename to pkg/sql/pg_metadata_test.go index 53cd4e57a735..c707531902a1 100644 --- a/pkg/sql/pg_catalog_test.go +++ b/pkg/sql/pg_metadata_test.go @@ -46,13 +46,16 @@ import ( // Test data files const ( - pgCatalogDump = "pg_catalog_tables.json" // PostgreSQL pg_catalog schema - expectedDiffs = "pg_catalog_test_expected_diffs.json" // Contains expected difference between postgres and cockroach - testdata = "testdata" // testdata directory + catalogDump = "%s_tables.json" // PostgreSQL pg_catalog schema + expectedDiffs = "%s_test_expected_diffs.json" // Contains expected difference between postgres and cockroach + testdata = "testdata" // testdata directory ) // When running test with -rewrite-diffs test will pass and re-create pg_catalog_test-diffs.json -var rewriteFlag = flag.Bool("rewrite-diffs", false, "This will re-create the expected diffs file") +var ( + rewriteFlag = flag.Bool("rewrite-diffs", false, "This will re-create the expected diffs file") + catalogName = flag.String("catalog", "pg_catalog", "Catalog or namespace, default: pg_catalog") +) // summary will keep accountability for any unexpected difference and report it in the log type summary struct { @@ -77,9 +80,9 @@ func (sum *summary) report(t *testing.T) { } // loadTestData retrieves the pg_catalog from the dumpfile generated from Postgres -func loadTestData(t testing.TB) PGCatalogTables { - var pgCatalogFile PGCatalogFile - testdataFile := filepath.Join(testdata, pgCatalogDump) +func loadTestData(t testing.TB) PGMetadataTables { + var pgCatalogFile PGMetadataFile + testdataFile := filepath.Join(testdata, fmt.Sprintf(catalogDump, *catalogName)) f, err := os.Open(testdataFile) if err != nil { t.Fatal(err) @@ -95,17 +98,17 @@ func loadTestData(t testing.TB) PGCatalogTables { t.Fatal(err) } - return pgCatalogFile.PgCatalog + return pgCatalogFile.PGMetadata } // loadCockroachPgCatalog retrieves pg_catalog schema from cockroach db -func loadCockroachPgCatalog(t testing.TB) PGCatalogTables { - crdbTables := make(PGCatalogTables) +func loadCockroachPgCatalog(t testing.TB) PGMetadataTables { + crdbTables := make(PGMetadataTables) ctx := context.Background() s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(ctx) sqlRunner := sqlutils.MakeSQLRunner(db) - rows := sqlRunner.Query(t, GetPGCatalogSQL) + rows := sqlRunner.Query(t, GetPGMetadataSQL, *catalogName) defer rows.Close() for rows.Next() { @@ -120,15 +123,15 @@ func loadCockroachPgCatalog(t testing.TB) PGCatalogTables { } // loadExpectedDiffs get all differences that will be skipped by the this test -func loadExpectedDiffs(t *testing.T) (diffs PGCatalogTables) { - diffs = PGCatalogTables{} +func loadExpectedDiffs(t *testing.T) (diffs PGMetadataTables) { + diffs = PGMetadataTables{} if *rewriteFlag { // For rewrite we want this to be empty and get populated return } - diffFile := filepath.Join(testdata, expectedDiffs) + diffFile := filepath.Join(testdata, fmt.Sprintf(expectedDiffs, *catalogName)) if _, err := os.Stat(diffFile); err != nil { if oserror.IsNotExist(err) { // File does not exists it means diffs are not expected @@ -160,7 +163,7 @@ func errorf(t *testing.T, format string, args ...interface{}) { } } -func rewriteDiffs(t *testing.T, diffs PGCatalogTables, diffsFile string) { +func rewriteDiffs(t *testing.T, diffs PGMetadataTables, diffsFile string) { if !*rewriteFlag { return } @@ -217,5 +220,5 @@ func TestPGCatalog(t *testing.T) { } sum.report(t) - rewriteDiffs(t, diffs, filepath.Join(testdata, expectedDiffs)) + rewriteDiffs(t, diffs, filepath.Join(testdata, fmt.Sprintf(expectedDiffs, *catalogName))) } diff --git a/pkg/sql/scrub_constraint.go b/pkg/sql/scrub_constraint.go index 098f469fb3f9..845617642f9b 100644 --- a/pkg/sql/scrub_constraint.go +++ b/pkg/sql/scrub_constraint.go @@ -98,7 +98,7 @@ func (o *sqlCheckConstraintCheckOperation) Start(params runParams) error { } } - rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Query( + rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( ctx, "check-constraint", params.p.txn, tree.AsStringWithFlags(sel, tree.FmtParsable), ) if err != nil { diff --git a/pkg/sql/scrub_fk.go b/pkg/sql/scrub_fk.go index e6ead264bb0c..fd856a6004d0 100644 --- a/pkg/sql/scrub_fk.go +++ b/pkg/sql/scrub_fk.go @@ -74,7 +74,7 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { return err } - rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Query( + rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( ctx, "scrub-fk", params.p.txn, checkQuery, ) if err != nil { @@ -93,7 +93,7 @@ func (o *sqlForeignKeyCheckOperation) Start(params runParams) error { if err != nil { return err } - rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Query( + rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( ctx, "scrub-fk", params.p.txn, checkNullsQuery, ) if err != nil { diff --git a/pkg/sql/scrub_index.go b/pkg/sql/scrub_index.go index 1f828eaf1522..fbecb3cd09dc 100644 --- a/pkg/sql/scrub_index.go +++ b/pkg/sql/scrub_index.go @@ -123,7 +123,7 @@ func (o *indexCheckOperation) Start(params runParams) error { colNames(pkColumns), colNames(otherColumns), o.tableDesc.GetID(), o.indexDesc.ID, ) - rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Query( + rows, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryBuffered( ctx, "scrub-index", params.p.txn, checkQuery, ) if err != nil { diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index ea0c50d8bff4..4ccb7c3af45c 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -45,12 +45,14 @@ type comment struct { func selectComment(ctx context.Context, p PlanHookState, tableID descpb.ID) (tc *tableComments) { query := fmt.Sprintf("SELECT type, object_id, sub_id, comment FROM system.comments WHERE object_id = %d", tableID) - commentRows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( + it, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryIterator( ctx, "show-tables-with-comment", p.Txn(), query) if err != nil { log.VEventf(ctx, 1, "%q", err) } else { - for _, row := range commentRows { + var ok bool + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + row := it.Cur() commentType := int(tree.MustBeDInt(row[0])) switch commentType { case keys.TableCommentType, keys.ColumnCommentType, keys.IndexCommentType: @@ -71,6 +73,10 @@ func selectComment(ctx context.Context, p PlanHookState, tableID descpb.ID) (tc } } } + if err != nil { + log.VEventf(ctx, 1, "%q", err) + tc = nil + } } return tc diff --git a/pkg/sql/show_stats.go b/pkg/sql/show_stats.go index ad250baf72b4..f5e74acc7dbe 100644 --- a/pkg/sql/show_stats.go +++ b/pkg/sql/show_stats.go @@ -64,7 +64,9 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p // - convert column IDs to column names // - if the statistic has a histogram, we return the statistic ID as a // "handle" which can be used with SHOW HISTOGRAM. - rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query( + // TODO(yuzefovich): refactor the code to use the iterator API + // (currently it is not possible due to a panic-catcher below). + rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryBuffered( ctx, "read-table-stats", p.txn, diff --git a/pkg/sql/sqlutil/internal_executor.go b/pkg/sql/sqlutil/internal_executor.go index 706483e42009..aa0d2f7d5253 100644 --- a/pkg/sql/sqlutil/internal_executor.go +++ b/pkg/sql/sqlutil/internal_executor.go @@ -84,6 +84,23 @@ type InternalExecutor interface { qargs ...interface{}, ) (tree.Datums, colinfo.ResultColumns, error) + // QueryBuffered executes the supplied SQL statement and returns the + // resulting rows (meaning all of them are buffered at once). If no user has + // been previously set through SetSessionData, the statement is executed as + // the root user. + // + // If txn is not nil, the statement will be executed in the respective txn. + // + // QueryBuffered is deprecated because it may transparently execute a query + // as root. Use QueryBufferedEx instead. + QueryBuffered( + ctx context.Context, + opName string, + txn *kv.Txn, + stmt string, + qargs ...interface{}, + ) ([]tree.Datums, error) + // QueryBufferedEx executes the supplied SQL statement and returns the // resulting rows (meaning all of them are buffered at once). // diff --git a/pkg/sql/stats/stats_cache.go b/pkg/sql/stats/stats_cache.go index 7b03b73bff1d..e7500033bf9a 100644 --- a/pkg/sql/stats/stats_cache.go +++ b/pkg/sql/stats/stats_cache.go @@ -490,7 +490,7 @@ func (sc *TableStatisticsCache) parseStats( // type that doesn't exist) and returns the rest (with no error). func (sc *TableStatisticsCache) getTableStatsFromDB( ctx context.Context, tableID descpb.ID, -) (_ []*TableStatistic, retErr error) { +) ([]*TableStatistic, error) { const getTableStatisticsStmt = ` SELECT "tableID", diff --git a/pkg/sql/testdata/information_schema_tables.json b/pkg/sql/testdata/information_schema_tables.json new file mode 100644 index 000000000000..d42e55ab1599 --- /dev/null +++ b/pkg/sql/testdata/information_schema_tables.json @@ -0,0 +1,4107 @@ +{ + "pgVersion": "13.0", + "pgMetadata": { + "_pg_foreign_data_wrappers": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "fdwoptions": { + "oid": 1009, + "dataType": "_text", + "expectedOid": null, + "expectedDataType": null + }, + "fdwowner": { + "oid": 26, + "dataType": "oid", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_language": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "oid": { + "oid": 26, + "dataType": "oid", + "expectedOid": null, + "expectedDataType": null + } + }, + "_pg_foreign_servers": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_version": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "oid": { + "oid": 26, + "dataType": "oid", + "expectedOid": null, + "expectedDataType": null + }, + "srvoptions": { + "oid": 1009, + "dataType": "_text", + "expectedOid": null, + "expectedDataType": null + } + }, + "_pg_foreign_table_columns": { + "attfdwoptions": { + "oid": 1009, + "dataType": "_text", + "expectedOid": null, + "expectedDataType": null + }, + "attname": { + "oid": 19, + "dataType": "name", + "expectedOid": null, + "expectedDataType": null + }, + "nspname": { + "oid": 19, + "dataType": "name", + "expectedOid": null, + "expectedDataType": null + }, + "relname": { + "oid": 19, + "dataType": "name", + "expectedOid": null, + "expectedDataType": null + } + }, + "_pg_foreign_tables": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "ftoptions": { + "oid": 1009, + "dataType": "_text", + "expectedOid": null, + "expectedDataType": null + } + }, + "_pg_user_mappings": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "oid": { + "oid": 26, + "dataType": "oid", + "expectedOid": null, + "expectedDataType": null + }, + "srvowner": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "umoptions": { + "oid": 1009, + "dataType": "_text", + "expectedOid": null, + "expectedDataType": null + }, + "umuser": { + "oid": 26, + "dataType": "oid", + "expectedOid": null, + "expectedDataType": null + } + }, + "administrable_role_authorizations": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "role_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "applicable_roles": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "role_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "attributes": { + "attribute_default": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "attribute_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "attribute_udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "attribute_udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "attribute_udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_derived_reference_attribute": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_nullable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "ordinal_position": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "character_sets": { + "character_repertoire": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_collate_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_collate_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_collate_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "form_of_use": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "check_constraint_routine_usage": { + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "check_constraints": { + "check_clause": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "collation_character_set_applicability": { + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "collations": { + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "pad_attribute": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "column_column_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "dependent_column": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "column_domain_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "column_options": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "column_privileges": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "column_udt_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "columns": { + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "column_default": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "domain_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "generation_expression": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "identity_cycle": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "identity_generation": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "identity_increment": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "identity_maximum": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "identity_minimum": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "identity_start": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_generated": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_identity": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_nullable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_self_referencing": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_updatable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "ordinal_position": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "constraint_column_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "constraint_table_usage": { + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "data_type_privileges": { + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "domain_constraints": { + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "initially_deferred": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_deferrable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + } + }, + "domain_udt_usage": { + "domain_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "domains": { + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "domain_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_default": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "domain_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "domain_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "element_types": { + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collection_type_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "domain_default": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "enabled_roles": { + "role_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_data_wrapper_options": { + "foreign_data_wrapper_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_data_wrappers": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_language": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "library_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_server_options": { + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_servers": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_data_wrapper_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_version": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_table_options": { + "foreign_table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "foreign_tables": { + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "information_schema_catalog_name": { + "catalog_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "key_column_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "ordinal_position": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "position_in_unique_constraint": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "parameters": { + "as_locator": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_result": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "ordinal_position": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "parameter_default": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "parameter_mode": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "parameter_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "referential_constraints": { + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "delete_rule": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "match_option": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "unique_constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "unique_constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "unique_constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "update_rule": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "role_column_grants": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "role_routine_grants": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "routine_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "role_table_grants": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "with_hierarchy": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + } + }, + "role_udt_grants": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "role_usage_grants": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "routine_privileges": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "routine_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "routines": { + "as_locator": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "created": { + "oid": 13448, + "dataType": "time_stamp", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "external_language": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "external_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_deterministic": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_implicitly_invocable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_null_call": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_udt_dependent": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_user_defined_cast": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "last_altered": { + "oid": 13448, + "dataType": "time_stamp", + "expectedOid": null, + "expectedDataType": null + }, + "max_dynamic_result_sets": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "module_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "module_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "module_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "new_savepoint_level": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "parameter_style": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_as_locator": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_char_max_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_char_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_char_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_char_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_char_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_from_data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_maximum_cardinality": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_type_udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_type_udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "result_cast_type_udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_body": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "routine_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_definition": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "routine_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "routine_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "schema_level_routine": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "scope_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "scope_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "security_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "sql_data_access": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "sql_path": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "to_sql_specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "to_sql_specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "to_sql_specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "type_udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "type_udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "type_udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "schemata": { + "catalog_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "default_character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "schema_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "schema_owner": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "sql_path": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "sequences": { + "cycle_option": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "increment": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "maximum_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "minimum_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "sequence_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "sequence_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "sequence_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "start_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "sql_features": { + "comments": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "feature_id": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "feature_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_supported": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_verified_by": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "sub_feature_id": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "sub_feature_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "sql_implementation_info": { + "character_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "comments": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "implementation_info_id": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "implementation_info_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "integer_value": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + } + }, + "sql_parts": { + "comments": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "feature_id": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "feature_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_supported": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_verified_by": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "sql_sizing": { + "comments": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "sizing_id": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "sizing_name": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "supported_value": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + } + }, + "table_constraints": { + "constraint_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "constraint_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "enforced": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "initially_deferred": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_deferrable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "table_privileges": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "with_hierarchy": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + } + }, + "tables": { + "commit_action": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_insertable_into": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_typed": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "reference_generation": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "self_referencing_column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "transforms": { + "group_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "transform_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "triggered_update_columns": { + "event_object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_column": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_table": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "triggers": { + "action_condition": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "action_order": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "action_orientation": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "action_reference_new_row": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "action_reference_new_table": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "action_reference_old_row": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "action_reference_old_table": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "action_statement": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "action_timing": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "created": { + "oid": 13448, + "dataType": "time_stamp", + "expectedOid": null, + "expectedDataType": null + }, + "event_manipulation": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "event_object_table": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "trigger_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "udt_privileges": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "udt_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "udt_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "usage_privileges": { + "grantee": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "grantor": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "is_grantable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "object_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "object_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "privilege_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "user_defined_types": { + "character_maximum_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_octet_length": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "character_set_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "collation_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "data_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "datetime_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "interval_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_final": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_instantiable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_precision_radix": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "numeric_scale": { + "oid": 13438, + "dataType": "cardinal_number", + "expectedOid": null, + "expectedDataType": null + }, + "ordering_category": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "ordering_form": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "ordering_routine_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "ordering_routine_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "ordering_routine_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "ref_dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "reference_type": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "source_dtd_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_category": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "user_defined_type_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "user_mapping_options": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "option_value": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + }, + "user_mappings": { + "authorization_identifier": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "foreign_server_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "view_column_usage": { + "column_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "view_routine_usage": { + "specific_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "specific_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "view_table_usage": { + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + } + }, + "views": { + "check_option": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + }, + "is_insertable_into": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_trigger_deletable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_trigger_insertable_into": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_trigger_updatable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "is_updatable": { + "oid": 13450, + "dataType": "yes_or_no", + "expectedOid": null, + "expectedDataType": null + }, + "table_catalog": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_name": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "table_schema": { + "oid": 13443, + "dataType": "sql_identifier", + "expectedOid": null, + "expectedDataType": null + }, + "view_definition": { + "oid": 13441, + "dataType": "character_data", + "expectedOid": null, + "expectedDataType": null + } + } + } +} \ No newline at end of file diff --git a/pkg/sql/testdata/information_schema_test_expected_diffs.json b/pkg/sql/testdata/information_schema_test_expected_diffs.json new file mode 100644 index 000000000000..ed4820aa3e83 --- /dev/null +++ b/pkg/sql/testdata/information_schema_test_expected_diffs.json @@ -0,0 +1,1793 @@ +{ + "_pg_foreign_data_wrappers": {}, + "_pg_foreign_servers": {}, + "_pg_foreign_table_columns": {}, + "_pg_foreign_tables": {}, + "_pg_user_mappings": {}, + "administrable_role_authorizations": { + "grantee": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "is_grantable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "role_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "applicable_roles": { + "grantee": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "is_grantable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "role_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "attributes": {}, + "character_sets": { + "character_repertoire": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "default_collate_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "default_collate_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "default_collate_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "form_of_use": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "check_constraint_routine_usage": {}, + "check_constraints": { + "check_clause": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "collation_character_set_applicability": { + "character_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "collations": { + "collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "pad_attribute": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + } + }, + "column_column_usage": {}, + "column_domain_usage": {}, + "column_options": {}, + "column_privileges": { + "column_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "grantee": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "grantor": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "is_grantable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "privilege_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "column_udt_usage": { + "column_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "columns": { + "character_maximum_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_octet_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "column_default": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "column_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "data_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "datetime_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "domain_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "domain_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "domain_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "dtd_identifier": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "generation_expression": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "identity_cycle": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "identity_generation": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "identity_increment": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "identity_maximum": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "identity_minimum": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "identity_start": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "interval_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "interval_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "is_generated": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "is_identity": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_nullable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_self_referencing": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_updatable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "maximum_cardinality": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision_radix": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_scale": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "ordinal_position": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "scope_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "constraint_column_usage": { + "column_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "constraint_table_usage": {}, + "data_type_privileges": {}, + "domain_constraints": {}, + "domain_udt_usage": {}, + "domains": {}, + "element_types": {}, + "enabled_roles": { + "role_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "foreign_data_wrapper_options": {}, + "foreign_data_wrappers": {}, + "foreign_server_options": {}, + "foreign_servers": {}, + "foreign_table_options": {}, + "foreign_tables": {}, + "information_schema_catalog_name": {}, + "key_column_usage": { + "column_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "ordinal_position": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "position_in_unique_constraint": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "parameters": { + "as_locator": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "character_maximum_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_octet_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "data_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "datetime_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "dtd_identifier": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "interval_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "interval_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "is_result": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "maximum_cardinality": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision_radix": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_scale": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "ordinal_position": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "parameter_default": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "parameter_mode": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "parameter_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "specific_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "specific_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "specific_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "referential_constraints": { + "constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "delete_rule": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "match_option": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "unique_constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "unique_constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "unique_constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "update_rule": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + } + }, + "role_column_grants": {}, + "role_routine_grants": {}, + "role_table_grants": { + "grantee": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "grantor": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "is_grantable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "privilege_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "with_hierarchy": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + } + }, + "role_udt_grants": {}, + "role_usage_grants": {}, + "routine_privileges": {}, + "routines": { + "as_locator": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "character_maximum_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_octet_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "character_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "character_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "created": { + "oid": 1184, + "dataType": "timestamptz", + "expectedOid": 13448, + "expectedDataType": "time_stamp" + }, + "data_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "datetime_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "dtd_identifier": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "external_language": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "external_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "interval_precision": { + "oid": 25, + "dataType": "text", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "interval_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "is_deterministic": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_implicitly_invocable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_null_call": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_udt_dependent": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_user_defined_cast": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "last_altered": { + "oid": 1184, + "dataType": "timestamptz", + "expectedOid": 13448, + "expectedDataType": "time_stamp" + }, + "max_dynamic_result_sets": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "maximum_cardinality": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "module_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "module_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "module_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "new_savepoint_level": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "numeric_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision_radix": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_scale": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "parameter_style": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "result_cast_as_locator": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "result_cast_char_max_length": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_char_octet_length": { + "oid": 25, + "dataType": "text", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_char_set_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_char_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_char_set_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_collation_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_collation_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_collation_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_datetime_precision": { + "oid": 25, + "dataType": "text", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_dtd_identifier": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_from_data_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "result_cast_interval_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_interval_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "result_cast_maximum_cardinality": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_numeric_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_numeric_precision_radix": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_numeric_scale": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "result_cast_scope_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_scope_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_scope_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_type_udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_type_udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "result_cast_type_udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "routine_body": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "routine_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "routine_definition": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "routine_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "routine_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "routine_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "schema_level_routine": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "scope_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "scope_schema": null, + "security_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "specific_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "specific_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "specific_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "sql_data_access": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "sql_path": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "to_sql_specific_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "to_sql_specific_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "to_sql_specific_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "type_udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "type_udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "type_udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "udt_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "schemata": { + "catalog_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "default_character_set_catalog": null, + "default_character_set_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "default_character_set_schema": null, + "schema_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "schema_owner": null, + "sql_path": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + } + }, + "sequences": { + "cycle_option": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "data_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "increment": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "maximum_value": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "minimum_value": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "numeric_precision": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_precision_radix": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "numeric_scale": { + "oid": 20, + "dataType": "int8", + "expectedOid": 13438, + "expectedDataType": "cardinal_number" + }, + "sequence_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "sequence_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "sequence_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "start_value": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + } + }, + "sql_features": {}, + "sql_implementation_info": {}, + "sql_parts": {}, + "sql_sizing": {}, + "table_constraints": { + "constraint_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "constraint_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "enforced": null, + "initially_deferred": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_deferrable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + } + }, + "table_privileges": { + "grantee": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "grantor": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "is_grantable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "privilege_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "with_hierarchy": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + } + }, + "tables": { + "commit_action": null, + "is_insertable_into": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_typed": null, + "reference_generation": null, + "self_referencing_column_name": null, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_type": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "user_defined_type_catalog": null, + "user_defined_type_name": null, + "user_defined_type_schema": null + }, + "transforms": {}, + "triggered_update_columns": {}, + "triggers": {}, + "udt_privileges": {}, + "usage_privileges": {}, + "user_defined_types": {}, + "user_mapping_options": {}, + "user_mappings": {}, + "view_column_usage": {}, + "view_routine_usage": {}, + "view_table_usage": {}, + "views": { + "check_option": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + }, + "is_insertable_into": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_trigger_deletable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_trigger_insertable_into": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_trigger_updatable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "is_updatable": { + "oid": 25, + "dataType": "text", + "expectedOid": 13450, + "expectedDataType": "yes_or_no" + }, + "table_catalog": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_name": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "table_schema": { + "oid": 25, + "dataType": "text", + "expectedOid": 13443, + "expectedDataType": "sql_identifier" + }, + "view_definition": { + "oid": 25, + "dataType": "text", + "expectedOid": 13441, + "expectedDataType": "character_data" + } + } +} \ No newline at end of file diff --git a/pkg/sql/testdata/pg_catalog_tables.json b/pkg/sql/testdata/pg_catalog_tables.json index a7b928db5b94..1938acbe1f8d 100644 --- a/pkg/sql/testdata/pg_catalog_tables.json +++ b/pkg/sql/testdata/pg_catalog_tables.json @@ -1,6 +1,6 @@ { "pgVersion": "13.0", - "pgCatalog": { + "pgMetadata": { "pg_aggregate": { "aggcombinefn": { "oid": 24,