Skip to content

Commit

Permalink
Merge #52483
Browse files Browse the repository at this point in the history
52483: types: refactor user defined type API's in types package r=rohany a=rohany

Fixes #52451.

This commit removes the notion of type descriptor IDs from the `types`
and `tree` packages. It cleans up this API by using only type OIDs and
letting higher levels of the system translate these OIDs into descriptor
IDs.

Release note (backward-incompatible change): Clusters running alphas of
20.2 that use ENUM types will not be able to upgrade to betas or major
releases of 20.2 due to internal representation changes.

Co-authored-by: Rohan Yadav <rohany@alumni.cmu.edu>
  • Loading branch information
craig[bot] and rohany committed Aug 10, 2020
2 parents 7e4e0bf + e049fda commit c285413
Show file tree
Hide file tree
Showing 30 changed files with 285 additions and 282 deletions.
34 changes: 17 additions & 17 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

// DescRewriteMap maps old descriptor IDs to new descriptor and parent IDs.
Expand Down Expand Up @@ -99,10 +100,10 @@ func rewriteTypesInExpr(expr string, rewrites DescRewriteMap) (string, error) {
return "", err
}
ctx := tree.NewFmtCtx(tree.FmtSerializable)
ctx.SetIndexedTypeFormat(func(ctx *tree.FmtCtx, ref *tree.IDTypeReference) {
ctx.SetIndexedTypeFormat(func(ctx *tree.FmtCtx, ref *tree.OIDTypeReference) {
newRef := ref
if rw, ok := rewrites[descpb.ID(ref.ID)]; ok {
newRef = &tree.IDTypeReference{ID: uint32(rw.ID)}
if rw, ok := rewrites[sqlbase.UserDefinedTypeOIDToID(ref.OID)]; ok {
newRef = &tree.OIDTypeReference{OID: sqlbase.TypeIDToOID(rw.ID)}
}
ctx.WriteString(newRef.SQLString())
})
Expand Down Expand Up @@ -205,11 +206,11 @@ func allocateDescriptorRewrites(
// Ensure that all referenced types are present.
if col.Type.UserDefined() {
// TODO (rohany): This can be turned into an option later.
if _, ok := typesByID[descpb.ID(col.Type.StableTypeID())]; !ok {
if _, ok := typesByID[sqlbase.GetTypeDescID(col.Type)]; !ok {
return nil, errors.Errorf(
"cannot restore table %q without referenced type %d",
table.Name,
col.Type.StableTypeID(),
sqlbase.GetTypeDescID(col.Type),
)
}
}
Expand Down Expand Up @@ -633,21 +634,20 @@ func rewriteIDsInTypesT(typ *types.T, descriptorRewrites DescRewriteMap) {
if !typ.UserDefined() {
return
}
// TODO (rohany): Probably should expose some functions on the types.T to
// set this information, rather than reaching into the internal type.
if rw, ok := descriptorRewrites[descpb.ID(typ.StableTypeID())]; ok {
typ.InternalType.UDTMetadata.StableTypeID = uint32(rw.ID)
typ.InternalType.Oid = types.StableTypeIDToOID(uint32(rw.ID))
// Collect potential new OID values.
var newOID, newArrayOID oid.Oid
if rw, ok := descriptorRewrites[sqlbase.GetTypeDescID(typ)]; ok {
newOID = sqlbase.TypeIDToOID(rw.ID)
}

if typ.Family() != types.ArrayFamily {
if rw, ok := descriptorRewrites[sqlbase.GetArrayTypeDescID(typ)]; ok {
newArrayOID = sqlbase.TypeIDToOID(rw.ID)
}
}
types.RemapUserDefinedTypeOIDs(typ, newOID, newArrayOID)
// If the type is an array, then we need to rewrite the element type as well.
if typ.Family() == types.ArrayFamily {
rewriteIDsInTypesT(typ.ArrayContents(), descriptorRewrites)
} else {
// If the type is not an array, then we just need to updated the array
// type ID in the type metadata.
if rw, ok := descriptorRewrites[descpb.ID(typ.StableArrayTypeID())]; ok {
typ.InternalType.UDTMetadata.StableArrayTypeID = uint32(rw.ID)
}
}
}

Expand Down
15 changes: 8 additions & 7 deletions pkg/ccl/backupccl/targets_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,8 +53,9 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
mkTyp := func(desc typDesc) *sqlbase.ImmutableTypeDescriptor {
return sqlbase.NewImmutableTypeDescriptor(desc)
}
typeExpr := "'hello'::@15 = 'hello'::@15"
typeArrExpr := "'hello'::@16 = 'hello'::@16"
toOid := sqlbase.TypeIDToOID
typeExpr := "'hello'::@100015 = 'hello'::@100015"
typeArrExpr := "'hello'::@100016 = 'hello'::@100016"
descriptors = []sqlbase.Descriptor{
mkDB(0, "system"),
mkTable(tbDesc{ID: 1, Name: "foo", ParentID: 0}),
Expand All @@ -73,15 +74,15 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
// array of the user defined type, and that is tracked by the ArrayTypeID
// field on the type descriptor.
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 8, Name: "enum1", ArrayTypeID: 9, Kind: descpb.TypeDescriptor_ENUM}),
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 9, Name: "_enum1", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(8, 9)}),
mkTable(descpb.TableDescriptor{ParentID: 7, ID: 10, Name: "enum_tbl", Columns: []descpb.ColumnDescriptor{{ID: 0, Type: types.MakeEnum(8, 9)}}}),
mkTable(descpb.TableDescriptor{ParentID: 7, ID: 11, Name: "enum_arr_tbl", Columns: []descpb.ColumnDescriptor{{ID: 0, Type: types.MakeArray(types.MakeEnum(8, 9))}}}),
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 9, Name: "_enum1", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(toOid(8), toOid(9))}),
mkTable(descpb.TableDescriptor{ParentID: 7, ID: 10, Name: "enum_tbl", Columns: []descpb.ColumnDescriptor{{ID: 0, Type: types.MakeEnum(toOid(8), toOid(9))}}}),
mkTable(descpb.TableDescriptor{ParentID: 7, ID: 11, Name: "enum_arr_tbl", Columns: []descpb.ColumnDescriptor{{ID: 0, Type: types.MakeArray(types.MakeEnum(toOid(8), toOid(9)))}}}),
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 12, Name: "enum2", ArrayTypeID: 13, Kind: descpb.TypeDescriptor_ENUM}),
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 13, Name: "_enum2", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(12, 13)}),
mkTyp(descpb.TypeDescriptor{ParentID: 7, ID: 13, Name: "_enum2", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(toOid(12), toOid(13))}),
// Create some user defined types that are used in table expressions.
mkDB(14, "udts_expr"),
mkTyp(descpb.TypeDescriptor{ParentID: 14, ID: 15, Name: "enum1", ArrayTypeID: 16, Kind: descpb.TypeDescriptor_ENUM}),
mkTyp(descpb.TypeDescriptor{ParentID: 14, ID: 16, Name: "_enum1", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(15, 16)}),
mkTyp(descpb.TypeDescriptor{ParentID: 14, ID: 16, Name: "_enum1", Kind: descpb.TypeDescriptor_ALIAS, Alias: types.MakeEnum(toOid(15), toOid(16))}),
// Create a table with a default expression.
mkTable(tbDesc{
ID: 17,
Expand Down
11 changes: 7 additions & 4 deletions pkg/cli/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,18 +22,21 @@ import (

"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timetz"
"github.com/cockroachdb/cockroach/pkg/util/version"
"github.com/cockroachdb/errors"
"github.com/lib/pq"
"github.com/lib/pq/oid"
"github.com/spf13/cobra"
)

Expand Down Expand Up @@ -341,9 +344,9 @@ func (d *dumpTypeContext) ResolveType(
return typ, nil
}

// ResolveTypeByID implements the tree.TypeReferenceResolver interface.
func (d *dumpTypeContext) ResolveTypeByID(context.Context, uint32) (*types.T, error) {
return nil, errors.AssertionFailedf("cannot resolve types in dump by ID")
// ResolveTypeByOID implements the tree.TypeReferenceResolver interface.
func (d *dumpTypeContext) ResolveTypeByOID(context.Context, oid.Oid) (*types.T, error) {
return nil, errors.AssertionFailedf("cannot resolve types in dump by OID")
}

func collectUserDefinedSchemas(conn *sqlConn, dbName string, ts string) ([]string, error) {
Expand Down Expand Up @@ -470,7 +473,7 @@ WHERE
// type kind in the typing context.
switch {
case len(enumMembers) != 0:
typ := types.MakeEnum(uint32(id), 0 /* arrayTypeID */)
typ := types.MakeEnum(sqlbase.TypeIDToOID(descpb.ID(id)), 0 /* arrayTypeOID */)
typ.TypeMeta = types.UserDefinedTypeMetadata{
Name: &types.UserDefinedTypeName{
Name: name,
Expand Down
3 changes: 2 additions & 1 deletion pkg/internal/sqlsmith/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
// Import builtins so they are reflected in tree.FunDefs.
_ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -171,7 +172,7 @@ FROM
// Try to construct type information from the resulting row.
switch {
case len(members) > 0:
typ := types.MakeEnum(uint32(id), 0 /* arrayTypeID */)
typ := types.MakeEnum(sqlbase.TypeIDToOID(descpb.ID(id)), 0 /* arrayTypeID */)
typ.TypeMeta = types.UserDefinedTypeMetadata{
Name: &types.UserDefinedTypeName{
Schema: scName,
Expand Down
7 changes: 4 additions & 3 deletions pkg/internal/sqlsmith/type.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

func (s *Smither) typeFromName(name string) (*types.T, error) {
Expand Down Expand Up @@ -98,7 +99,7 @@ func (s *Smither) ResolveType(
return res, nil
}

// ResolveTypeByID implements the tree.TypeReferenceResolver interface.
func (s *Smither) ResolveTypeByID(context.Context, uint32) (*types.T, error) {
return nil, errors.AssertionFailedf("smither cannot resolve types by ID")
// ResolveTypeByOID implements the tree.TypeReferenceResolver interface.
func (s *Smither) ResolveTypeByOID(context.Context, oid.Oid) (*types.T, error) {
return nil, errors.AssertionFailedf("smither cannot resolve types by OID")
}
11 changes: 7 additions & 4 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

// UncommittedDatabase is a database that has been created/dropped
Expand Down Expand Up @@ -1235,9 +1236,11 @@ func (dt *DistSQLTypeResolver) ResolveType(
return nil, errors.AssertionFailedf("cannot resolve types in DistSQL by name")
}

// ResolveTypeByID implements the tree.TypeReferenceResolver interface.
func (dt *DistSQLTypeResolver) ResolveTypeByID(ctx context.Context, id uint32) (*types.T, error) {
name, desc, err := dt.GetTypeDescriptor(ctx, descpb.ID(id))
// ResolveTypeByOID implements the tree.TypeReferenceResolver interface.
func (dt *DistSQLTypeResolver) ResolveTypeByOID(
ctx context.Context, oid oid.Oid,
) (*types.T, error) {
name, desc, err := dt.GetTypeDescriptor(ctx, sqlbase.UserDefinedTypeOIDToID(oid))
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -1266,7 +1269,7 @@ func (dt *DistSQLTypeResolver) GetTypeDescriptor(
func (dt *DistSQLTypeResolver) HydrateTypeSlice(ctx context.Context, typs []*types.T) error {
for _, t := range typs {
if t.UserDefined() {
name, desc, err := dt.GetTypeDescriptor(ctx, descpb.ID(t.StableTypeID()))
name, desc, err := dt.GetTypeDescriptor(ctx, sqlbase.GetTypeDescID(t))
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/create_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func (p *planner) createArrayType(
var elemTyp *types.T
switch t := typDesc.Kind; t {
case descpb.TypeDescriptor_ENUM:
elemTyp = types.MakeEnum(uint32(typDesc.GetID()), uint32(id))
elemTyp = types.MakeEnum(sqlbase.TypeIDToOID(typDesc.GetID()), sqlbase.TypeIDToOID(id))
default:
return 0, errors.AssertionFailedf("cannot make array type for kind %s", t.String())
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -450,7 +450,7 @@ FROM
WHERE
descriptor_name = 'enum_array' AND column_name = 'x'
----
x family:ArrayFamily width:0 precision:0 locale:"" visible_type:0 oid:100064 array_contents:<InternalType:<family:EnumFamily width:0 precision:0 locale:"" visible_type:0 oid:100063 time_precision_is_set:false udt_metadata:<stable_type_id:63 stable_array_type_id:64 > > TypeMeta:<Version:0 > > time_precision_is_set:false udt_metadata:<stable_type_id:64 stable_array_type_id:0 >
x family:ArrayFamily width:0 precision:0 locale:"" visible_type:0 oid:100064 array_contents:<InternalType:<family:EnumFamily width:0 precision:0 locale:"" visible_type:0 oid:100063 time_precision_is_set:false udt_metadata:<array_type_oid:100064 > > TypeMeta:<Version:0 > > time_precision_is_set:false

# Test tables using enums in DEFAULT expressions.
statement ok
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/numeric_references
Original file line number Diff line number Diff line change
Expand Up @@ -216,5 +216,8 @@ DELETE FROM [$num_ref_id AS t]
statement error pq: user testuser does not have UPDATE privilege on relation num_ref
UPDATE [$num_ref_id AS t] SET d=1

# The @<num> reference is by type OID, which is
# type desc ID + oid.CockroachPredefinedOIDMax. This error reports that the
# type desc with ID 15210 is not found, hence the slightly different error message.
statement error pq: type with ID 15210 does not exist
SELECT 1::@15210
SELECT 1::@115210
5 changes: 3 additions & 2 deletions pkg/sql/opt/cat/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/lib/pq/oid"
)

// StableID permanently and uniquely identifies a catalog object (table, view,
Expand Down Expand Up @@ -115,8 +116,8 @@ type Catalog interface {
ctx context.Context, flags Flags, id StableID,
) (_ DataSource, isAdding bool, _ error)

// ResolveTypeByID is used to look up a user defined type by ID.
ResolveTypeByID(ctx context.Context, id uint32) (*types.T, error)
// ResolveTypeByOID is used to look up a user defined type by ID.
ResolveTypeByOID(ctx context.Context, oid oid.Oid) (*types.T, error)

// CheckPrivilege verifies that the current user has the given privilege on
// the given catalog object. If not, then CheckPrivilege returns an error.
Expand Down
15 changes: 8 additions & 7 deletions pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

// SchemaID uniquely identifies the usage of a schema within the scope of a
Expand Down Expand Up @@ -94,7 +95,7 @@ type Metadata struct {
// because the installation of type metadata in tables doesn't go through
// the type resolver that the optimizer hijacks. However, we could update
// this map when adding a table via metadata.AddTable.
userDefinedTypes map[uint32]struct{}
userDefinedTypes map[oid.Oid]struct{}
userDefinedTypesSlice []*types.T

// deps stores information about all data source objects depended on by the
Expand Down Expand Up @@ -202,11 +203,11 @@ func (md *Metadata) CopyFrom(from *Metadata) {
md.tables = append(md.tables, from.tables...)

if (md.userDefinedTypes) == nil {
md.userDefinedTypes = make(map[uint32]struct{})
md.userDefinedTypes = make(map[oid.Oid]struct{})
}
for i := range from.userDefinedTypesSlice {
typ := from.userDefinedTypesSlice[i]
md.userDefinedTypes[typ.StableTypeID()] = struct{}{}
md.userDefinedTypes[typ.Oid()] = struct{}{}
md.userDefinedTypesSlice = append(md.userDefinedTypesSlice, typ)
}

Expand Down Expand Up @@ -308,7 +309,7 @@ func (md *Metadata) CheckDependencies(
}
// Check that all of the user defined types present have not changed.
for _, typ := range md.AllUserDefinedTypes() {
toCheck, err := catalog.ResolveTypeByID(ctx, typ.StableTypeID())
toCheck, err := catalog.ResolveTypeByOID(ctx, typ.Oid())
if err != nil {
// Handle when the type no longer exists.
if pgerror.GetPGCode(err) == pgcode.UndefinedObject {
Expand Down Expand Up @@ -341,10 +342,10 @@ func (md *Metadata) AddUserDefinedType(typ *types.T) {
return
}
if md.userDefinedTypes == nil {
md.userDefinedTypes = make(map[uint32]struct{})
md.userDefinedTypes = make(map[oid.Oid]struct{})
}
if _, ok := md.userDefinedTypes[typ.StableTypeID()]; !ok {
md.userDefinedTypes[typ.StableTypeID()] = struct{}{}
if _, ok := md.userDefinedTypes[typ.Oid()]; !ok {
md.userDefinedTypes[typ.Oid()] = struct{}{}
md.userDefinedTypesSlice = append(md.userDefinedTypesSlice, typ)
}
}
Expand Down
8 changes: 5 additions & 3 deletions pkg/sql/opt/metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package opt_test

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/opt"
Expand All @@ -30,7 +31,7 @@ func TestMetadata(t *testing.T) {
tabID := md.AddTable(&testcat.Table{}, &tree.TableName{})
seqID := md.AddSequence(&testcat.Sequence{})
md.AddView(&testcat.View{})
md.AddUserDefinedType(types.MakeEnum(15210, 15418))
md.AddUserDefinedType(types.MakeEnum(152100, 154180))

// Call Init and add objects from catalog, verifying that IDs have been reset.
testCat := testcat.New()
Expand All @@ -56,8 +57,9 @@ func TestMetadata(t *testing.T) {
t.Fatalf("unexpected views")
}

md.AddUserDefinedType(types.MakeEnum(15150, 15251))
md.AddUserDefinedType(types.MakeEnum(151500, 152510))
if len(md.AllUserDefinedTypes()) != 1 {
fmt.Println(md)
t.Fatalf("unexpected types")
}

Expand Down Expand Up @@ -91,7 +93,7 @@ func TestMetadata(t *testing.T) {
t.Fatalf("unexpected view")
}

if ts := mdNew.AllUserDefinedTypes(); len(ts) != 1 && ts[15150].Equal(types.MakeEnum(15150, 15251)) {
if ts := mdNew.AllUserDefinedTypes(); len(ts) != 1 && ts[151500].Equal(types.MakeEnum(151500, 152510)) {
t.Fatalf("unexpected type")
}

Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/errorutil"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

// Builder holds the context needed for building a memo structure from a SQL
Expand Down Expand Up @@ -415,11 +416,11 @@ func (o *optTrackingTypeResolver) ResolveType(
return typ, nil
}

// ResolveTypeByID implements the tree.TypeResolver interface.
func (o *optTrackingTypeResolver) ResolveTypeByID(
ctx context.Context, id uint32,
// ResolveTypeByOID implements the tree.TypeResolver interface.
func (o *optTrackingTypeResolver) ResolveTypeByOID(
ctx context.Context, oid oid.Oid,
) (*types.T, error) {
typ, err := o.res.ResolveTypeByID(ctx, id)
typ, err := o.res.ResolveTypeByOID(ctx, oid)
if err != nil {
return nil, err
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/opt/testutils/testcat/test_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/treeprinter"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)

const (
Expand Down Expand Up @@ -170,8 +171,8 @@ func (tc *Catalog) ResolveDataSourceByID(
"relation [%d] does not exist", id)
}

// ResolveTypeByID is part of the cat.Catalog interface.
func (tc *Catalog) ResolveTypeByID(context.Context, uint32) (*types.T, error) {
// ResolveTypeByOID is part of the cat.Catalog interface.
func (tc *Catalog) ResolveTypeByOID(context.Context, oid.Oid) (*types.T, error) {
return nil, errors.Newf("test catalog cannot handle user defined types")
}

Expand Down
Loading

0 comments on commit c285413

Please sign in to comment.