Skip to content

Commit

Permalink
Merge #49678
Browse files Browse the repository at this point in the history
49678: sql: create implicit array types for UDTs r=otan,jordanlewis a=rohany

Fixes #49197.

This PR adds the Postgres behavior of creating an implicit array type
for a user defined type when it is created. The implicit array type will
track the user defined type.

Release note: None

Co-authored-by: Rohan Yadav <rohany@alumni.cmu.edu>
  • Loading branch information
craig[bot] and rohany committed May 30, 2020
2 parents 094f563 + 8ee992b commit 021d876
Show file tree
Hide file tree
Showing 17 changed files with 1,378 additions and 903 deletions.
106 changes: 99 additions & 7 deletions pkg/sql/create_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package sql

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -22,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -99,6 +101,89 @@ func getCreateTypeParams(
return typeKey, id, nil
}

// createArrayType performs the implicit array type creation logic of Postgres.
// When a type is created in Postgres, Postgres will implicitly create an array
// type of that user defined type. This array type tracks changes to the
// original type, and is dropped when the original type is dropped.
// createArrayType creates the implicit array type for the input TypeDescriptor
// and returns the ID of the created type.
func (p *planner) createArrayType(
params runParams,
n *tree.CreateType,
typ *tree.TypeName,
typDesc *sqlbase.MutableTypeDescriptor,
db *DatabaseDescriptor,
) (sqlbase.ID, error) {
// Postgres starts off trying to create the type as _<typename>. It then
// continues adding "_" to the front of the name until it doesn't find
// a collision.
schemaID := sqlbase.ID(keys.PublicSchemaID)
arrayTypeName := "_" + typ.Type()
var arrayTypeKey sqlbase.DescriptorKey
for {
// See if there is a collision with the current name.
exists, _, err := sqlbase.LookupObjectID(
params.ctx,
params.p.txn,
params.ExecCfg().Codec,
db.ID,
schemaID,
arrayTypeName,
)
if err != nil {
return 0, err
}
// If we found an empty spot, then create the namespace key for this entry.
if !exists {
arrayTypeKey = sqlbase.MakePublicTableNameKey(
params.ctx,
params.ExecCfg().Settings,
db.ID,
arrayTypeName,
)
break
}
// Otherwise, append another "_" to the front of the name.
arrayTypeName = "_" + arrayTypeName
}

// Generate the stable ID for the array type.
id, err := catalogkv.GenerateUniqueDescID(params.ctx, params.ExecCfg().DB, params.ExecCfg().Codec)
if err != nil {
return 0, err
}

// Create the element type for the array. Note that it must know about the
// ID of the array type in order for the array type to correctly created.
var elemTyp *types.T
switch t := typDesc.Kind; t {
case sqlbase.TypeDescriptor_ENUM:
elemTyp = types.MakeEnum(uint32(typDesc.ID), uint32(id))
default:
return 0, errors.AssertionFailedf("cannot make array type for kind %s", t.String())
}

// Construct the descriptor for the array type.
arrayTypDesc := sqlbase.NewMutableCreatedTypeDescriptor(sqlbase.MakeTypeDescriptor(
db.ID, keys.PublicSchemaID, id, arrayTypeName,
))
arrayTypDesc.Kind = sqlbase.TypeDescriptor_ALIAS
arrayTypDesc.Alias = types.MakeArray(elemTyp)

jobStr := fmt.Sprintf("implicit array type creation for %s", tree.AsStringWithFQNames(n, params.Ann()))
if err := p.createDescriptorWithID(
params.ctx,
arrayTypeKey.Key(params.ExecCfg().Codec),
id,
arrayTypDesc,
params.EvalContext().Settings,
jobStr,
); err != nil {
return 0, err
}
return id, nil
}

func (p *planner) createEnum(params runParams, n *tree.CreateType) error {
// Make sure that all nodes in the cluster are able to recognize ENUM types.
if !p.ExecCfg().Settings.Version.IsActive(params.ctx, clusterversion.VersionEnums) {
Expand Down Expand Up @@ -150,15 +235,22 @@ func (p *planner) createEnum(params runParams, n *tree.CreateType) error {
// a free list of descriptor ID's (#48438), we should allocate an ID from
// there if id + oidext.CockroachPredefinedOIDMax overflows past the
// maximum uint32 value.
typeDesc := &sqlbase.TypeDescriptor{
ParentID: db.ID,
ParentSchemaID: keys.PublicSchemaID,
Name: typeName.Type(),
ID: id,
Kind: sqlbase.TypeDescriptor_ENUM,
EnumMembers: members,
typeDesc := sqlbase.NewMutableCreatedTypeDescriptor(sqlbase.MakeTypeDescriptor(
db.ID, keys.PublicSchemaID, id, typeName.Type(),
))
typeDesc.Kind = sqlbase.TypeDescriptor_ENUM
typeDesc.EnumMembers = members

// Create the implicit array type for this type before finishing the type.
arrayTypeID, err := p.createArrayType(params, n, typeName, typeDesc, db)
if err != nil {
return err
}

// Update the typeDesc with the created array type ID.
typeDesc.ArrayTypeID = arrayTypeID

// Now create the type after the implicit array type as been created.
return p.createDescriptorWithID(
params.ctx,
typeKey.Key(params.ExecCfg().Codec),
Expand Down
47 changes: 19 additions & 28 deletions pkg/sql/execinfrapb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,10 @@ import (
"fmt"
"sync"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -87,50 +90,38 @@ func (tr *DistSQLTypeResolver) ResolveType(
return nil, errors.AssertionFailedf("cannot resolve types in DistSQL by name")
}

func makeTypeLookupFunc(
ctx context.Context, txn *kv.Txn, codec keys.SQLCodec,
) sqlbase.TypeLookupFunc {
return func(id sqlbase.ID) (*tree.TypeName, *sqlbase.TypeDescriptor, error) {
return resolver.ResolveTypeDescByID(ctx, txn, codec, id)
}
}

// ResolveTypeByID implements tree.ResolvableTypeReference.
func (tr *DistSQLTypeResolver) ResolveTypeByID(ctx context.Context, id uint32) (*types.T, error) {
// TODO (rohany): This should eventually look into the set of cached type
// descriptors before attempting to access it here.
typDesc, err := sqlbase.GetTypeDescFromID(
ctx,
tr.EvalContext.Txn,
tr.EvalContext.Codec,
sqlbase.ID(id),
)
lookup := makeTypeLookupFunc(ctx, tr.EvalContext.Txn, tr.EvalContext.Codec)
name, typDesc, err := lookup(sqlbase.ID(id))
if err != nil {
return nil, err
}
var typ *types.T
switch t := typDesc.Kind; t {
case sqlbase.TypeDescriptor_ENUM:
typ = types.MakeEnum(id)
case sqlbase.TypeDescriptor_ALIAS:
return typDesc.Alias, nil
default:
return nil, errors.AssertionFailedf("unknown type kind %s", t)
}
if err := typDesc.HydrateTypeInfo(typ); err != nil {
return nil, err
}
return typ, nil
return typDesc.MakeTypesT(name, lookup)
}

// HydrateTypeSlice hydrates all user defined types in an input slice of types.
func HydrateTypeSlice(evalCtx *tree.EvalContext, typs []*types.T) error {
// TODO (rohany): This should eventually look into the set of cached type
// descriptors before attempting to access it here.
lookup := makeTypeLookupFunc(evalCtx.Context, evalCtx.Txn, evalCtx.Codec)
for _, t := range typs {
if t.UserDefined() {
// TODO (rohany): This should eventually look into the set of cached type
// descriptors before attempting to access it here.
typDesc, err := sqlbase.GetTypeDescFromID(
evalCtx.Context,
evalCtx.Txn,
evalCtx.Codec,
sqlbase.ID(t.StableTypeID()),
)
name, typDesc, err := lookup(sqlbase.ID(t.StableTypeID()))
if err != nil {
return err
}
if err := typDesc.HydrateTypeInfo(t); err != nil {
if err := typDesc.HydrateTypeInfoWithName(t, name, lookup); err != nil {
return err
}
}
Expand Down
32 changes: 16 additions & 16 deletions pkg/sql/logictest/testdata/logic_test/dependencies
Original file line number Diff line number Diff line change
Expand Up @@ -15,22 +15,22 @@ CREATE TABLE test_kv(k INT PRIMARY KEY, v INT, w DECIMAL);
query ITITTBTB colnames
SELECT * FROM crdb_internal.table_columns WHERE descriptor_name LIKE 'test_%' ORDER BY descriptor_id, column_id
----
descriptor_id descriptor_name column_id column_name column_type nullable default_expr hidden
53 test_kv 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
53 test_kv 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
53 test_kv 3 w family:DecimalFamily width:0 precision:0 locale:"" visible_type:0 oid:1700 time_precision_is_set:false stable_type_id:0 true NULL false
54 test_kvr1 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
55 test_kvr2 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
55 test_kvr2 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
55 test_kvr2 3 rowid family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false unique_rowid() true
56 test_kvr3 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
56 test_kvr3 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
56 test_kvr3 3 rowid family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false unique_rowid() true
57 test_kvi1 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
58 test_kvi2 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
58 test_kvi2 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 true NULL false
59 test_v1 1 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
60 test_v2 1 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false stable_type_id:0 false NULL false
descriptor_id descriptor_name column_id column_name column_type nullable default_expr hidden
53 test_kv 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false
53 test_kv 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
53 test_kv 3 w family:DecimalFamily width:0 precision:0 locale:"" visible_type:0 oid:1700 time_precision_is_set:false true NULL false
54 test_kvr1 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false
55 test_kvr2 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
55 test_kvr2 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
55 test_kvr2 3 rowid family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false unique_rowid() true
56 test_kvr3 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
56 test_kvr3 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
56 test_kvr3 3 rowid family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false unique_rowid() true
57 test_kvi1 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false
58 test_kvi2 1 k family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false
58 test_kvi2 2 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false true NULL false
59 test_v1 1 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false
60 test_v2 1 v family:IntFamily width:64 precision:0 locale:"" visible_type:0 oid:20 time_precision_is_set:false false NULL false

query ITITTBB colnames
SELECT * FROM crdb_internal.table_indexes WHERE descriptor_name LIKE 'test_%' ORDER BY descriptor_id, index_id
Expand Down
48 changes: 48 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -410,3 +410,51 @@ CREATE TABLE t1 (
INDEX i (x ASC),
FAMILY "primary" (x, rowid)
)

# Test that the implicit array type has been created, and that we can use it.
query TT
SELECT ARRAY['hello']::_greeting, ARRAY['hello'::greeting]
----
{hello} {hello}

# Test that we can't mix enums in an array.
query error pq: expected 'cockroach'::dbs to be of type greeting, found type dbs
SELECT ARRAY['hello'::greeting, 'cockroach'::dbs]

statement ok
CREATE TABLE enum_array (x _greeting, y greeting[]);
INSERT INTO enum_array VALUES (ARRAY['hello'], ARRAY['hello']), (ARRAY['howdy'], ARRAY['howdy'])

query TT rowsort
SELECT * FROM enum_array
----
{hello} {hello}
{howdy} {howdy}

query TTT
SELECT pg_typeof(x), pg_typeof(x[1]), pg_typeof(ARRAY['hello']::_greeting) FROM enum_array LIMIT 1
----
test.public.greeting[] test.public.greeting test.public.greeting[]

# Ensure that the implicitly created array type will tolerate collisions.
# _collision will create __collision as its implicit array type, so the
# creation of collision will have to retry twice before it finds the open
# spot of ___collision for its implicit array type.
statement ok
CREATE TYPE _collision AS ENUM ();
CREATE TYPE collision AS ENUM ();

# _collision and __collision typelem and typarray should point back at each
# other, and vice versa for collision and ___collision.
query TOOO rowsort
SELECT
typname, oid, typelem, typarray
FROM
pg_type
WHERE
typname IN ('collision', '_collision', '__collision', '___collision')
----
_collision 100082 0 100083
__collision 100083 100082 0
collision 100084 0 100085
___collision 100085 100084 0
Loading

0 comments on commit 021d876

Please sign in to comment.