Skip to content

Commit

Permalink
sql: add support for virtual schema qualified types
Browse files Browse the repository at this point in the history
Fixes #16395.

Release note (sql change): Allows for referencing static data types
under the `pg_catalog` qualification like `pg_catalog.int`.
  • Loading branch information
rohany committed May 26, 2020
1 parent 6f92e58 commit fbe54be
Show file tree
Hide file tree
Showing 13 changed files with 517 additions and 358 deletions.
52 changes: 13 additions & 39 deletions pkg/sql/catalog/accessors/logical_schema_accessors.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/errors"
)

// This file provides reference implementations of the schema accessor
Expand All @@ -42,8 +41,6 @@ func NewLogicalAccessor(
type LogicalSchemaAccessor struct {
catalog.Accessor
vs catalog.VirtualSchemas
// Used to avoid allocations.
tn tree.TableName
}

var _ catalog.Accessor = &LogicalSchemaAccessor{}
Expand Down Expand Up @@ -96,43 +93,20 @@ func (l *LogicalSchemaAccessor) GetObjectDesc(
db, schema, object string,
flags tree.ObjectLookupFlags,
) (catalog.Descriptor, error) {
switch flags.DesiredObjectKind {
case tree.TypeObject:
// TODO(ajwerner): Change this function if we ever expose non-table objects
// underneath virtual schemas. For now we assume that the only objects
// ever handed back from GetObjectByName are tables. Instead we fallthrough
// to the underlying physical accessor.
return l.Accessor.GetObjectDesc(ctx, txn, settings, codec, db, schema, object, flags)
case tree.TableObject:
l.tn = tree.MakeTableNameWithSchema(tree.Name(db), tree.Name(schema), tree.Name(object))
if scEntry, ok := l.vs.GetVirtualSchema(schema); ok {
table, err := scEntry.GetObjectByName(object)
if err != nil {
return nil, err
}
if table == nil {
if flags.Required {
return nil, sqlbase.NewUndefinedRelationError(&l.tn)
}
return nil, nil
}
desc := table.Desc().TableDesc()
if desc == nil {
// This can only happen if we have a non-table object stored on a
// virtual schema. For now we'll return an assertion error.
return nil, errors.AssertionFailedf(
"non-table object of type %T returned from virtual schema for %v",
table.Desc(), l.tn)
}
if flags.RequireMutable {
return sqlbase.NewMutableExistingTableDescriptor(*desc), nil
if scEntry, ok := l.vs.GetVirtualSchema(schema); ok {
desc, err := scEntry.GetObjectByName(object, flags)
if err != nil {
return nil, err
}
if desc == nil {
if flags.Required {
obj := tree.NewQualifiedObjectName(db, schema, object, flags.DesiredObjectKind)
return nil, sqlbase.NewUndefinedObjectError(obj, flags.DesiredObjectKind)
}
return sqlbase.NewImmutableTableDescriptor(*desc), nil
return nil, nil
}

// Fallthrough.
return l.Accessor.GetObjectDesc(ctx, txn, settings, codec, db, schema, object, flags)
default:
return nil, errors.AssertionFailedf("unknown desired object kind %d", flags.DesiredObjectKind)
return desc.Desc(), nil
}
// Fallthrough.
return l.Accessor.GetObjectDesc(ctx, txn, settings, codec, db, schema, object, flags)
}
2 changes: 1 addition & 1 deletion pkg/sql/catalog/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ type VirtualSchema interface {
Desc() Descriptor
NumTables() int
VisitTables(func(object VirtualObject))
GetObjectByName(name string) (VirtualObject, error)
GetObjectByName(name string, flags tree.ObjectLookupFlags) (VirtualObject, error)
}

// VirtualObject is a virtual schema object.
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/execinfrapb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,8 @@ func (tr *DistSQLTypeResolver) ResolveTypeByID(id uint32) (*types.T, error) {
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)
}
Expand Down
5 changes: 0 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -56,11 +56,6 @@ SELECT 'Z'::public.int
----
Z

query T
SELECT 'Z'::"int"
----
Z

statement ok
CREATE TYPE greeting AS ENUM ('hello', 'howdy', 'hi')

Expand Down
15 changes: 15 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/typing
Original file line number Diff line number Diff line change
Expand Up @@ -232,3 +232,18 @@ query T
SELECT max(NULL) FROM (VALUES (NULL), (NULL)) t0(c0)
----
NULL

# Test qualified type references.
query IITR
SELECT 1::pg_catalog.int4, 1::pg_catalog.int8, 'aa'::pg_catalog.text, 4.2::pg_catalog.float4
----
1 1 aa 4.2

# Test that we error out referencing unknown types in pg_catalog.
query error pq: type "pg_catalog.special_int" does not exist
SELECT 1::pg_catalog.special_int

# Test that we error out trying to reference types in schemas that
# don't have types.
query error pq: type "crdb_internal.mytype" does not exist
SELECT 1::crdb_internal.mytype
1 change: 1 addition & 0 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,7 @@ var pgCatalog = virtualSchema{
// database set. Simply reject any attempts to use them in that
// case.
validWithNoDatabaseContext: false,
containsTypes: true,
}

// The catalog pg_am stores information about relation access methods.
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,8 @@ func (p *planner) ResolveType(name *tree.UnresolvedObjectName) (*types.T, error)
// Override the hydrated name with the fully resolved type name.
typ.TypeMeta.Name = &tn
return typ, nil
case sqlbase.TypeDescriptor_ALIAS:
return tdesc.Alias, nil
default:
return nil, errors.AssertionFailedf("unknown type kind %s", t.String())
}
Expand Down
15 changes: 15 additions & 0 deletions pkg/sql/sem/tree/name_resolution.go
Original file line number Diff line number Diff line change
Expand Up @@ -581,6 +581,21 @@ const (
TypeObject
)

// NewQualifiedObjectName returns an ObjectName of the corresponding kind.
// It is used mainly for constructing appropriate error messages depending
// on what kind of object was requested.
func NewQualifiedObjectName(catalog, schema, object string, kind DesiredObjectKind) ObjectName {
switch kind {
case TableObject:
name := MakeTableNameWithSchema(Name(catalog), Name(schema), Name(object))
return &name
case TypeObject:
name := MakeNewQualifiedTypeName(catalog, schema, object)
return &name
}
return nil
}

// ObjectLookupFlags is the flag struct suitable for GetObjectDesc().
type ObjectLookupFlags struct {
CommonLookupFlags
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/sem/tree/type_name.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,17 @@ func MakeTypeNameFromPrefix(prefix ObjectNamePrefix, object Name) TypeName {
}}
}

// MakeNewQualifiedTypeName creates a fully qualified type name.
func MakeNewQualifiedTypeName(db, schema, typ string) TypeName {
return TypeName{objName{
ObjectNamePrefix: ObjectNamePrefix{
CatalogName: Name(db),
SchemaName: Name(schema),
},
ObjectName: Name(typ),
}}
}

// TypeReferenceResolver is the interface that will provide the ability
// to actually look up type metadata and transform references into
// *types.T's.
Expand Down
17 changes: 17 additions & 0 deletions pkg/sql/sqlbase/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -4222,11 +4222,28 @@ func (desc *TypeDescriptor) HydrateTypeInfo(typ *types.T) error {
PhysicalRepresentations: physical,
}
return nil
case TypeDescriptor_ALIAS:
// This is a noop until we possibly allow aliases to user defined types.
return nil
default:
return errors.AssertionFailedf("unknown type descriptor kind %s", desc.Kind)
}
}

// MakeSimpleAliasTypeDescriptor creates a type descriptor that is an alias
// for the input type. It is intended to be used as an intermediate for name
// resolution, and should not be serialized and stored on disk.
func MakeSimpleAliasTypeDescriptor(typ *types.T) *TypeDescriptor {
return &TypeDescriptor{
ParentID: InvalidID,
ParentSchemaID: InvalidID,
Name: typ.Name(),
ID: InvalidID,
Kind: TypeDescriptor_ALIAS,
Alias: typ,
}
}

// NameResolutionResult implements the NameResolutionResult interface.
func (desc *TypeDescriptor) NameResolutionResult() {}

Expand Down
Loading

0 comments on commit fbe54be

Please sign in to comment.