diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index b3048b0c3b2f..8ad614711543 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -2847,6 +2847,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)
pg_column_size(anyelement...) → int | Return size in bytes of the column provided as an argument
|
pg_sleep(seconds: float) → bool | pg_sleep makes the current session’s process sleep until seconds seconds have elapsed. seconds is a value of type double precision, so fractional-second delays can be specified.
+ |
+pg_table_is_visible(oid: oid) → bool | Returns whether the table with the given OID belongs to one of the schemas on the search path.
|
diff --git a/pkg/bench/ddl_analysis/BUILD.bazel b/pkg/bench/ddl_analysis/BUILD.bazel
index 3b90a281b303..247057896493 100644
--- a/pkg/bench/ddl_analysis/BUILD.bazel
+++ b/pkg/bench/ddl_analysis/BUILD.bazel
@@ -26,6 +26,7 @@ go_test(
"drop_bench_test.go",
"grant_revoke_bench_test.go",
"grant_revoke_role_bench_test.go",
+ "orm_queries_bench_test.go",
"system_bench_test.go",
"truncate_bench_test.go",
"validate_benchmark_data_test.go",
diff --git a/pkg/bench/ddl_analysis/orm_queries_bench_test.go b/pkg/bench/ddl_analysis/orm_queries_bench_test.go
new file mode 100644
index 000000000000..f7e2730a4163
--- /dev/null
+++ b/pkg/bench/ddl_analysis/orm_queries_bench_test.go
@@ -0,0 +1,86 @@
+// Copyright 2021 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+package bench
+
+import "testing"
+
+func BenchmarkORMQueries(b *testing.B) {
+ tests := []RoundTripBenchTestCase{
+ {
+ name: "django table introspection 1 table",
+ setup: `CREATE TABLE t1(a int primary key, b int);`,
+ stmt: `SELECT
+ a.attname AS column_name,
+ NOT (a.attnotnull OR ((t.typtype = 'd') AND t.typnotnull)) AS is_nullable,
+ pg_get_expr(ad.adbin, ad.adrelid) AS column_default
+FROM pg_attribute AS a
+LEFT JOIN pg_attrdef AS ad ON (a.attrelid = ad.adrelid) AND (a.attnum = ad.adnum)
+JOIN pg_type AS t ON a.atttypid = t.oid JOIN pg_class AS c ON a.attrelid = c.oid
+JOIN pg_namespace AS n ON c.relnamespace = n.oid
+WHERE (
+ (
+ (c.relkind IN ('f', 'm', 'p', 'r', 'v')) AND
+ (c.relname = '')
+ ) AND (n.nspname NOT IN ('pg_catalog', 'pg_toast'))
+) AND pg_table_is_visible(c.oid)`,
+ },
+
+ {
+ name: "django table introspection 4 tables",
+ setup: `CREATE TABLE t1(a int primary key, b int);
+CREATE TABLE t2(a int primary key, b int);
+CREATE TABLE t3(a int primary key, b int);
+CREATE TABLE t4(a int primary key, b int);`,
+ stmt: `SELECT
+ a.attname AS column_name,
+ NOT (a.attnotnull OR ((t.typtype = 'd') AND t.typnotnull)) AS is_nullable,
+ pg_get_expr(ad.adbin, ad.adrelid) AS column_default
+FROM pg_attribute AS a
+LEFT JOIN pg_attrdef AS ad ON (a.attrelid = ad.adrelid) AND (a.attnum = ad.adnum)
+JOIN pg_type AS t ON a.atttypid = t.oid JOIN pg_class AS c ON a.attrelid = c.oid
+JOIN pg_namespace AS n ON c.relnamespace = n.oid
+WHERE (
+ (
+ (c.relkind IN ('f', 'm', 'p', 'r', 'v')) AND
+ (c.relname = '')
+ ) AND (n.nspname NOT IN ('pg_catalog', 'pg_toast'))
+) AND pg_table_is_visible(c.oid)`,
+ },
+
+ {
+ name: "django table introspection 8 tables",
+ setup: `CREATE TABLE t1(a int primary key, b int);
+CREATE TABLE t2(a int primary key, b int);
+CREATE TABLE t3(a int primary key, b int);
+CREATE TABLE t4(a int primary key, b int);
+CREATE TABLE t5(a int primary key, b int);
+CREATE TABLE t6(a int primary key, b int);
+CREATE TABLE t7(a int primary key, b int);
+CREATE TABLE t8(a int primary key, b int);`,
+ stmt: `SELECT
+ a.attname AS column_name,
+ NOT (a.attnotnull OR ((t.typtype = 'd') AND t.typnotnull)) AS is_nullable,
+ pg_get_expr(ad.adbin, ad.adrelid) AS column_default
+FROM pg_attribute AS a
+LEFT JOIN pg_attrdef AS ad ON (a.attrelid = ad.adrelid) AND (a.attnum = ad.adnum)
+JOIN pg_type AS t ON a.atttypid = t.oid JOIN pg_class AS c ON a.attrelid = c.oid
+JOIN pg_namespace AS n ON c.relnamespace = n.oid
+WHERE (
+ (
+ (c.relkind IN ('f', 'm', 'p', 'r', 'v')) AND
+ (c.relname = '')
+ ) AND (n.nspname NOT IN ('pg_catalog', 'pg_toast'))
+) AND pg_table_is_visible(c.oid)`,
+ },
+ }
+
+ RunRoundTripBenchmark(b, tests)
+}
diff --git a/pkg/bench/ddl_analysis/testdata/benchmark_expectations b/pkg/bench/ddl_analysis/testdata/benchmark_expectations
index 23b91c4285b5..c50e956e4008 100644
--- a/pkg/bench/ddl_analysis/testdata/benchmark_expectations
+++ b/pkg/bench/ddl_analysis/testdata/benchmark_expectations
@@ -68,3 +68,6 @@ min max benchmark
2 2 SystemDatabaseQueries/select_system.users_without_schema_name
1 1 SystemDatabaseQueries/select_system.users_with_empty_database_name
1 1 SystemDatabaseQueries/select_system.users_with_schema_name
+2 2 ORMQueries/django_table_introspection_1_table
+2 2 ORMQueries/django_table_introspection_4_tables
+2 2 ORMQueries/django_table_introspection_8_tables
diff --git a/pkg/ccl/importccl/import_table_creation.go b/pkg/ccl/importccl/import_table_creation.go
index 30db38ef49e5..6d2fa0eb977d 100644
--- a/pkg/ccl/importccl/import_table_creation.go
+++ b/pkg/ccl/importccl/import_table_creation.go
@@ -232,6 +232,13 @@ func (so *importSequenceOperators) LookupSchema(
return false, nil, errSequenceOperators
}
+// IsTableVisible is part of the tree.EvalDatabase interface.
+func (so *importSequenceOperators) IsTableVisible(
+ ctx context.Context, curDB string, searchPath sessiondata.SearchPath, tableID int64,
+) (bool, bool, error) {
+ return false, false, errors.WithStack(errSequenceOperators)
+}
+
// Implements the tree.SequenceOperators interface.
func (so *importSequenceOperators) IncrementSequence(
ctx context.Context, seqName *tree.TableName,
diff --git a/pkg/sql/faketreeeval/BUILD.bazel b/pkg/sql/faketreeeval/BUILD.bazel
index ca6c649f1c41..5f3267f9d445 100644
--- a/pkg/sql/faketreeeval/BUILD.bazel
+++ b/pkg/sql/faketreeeval/BUILD.bazel
@@ -13,6 +13,7 @@ go_library(
"//pkg/sql/pgwire/pgnotice",
"//pkg/sql/roleoption",
"//pkg/sql/sem/tree",
+ "//pkg/sql/sessiondata",
"//pkg/sql/types",
"//pkg/util/errorutil/unimplemented",
"@com_github_cockroachdb_errors//:errors",
diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go
index 6c0c996ac63a..ef5f45b491ec 100644
--- a/pkg/sql/faketreeeval/evalctx.go
+++ b/pkg/sql/faketreeeval/evalctx.go
@@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
@@ -62,6 +63,13 @@ func (so *DummySequenceOperators) LookupSchema(
return false, nil, errors.WithStack(errSequenceOperators)
}
+// IsTableVisible is part of the tree.EvalDatabase interface.
+func (so *DummySequenceOperators) IsTableVisible(
+ ctx context.Context, curDB string, searchPath sessiondata.SearchPath, tableID int64,
+) (bool, bool, error) {
+ return false, false, errors.WithStack(errSequenceOperators)
+}
+
// IncrementSequence is part of the tree.SequenceOperators interface.
func (so *DummySequenceOperators) IncrementSequence(
ctx context.Context, seqName *tree.TableName,
@@ -169,6 +177,13 @@ func (ep *DummyEvalPlanner) LookupSchema(
return false, nil, errors.WithStack(errEvalPlanner)
}
+// IsTableVisible is part of the tree.EvalDatabase interface.
+func (ep *DummyEvalPlanner) IsTableVisible(
+ ctx context.Context, curDB string, searchPath sessiondata.SearchPath, tableID int64,
+) (bool, bool, error) {
+ return false, false, errors.WithStack(errEvalPlanner)
+}
+
// ResolveTableName is part of the tree.EvalDatabase interface.
func (ep *DummyEvalPlanner) ResolveTableName(
ctx context.Context, tn *tree.TableName,
diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins
index 88153987d0f6..1a9bd2d6dafa 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_builtins
+++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins
@@ -80,3 +80,50 @@ query I
SELECT pg_column_size(NULL::int)
----
NULL
+
+statement ok
+CREATE TABLE is_visible(a int primary key);
+CREATE SCHEMA other;
+CREATE TABLE other.not_visible(a int primary key);
+CREATE DATABASE db2;
+SET DATABASE = db2;
+CREATE TABLE table_in_db2(a int primary key);
+
+let $table_in_db2_id
+SELECT c.oid FROM pg_class c WHERE c.relname = 'table_in_db2';
+
+statement ok
+SET DATABASE = test;
+
+query B
+SELECT pg_table_is_visible(c.oid)
+FROM pg_class c
+WHERE c.relname = 'is_visible'
+----
+true
+
+query B
+SELECT pg_table_is_visible(c.oid)
+FROM pg_class c
+WHERE c.relname = 'pg_type'
+----
+true
+
+query B
+SELECT pg_table_is_visible(c.oid)
+FROM pg_class c
+WHERE c.relname = 'not_visible'
+----
+false
+
+# Looking up a table in a different database should return NULL.
+query B
+SELECT pg_table_is_visible($table_in_db2_id)
+----
+NULL
+
+# Looking up a non-existent OID should return NULL.
+query B
+SELECT pg_table_is_visible(1010101010)
+----
+NULL
diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go
index 45a972fd8d99..1ffa200e48b8 100644
--- a/pkg/sql/resolver.go
+++ b/pkg/sql/resolver.go
@@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
@@ -205,6 +206,48 @@ func (p *planner) CommonLookupFlags(required bool) tree.CommonLookupFlags {
}
}
+// IsTableVisible is part of the tree.EvalDatabase interface.
+func (p *planner) IsTableVisible(
+ ctx context.Context, curDB string, searchPath sessiondata.SearchPath, tableID int64,
+) (isVisible, exists bool, err error) {
+ tableDesc, err := p.LookupTableByID(ctx, descpb.ID(tableID))
+ if err != nil {
+ // If an error happened here, it means the table doesn't exist, so we
+ // return "not exists" rather than the error.
+ return false, false, nil //nolint:returnerrcheck
+ }
+ schemaID := tableDesc.GetParentSchemaID()
+ schemaDesc, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.Txn(), schemaID,
+ tree.SchemaLookupFlags{
+ Required: true,
+ AvoidCached: p.avoidCachedDescriptors})
+ if err != nil {
+ return false, false, err
+ }
+ if schemaDesc.Kind != catalog.SchemaVirtual {
+ dbID := tableDesc.GetParentID()
+ dbDesc, err := p.Descriptors().GetImmutableDatabaseByID(ctx, p.Txn(), dbID,
+ tree.DatabaseLookupFlags{
+ Required: true,
+ AvoidCached: p.avoidCachedDescriptors})
+ if err != nil {
+ return false, false, err
+ }
+ if dbDesc.Name != curDB {
+ // If the table is in a different database, then it's considered to be
+ // "not existing" instead of just "not visible"; this matches PostgreSQL.
+ return false, false, nil
+ }
+ }
+ iter := searchPath.Iter()
+ for scName, ok := iter.Next(); ok; scName, ok = iter.Next() {
+ if schemaDesc.Name == scName {
+ return true, true, nil
+ }
+ }
+ return false, true, nil
+}
+
// GetTypeDescriptor implements the descpb.TypeDescriptorResolver interface.
func (p *planner) GetTypeDescriptor(
ctx context.Context, id descpb.ID,
diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go
index 33aa4c26ce6b..b9e84eb38558 100644
--- a/pkg/sql/sem/builtins/pg_builtins.go
+++ b/pkg/sql/sem/builtins/pg_builtins.go
@@ -1077,25 +1077,26 @@ SELECT description
},
),
// pg_table_is_visible returns true if the input oid corresponds to a table
- // that is part of the databases on the search path.
+ // that is part of the schemas on the search path.
// https://www.postgresql.org/docs/9.6/static/functions-info.html
"pg_table_is_visible": makeBuiltin(defProps(),
tree.Overload{
Types: tree.ArgTypes{{"oid", types.Oid}},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
- oid := args[0]
- t, err := ctx.InternalExecutor.QueryRow(
- ctx.Ctx(), "pg_table_is_visible",
- ctx.Txn,
- "SELECT nspname FROM pg_catalog.pg_class c JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid "+
- "WHERE c.oid=$1 AND nspname=ANY(current_schemas(true))", oid)
+ oid := tree.MustBeDOid(args[0])
+ isVisible, exists, err := ctx.Planner.IsTableVisible(
+ ctx.Context, ctx.SessionData.Database, ctx.SessionData.SearchPath, int64(oid.DInt),
+ )
if err != nil {
return nil, err
}
- return tree.MakeDBool(tree.DBool(t != nil)), nil
+ if !exists {
+ return tree.DNull, nil
+ }
+ return tree.MakeDBool(tree.DBool(isVisible)), nil
},
- Info: notUsableInfo,
+ Info: "Returns whether the table with the given OID belongs to one of the schemas on the search path.",
Volatility: tree.VolatilityStable,
},
),
diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go
index b3580ad82166..a3174c727811 100644
--- a/pkg/sql/sem/tree/eval.go
+++ b/pkg/sql/sem/tree/eval.go
@@ -3021,6 +3021,15 @@ type EvalDatabase interface {
// LookupSchema looks up the schema with the given name in the given
// database.
LookupSchema(ctx context.Context, dbName, scName string) (found bool, scMeta SchemaMeta, err error)
+
+ // IsTableVisible checks if the table with the given ID belongs to a schema
+ // on the given sessiondata.SearchPath.
+ IsTableVisible(
+ ctx context.Context,
+ curDB string,
+ searchPath sessiondata.SearchPath,
+ tableID int64,
+ ) (isVisible bool, exists bool, err error)
}
// EvalPlanner is a limited planner that can be used from EvalContext.