Skip to content

Commit

Permalink
Merge #90657 #92957
Browse files Browse the repository at this point in the history
90657: builtins: fix pg_function_is_visible to work with UDFs r=e-mbrown a=rafiss

fixes #89546

### builtins: fix pg_function_is_visible to work with UDFs

Release note (bug fix): The pg_function_is_visible function now
correctly reports visibility based on the functions that are visible on
the current search_path.

### builtins: use pg_type to implement pg_type_is_visible

This saves us from having to maintain the old code, which I personally
found to be a bit hard to work with. Using the internal executor like
this should be faster than it used to be, since there is an index on
pg_type(oid) that will avoid any lookups for builtin types.

### sessiondata: consolidate logic for searching the search_path 

92957: sql: enable storage for tsvector/tsquery r=jordanlewis a=jordanlewis

This commit adds the ability to store tsvector and tsquery data in ordinary, unindexed columns.

Updates #41288
This functionality is gated behind the 23.1 version.

Epic: CRDB-22357

Release note (sql change): permit non-indexed storage of tsvector and tsquery datatypes

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Jordan Lewis <[email protected]>
  • Loading branch information
3 people committed Dec 13, 2022
3 parents 1ee3989 + 9d330c7 + c6cb646 commit 16c786f
Show file tree
Hide file tree
Showing 21 changed files with 229 additions and 138 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3440,6 +3440,8 @@ table. Returns an error if validation fails.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="pg_column_size"></a><code>pg_column_size(anyelement...) &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>Return size in bytes of the column provided as an argument</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="pg_function_is_visible"></a><code>pg_function_is_visible(oid: oid) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>Returns whether the function with the given OID belongs to one of the schemas on the search path.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="pg_get_function_identity_arguments"></a><code>pg_get_function_identity_arguments(func_oid: oid) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the argument list (without defaults) necessary to identify a function, in the form it would need to appear in within ALTER FUNCTION, for instance.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="pg_get_function_result"></a><code>pg_get_function_result(func_oid: oid) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Returns the types of the result of the specified function.</p>
Expand Down
13 changes: 8 additions & 5 deletions pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.Enc
return rows, nil
}

func parseAvroSchema(t *testing.T, j string) (*avroDataRecord, error) {
func parseAvroSchema(t *testing.T, evalCtx *eval.Context, j string) (*avroDataRecord, error) {
var s avroDataRecord
if err := json.Unmarshal([]byte(j), &s); err != nil {
return nil, err
Expand All @@ -145,7 +145,7 @@ func parseAvroSchema(t *testing.T, j string) (*avroDataRecord, error) {
// They're needed for serialization/deserialization, so fake out a
// column descriptor so that we can reuse columnToAvroSchema to get
// all the various fields of avroSchemaField populated for free.
colDesc, err := avroFieldMetadataToColDesc(f.Metadata)
colDesc, err := avroFieldMetadataToColDesc(evalCtx, f.Metadata)
if err != nil {
return nil, err
}
Expand All @@ -167,15 +167,17 @@ func parseAvroSchema(t *testing.T, j string) (*avroDataRecord, error) {
), "", "")
}

func avroFieldMetadataToColDesc(metadata string) (*descpb.ColumnDescriptor, error) {
func avroFieldMetadataToColDesc(
evalCtx *eval.Context, metadata string,
) (*descpb.ColumnDescriptor, error) {
parsed, err := parser.ParseOne(`ALTER TABLE FOO ADD COLUMN ` + metadata)
if err != nil {
return nil, err
}
def := parsed.AST.(*tree.AlterTable).Cmds[0].(*tree.AlterTableAddColumn).ColumnDef
ctx := context.Background()
semaCtx := makeTestSemaCtx()
cdd, err := tabledesc.MakeColumnDefDescs(ctx, def, &semaCtx, &eval.Context{})
cdd, err := tabledesc.MakeColumnDefDescs(ctx, def, &semaCtx, evalCtx)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -382,6 +384,7 @@ func TestAvroSchema(t *testing.T) {
tests = append(tests, randTypeTest)
}

evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
tableDesc, err := parseTableDesc(
Expand All @@ -392,7 +395,7 @@ func TestAvroSchema(t *testing.T) {
avroSchemaNoSuffix, "")
require.NoError(t, err)
jsonSchema := origSchema.codec.Schema()
roundtrippedSchema, err := parseAvroSchema(t, jsonSchema)
roundtrippedSchema, err := parseAvroSchema(t, evalCtx, jsonSchema)
require.NoError(t, err)
// It would require some work, but we could also check that the
// roundtrippedSchema can be used to recreate the original `CREATE
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func AlterColumnType(
}
}

err = colinfo.ValidateColumnDefType(typ)
err = colinfo.ValidateColumnDefType(ctx, params.EvalContext().Settings.Version, typ)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/colinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catpb",
"//pkg/sql/catalog/descpb",
Expand Down
12 changes: 10 additions & 2 deletions pkg/sql/catalog/colinfo/col_type_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,10 @@
package colinfo

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -68,7 +70,7 @@ func (ti ColTypeInfo) Type(idx int) *types.T {

// ValidateColumnDefType returns an error if the type of a column definition is
// not valid. It is checked when a column is created or altered.
func ValidateColumnDefType(t *types.T) error {
func ValidateColumnDefType(ctx context.Context, version clusterversion.Handle, t *types.T) error {
switch t.Family() {
case types.StringFamily, types.CollatedStringFamily:
if t.Family() == types.CollatedStringFamily {
Expand Down Expand Up @@ -100,7 +102,7 @@ func ValidateColumnDefType(t *types.T) error {
if err := types.CheckArrayElementType(t.ArrayContents()); err != nil {
return err
}
return ValidateColumnDefType(t.ArrayContents())
return ValidateColumnDefType(ctx, version, t.ArrayContents())

case types.BitFamily, types.IntFamily, types.FloatFamily, types.BoolFamily, types.BytesFamily, types.DateFamily,
types.INetFamily, types.IntervalFamily, types.JsonFamily, types.OidFamily, types.TimeFamily,
Expand All @@ -116,6 +118,12 @@ func ValidateColumnDefType(t *types.T) error {
return unimplemented.NewWithIssue(70099, "cannot use table record type as table column")
}

case types.TSQueryFamily, types.TSVectorFamily:
if !version.IsActive(ctx, clusterversion.V23_1) {
return pgerror.Newf(pgcode.FeatureNotSupported,
"TSVector/TSQuery not supported until version 23.1")
}

default:
return pgerror.Newf(pgcode.InvalidTableDefinition,
"value type %s cannot be used for table columns", t.String())
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/tabledesc/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ func MakeColumnDefDescs(
if err != nil {
return nil, err
}
if err = colinfo.ValidateColumnDefType(resType); err != nil {
if err = colinfo.ValidateColumnDefType(ctx, evalCtx.Settings.Version, resType); err != nil {
return nil, err
}
col.Type = resType
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/crdb_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
Expand Down Expand Up @@ -223,7 +224,8 @@ CREATE TABLE t.test (k INT);
t.Fatal(err)
}
colDef := alterCmd.AST.(*tree.AlterTable).Cmds[0].(*tree.AlterTableAddColumn).ColumnDef
cdd, err := tabledesc.MakeColumnDefDescs(ctx, colDef, nil, nil)
evalCtx := eval.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
cdd, err := tabledesc.MakeColumnDefDescs(ctx, colDef, nil, evalCtx)
if err != nil {
t.Fatal(err)
}
Expand Down
14 changes: 0 additions & 14 deletions pkg/sql/faketreeeval/evalctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,13 +68,6 @@ func (so *DummySequenceOperators) SchemaExists(
return false, errors.WithStack(errSequenceOperators)
}

// IsTypeVisible is part of the eval.DatabaseCatalog interface.
func (so *DummySequenceOperators) IsTypeVisible(
ctx context.Context, curDB string, searchPath sessiondata.SearchPath, typeID oid.Oid,
) (bool, bool, error) {
return false, false, errors.WithStack(errEvalPlanner)
}

// HasAnyPrivilege is part of the eval.DatabaseCatalog interface.
func (so *DummySequenceOperators) HasAnyPrivilege(
ctx context.Context,
Expand Down Expand Up @@ -365,13 +358,6 @@ func (ep *DummyEvalPlanner) SchemaExists(ctx context.Context, dbName, scName str
return false, errors.WithStack(errEvalPlanner)
}

// IsTypeVisible is part of the eval.DatabaseCatalog interface.
func (ep *DummyEvalPlanner) IsTypeVisible(
ctx context.Context, curDB string, searchPath sessiondata.SearchPath, typeID oid.Oid,
) (bool, bool, error) {
return false, false, errors.WithStack(errEvalPlanner)
}

// HasAnyPrivilege is part of the eval.DatabaseCatalog interface.
func (ep *DummyEvalPlanner) HasAnyPrivilege(
ctx context.Context,
Expand Down
7 changes: 0 additions & 7 deletions pkg/sql/importer/import_table_creation.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,13 +303,6 @@ func (so *importSequenceOperators) SchemaExists(
return false, errSequenceOperators
}

// IsTypeVisible is part of the eval.DatabaseCatalog interface.
func (so *importSequenceOperators) IsTypeVisible(
ctx context.Context, curDB string, searchPath sessiondata.SearchPath, typeID oid.Oid,
) (bool, bool, error) {
return false, false, errors.WithStack(errSequenceOperators)
}

// HasAnyPrivilege is part of the eval.DatabaseCatalog interface.
func (so *importSequenceOperators) HasAnyPrivilege(
ctx context.Context,
Expand Down
49 changes: 44 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/pg_builtins
Original file line number Diff line number Diff line change
Expand Up @@ -79,20 +79,26 @@ NULL
statement ok
CREATE TABLE is_visible(a int primary key);
CREATE TYPE visible_type AS ENUM('a');
CREATE FUNCTION visible_function() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$;
CREATE SCHEMA other;
CREATE TABLE other.not_visible(a int primary key);
CREATE TYPE other.not_visible_type AS ENUM('b');
CREATE FUNCTION other.function_not_visible() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$;
CREATE DATABASE db2;
SET DATABASE = db2;
CREATE TABLE table_in_db2(a int primary key);
CREATE TYPE type_in_db2 AS ENUM('c');
CREATE FUNCTION function_in_db2() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$;

let $table_in_db2_id
SELECT c.oid FROM pg_class c WHERE c.relname = 'table_in_db2';

let $type_in_db2_id
SELECT t.oid FROM pg_type t WHERE t.typname = 'type_in_db2';

let $function_in_db2_id
SELECT p.oid FROM pg_proc p WHERE p.proname = 'function_in_db2';

statement ok
SET DATABASE = test;

Expand Down Expand Up @@ -150,6 +156,39 @@ SELECT pg_type_is_visible(NULL)
----
NULL

query TB rowsort
SELECT p.proname, pg_function_is_visible(p.oid)
FROM pg_proc p
WHERE p.proname IN ('visible_function', 'function_not_visible')
----
function_not_visible false
visible_function true

# Looking up a function in a different database should return NULL.
query B
SELECT pg_function_is_visible($function_in_db2_id)
----
NULL

# Looking up a non-existent OID should return NULL.
query B
SELECT pg_function_is_visible(1010101010)
----
NULL

query B
SELECT pg_function_is_visible(NULL)
----
NULL

# Looking up builtins should work.
query BBB
SELECT
pg_function_is_visible('pg_function_is_visible'::regproc),
pg_function_is_visible('uuid_generate_v4'::regproc),
pg_function_is_visible('crdb_internal.force_panic'::regproc)
----
true true true

query TT
SELECT pg_get_partkeydef(1), pg_get_partkeydef(NULL)
Expand All @@ -174,11 +213,11 @@ WHERE c.relname IN ('is_updatable', 'is_updatable_view', 'pg_class')
ORDER BY c.oid, a.attnum
----
relname attname oid attnum pg_relation_is_updatable pg_column_is_updatable
is_updatable a 120 1 28 true
is_updatable b 120 2 28 true
is_updatable c 120 3 28 false
is_updatable_view a 121 1 0 false
is_updatable_view b 121 2 0 false
is_updatable a 123 1 28 true
is_updatable b 123 2 28 true
is_updatable c 123 3 28 false
is_updatable_view a 124 1 0 false
is_updatable_view b 124 2 0 false
pg_class oid 4294967123 1 0 false
pg_class relname 4294967123 2 0 false
pg_class relnamespace 4294967123 3 0 false
Expand Down
106 changes: 91 additions & 15 deletions pkg/sql/logictest/testdata/logic_test/tsvector
Original file line number Diff line number Diff line change
Expand Up @@ -3,27 +3,103 @@ SELECT 'foo:1,2 bar:3'::tsvector @@ 'foo <-> bar'::tsquery, 'foo <-> bar'::tsque
----
true true

statement error cannot be used for table columns
CREATE TABLE a (v tsvector)

statement error cannot be used for table columns
CREATE TABLE a (q tsquery)

# Uncomment once tsvector and tsquery are usable as table columns.
#
# statement ok
# INSERT INTO a VALUES('foo:1,2 bar:4B'::tsvector, 'foo <2> bar'::tsquery)
#
# query TT
# SELECT * FROM a
# ----
# 'bar':4B 'foo':1,2 'foo' <2> 'bar'
statement ok
CREATE TABLE a (v tsvector, q tsquery)

statement ok
INSERT INTO a VALUES('foo:1,2 bar:4B'::tsvector, 'foo <2> bar'::tsquery)

query TT
SELECT * FROM a
----
'bar':4B 'foo':1,2 'foo' <2> 'bar'

query BB
SELECT 'foo:1,2 bar:4B'::tsvector @@ 'foo <2> bar'::tsquery, 'foo:1,2 bar:4B' @@ 'foo <-> bar'::tsquery
----
true false

query BB
SELECT v @@ 'foo <2> bar'::tsquery, v @@ 'foo <-> bar'::tsquery FROM a
----
true false

query B
SELECT v @@ q FROM a
----
true

# Test column modifiers.

statement ok
CREATE TABLE b (a INT PRIMARY KEY DEFAULT 1, v tsvector DEFAULT 'foo:1' ON UPDATE 'bar:2', v2 tsvector AS (v) STORED, v3 tsvector AS (v) VIRTUAL)

statement ok
CREATE TABLE c (a INT PRIMARY KEY DEFAULT 1, q tsquery DEFAULT 'foo' ON UPDATE 'bar', q2 tsquery AS (q) STORED, q3 tsquery AS (q) VIRTUAL)

statement ok
INSERT INTO b DEFAULT VALUES

statement ok
INSERT INTO c DEFAULT VALUES

query ITTT
SELECT * FROM b
----
1 'foo':1 'foo':1 'foo':1

query ITTT
SELECT * FROM c
----
1 'foo' 'foo' 'foo'

statement ok
UPDATE b SET a = 2 WHERE a = 1

statement ok
UPDATE c SET a = 2 WHERE a = 1

query ITTT
SELECT * FROM b
----
2 'bar':2 'bar':2 'bar':2

query ITTT
SELECT * FROM c
----
2 'bar' 'bar' 'bar'

statement ok
INSERT INTO b VALUES (3, 'foo:1,5 zoop:3')

statement error can't order by column type TSVECTOR
SELECT * FROM b ORDER BY v

statement error can't order by column type TSQUERY
SELECT * FROM c ORDER BY q

statement error arrays of tsvector not allowed
CREATE TABLE tsarray(a tsvector[])

statement error arrays of tsquery not allowed
CREATE TABLE tsarray(a tsquery[])

statement error unsupported comparison operator
SELECT a, v FROM b WHERE v > 'bar:2'::tsvector

statement error unsupported comparison operator
SELECT a, q FROM c WHERE q > 'abc'::tsquery

query IT
SELECT a, v FROM b WHERE v = 'bar:2'::tsvector
----
2 'bar':2

query IT
SELECT a, q FROM c WHERE q = 'bar'::tsquery
----
2 'bar'

# Ensure truncation of long position lists.
query T
SELECT ('foo:' || string_agg(g::TEXT,','))::tsvector from generate_series(1,280) g(g);
Expand Down
Loading

0 comments on commit 16c786f

Please sign in to comment.