Skip to content

Commit

Permalink
descs: push descriptor type hydration into the desc.Collection
Browse files Browse the repository at this point in the history
Fixes cockroachdb#49484.
Preparation for cockroachdb#51385.

Up until now, the `planner` was responsible for installing user defined
type metadata in tables that contained user defined types. This was
slightly messy and caused leakage of responsibility regarding when
descriptors had user defined types vs when they didn't. This commit
pushes that responsibility into the `descs.Collection`. It also paves
the way for work to avoid copying `ImmutableTableDescriptor`s that
contain user defined types every time that they need hydration.

Release note: None
  • Loading branch information
rohany committed Jul 20, 2020
1 parent 5c3b601 commit 91a0c1b
Show file tree
Hide file tree
Showing 4 changed files with 133 additions and 86 deletions.
15 changes: 0 additions & 15 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -656,20 +655,6 @@ func (sc *SchemaChanger) truncateIndexes(
return err
}

// Hydrate types used in the retrieved table.
// TODO (rohany): This can be removed once table access from the
// desc.Collection returns tables with hydrated types.
typLookup := func(ctx context.Context, id sqlbase.ID) (*tree.TypeName, sqlbase.TypeDescriptorInterface, error) {
return resolver.ResolveTypeDescByID(ctx, txn, sc.execCfg.Codec, id, tree.ObjectLookupFlags{})
}
if err := sqlbase.HydrateTypesInTableDescriptor(
ctx,
tableDesc.TableDesc(),
sqlbase.TypeLookupFunc(typLookup),
); err != nil {
return err
}

rd, err := row.MakeDeleter(
ctx,
txn,
Expand Down
153 changes: 132 additions & 21 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -74,6 +75,8 @@ func NewCollection(leaseMgr *lease.Manager, settings *cluster.Settings) *Collect
// collection is cleared using ReleaseAll() which is called at the
// end of each transaction on the session, or on hitting conditions such
// as errors, or retries that result in transaction timestamp changes.
// The descriptor collection always returns TableDescriptors that have user
// defined type metadata installed already.
type Collection struct {
// leaseMgr manages acquiring and releasing per-table leases.
leaseMgr *lease.Manager
Expand Down Expand Up @@ -148,12 +151,9 @@ func isSupportedSchemaName(n tree.Name) bool {
return n == tree.PublicSchemaName || strings.HasPrefix(string(n), "pg_temp")
}

// GetMutableTableDescriptor returns a mutable table descriptor.
//
// If flags.required is false, GetMutableTableDescriptor() will gracefully
// return a nil descriptor and no error if the table does not exist.
//
func (tc *Collection) GetMutableTableDescriptor(
// getMutableTableDescriptorImpl performs the logic of acquiring a
// MutableTableDescriptor by name.
func (tc *Collection) getMutableTableDescriptorImpl(
ctx context.Context, txn *kv.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
) (*sqlbase.MutableTableDescriptor, error) {
if log.V(2) {
Expand Down Expand Up @@ -221,6 +221,69 @@ func (tc *Collection) GetMutableTableDescriptor(
return mutDesc, nil
}

// GetMutableTableDescriptor returns a mutable table descriptor.
//
// If flags.required is false, GetMutableTableDescriptor() will gracefully
// return a nil descriptor and no error if the table does not exist.
//
func (tc *Collection) GetMutableTableDescriptor(
ctx context.Context, txn *kv.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
) (*sqlbase.MutableTableDescriptor, error) {
desc, err := tc.getMutableTableDescriptorImpl(ctx, txn, tn, flags)
if err != nil {
return nil, err
}
// If we found a descriptor, then attempt to hydrate the types in it.
if desc != nil {
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, desc)
if err != nil {
return nil, err
}
desc = hydrated.(*sqlbase.MutableTableDescriptor)
}
return desc, nil
}

// hydrateTypesInTableDesc installs user defined type metadata in all types.T
// present in the input TableDescriptor. It always returns the same type of
// TableDescriptor that was passed in. It ensures that ImmutableTableDescriptors
// are not modified during the process of metadata installation.
func (tc *Collection) hydrateTypesInTableDesc(
ctx context.Context, txn *kv.Txn, desc sqlbase.TableDescriptorInterface,
) (sqlbase.TableDescriptorInterface, error) {
getType := func(ctx context.Context, id sqlbase.ID) (*tree.TypeName, sqlbase.TypeDescriptorInterface, error) {
// TODO (rohany): Use the collection here.
return resolver.ResolveTypeDescByID(ctx, txn, tc.codec(), id, tree.ObjectLookupFlags{})
}
switch t := desc.(type) {
case *sqlbase.MutableTableDescriptor:
// It is safe to hydrate directly into MutableTableDescriptor since it is
// not shared.
return desc, sqlbase.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), sqlbase.TypeLookupFunc(getType))
case *sqlbase.ImmutableTableDescriptor:
// ImmutableTableDescriptors need to be copied before hydration, because
// they are potentially read by multiple threads. If there aren't any user
// defined types in the descriptor, then return early.
if !t.ContainsUserDefinedTypes() {
return desc, nil
}

// TODO (rohany, ajwerner): Here we would look into the cached set of
// hydrated table descriptors and potentially return without having to
// make a copy. However, we could avoid hitting the cache if any of the
// user defined types have been modified in this transaction.

// Make a copy of the underlying descriptor before hydration.
descBase := protoutil.Clone(t.TableDesc()).(*sqlbase.TableDescriptor)
if err := sqlbase.HydrateTypesInTableDescriptor(ctx, descBase, sqlbase.TypeLookupFunc(getType)); err != nil {
return nil, err
}
return sqlbase.NewImmutableTableDescriptor(*descBase), nil
default:
return desc, nil
}
}

// ResolveSchemaID attempts to lookup the schema from the schemaCache if it exists,
// otherwise falling back to a database lookup.
func (tc *Collection) ResolveSchemaID(
Expand Down Expand Up @@ -251,18 +314,8 @@ func (tc *Collection) ResolveSchemaID(
return exists, schemaID, err
}

// GetTableVersion returns a table descriptor with a version suitable for
// the transaction: table.ModificationTime <= txn.Timestamp < expirationTime.
// The table must be released by calling tc.ReleaseAll().
//
// If flags.required is false, GetTableVersion() will gracefully
// return a nil descriptor and no error if the table does not exist.
//
// It might also add a transaction deadline to the transaction that is
// enforced at the KV layer to ensure that the transaction doesn't violate
// the validity window of the table descriptor version returned.
//
func (tc *Collection) GetTableVersion(
// getTableVersionImpl performs the main logic of GetTableVersion.
func (tc *Collection) getTableVersionImpl(
ctx context.Context, txn *kv.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
) (*sqlbase.ImmutableTableDescriptor, error) {
if log.V(2) {
Expand Down Expand Up @@ -403,8 +456,36 @@ func (tc *Collection) GetTableVersion(
return table, nil
}

// GetTableVersionByID is a by-ID variant of GetTableVersion (i.e. uses same cache).
func (tc *Collection) GetTableVersionByID(
// GetTableVersion returns a table descriptor with a version suitable for
// the transaction: table.ModificationTime <= txn.Timestamp < expirationTime.
// The table must be released by calling tc.ReleaseAll().
//
// If flags.required is false, GetTableVersion() will gracefully
// return a nil descriptor and no error if the table does not exist.
//
// It might also add a transaction deadline to the transaction that is
// enforced at the KV layer to ensure that the transaction doesn't violate
// the validity window of the table descriptor version returned.
//
func (tc *Collection) GetTableVersion(
ctx context.Context, txn *kv.Txn, tn *tree.TableName, flags tree.ObjectLookupFlags,
) (*sqlbase.ImmutableTableDescriptor, error) {
desc, err := tc.getTableVersionImpl(ctx, txn, tn, flags)
if err != nil {
return nil, err
}
if desc != nil {
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, desc)
if err != nil {
return nil, err
}
desc = hydrated.(*sqlbase.ImmutableTableDescriptor)
}
return desc, nil
}

// getTableVersionByID performs the main logic of getTableVersionByID.
func (tc *Collection) getTableVersionByIDImpl(
ctx context.Context, txn *kv.Txn, tableID sqlbase.ID, flags tree.ObjectLookupFlags,
) (*sqlbase.ImmutableTableDescriptor, error) {
log.VEventf(ctx, 2, "planner getting table on table ID %d", tableID)
Expand Down Expand Up @@ -474,6 +555,24 @@ func (tc *Collection) GetTableVersionByID(
return table, nil
}

// GetTableVersionByID is a by-ID variant of GetTableVersion (i.e. uses same cache).
func (tc *Collection) GetTableVersionByID(
ctx context.Context, txn *kv.Txn, tableID sqlbase.ID, flags tree.ObjectLookupFlags,
) (*sqlbase.ImmutableTableDescriptor, error) {
desc, err := tc.getTableVersionByIDImpl(ctx, txn, tableID, flags)
if err != nil {
return nil, err
}
if desc != nil {
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, desc)
if err != nil {
return nil, err
}
desc = hydrated.(*sqlbase.ImmutableTableDescriptor)
}
return desc, nil
}

// GetMutableTableVersionByID is a variant of sqlbase.GetTableDescFromID which returns a mutable
// table descriptor of the table modified in the same transaction.
func (tc *Collection) GetMutableTableVersionByID(
Expand All @@ -485,7 +584,19 @@ func (tc *Collection) GetMutableTableVersionByID(
log.VEventf(ctx, 2, "found uncommitted table %d", tableID)
return table, nil
}
return sqlbase.GetMutableTableDescFromID(ctx, txn, tc.codec(), tableID)
desc, err := sqlbase.GetMutableTableDescFromID(ctx, txn, tc.codec(), tableID)
if err != nil {
return nil, err
}
// If we found a descriptor, then attempt to hydrate the types in it.
if desc != nil {
hydrated, err := tc.hydrateTypesInTableDesc(ctx, txn, desc)
if err != nil {
return nil, err
}
desc = hydrated.(*sqlbase.MutableTableDescriptor)
}
return desc, nil
}

// ReleaseTableLeases releases the leases for the tables with ids in
Expand Down
8 changes: 1 addition & 7 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -490,13 +490,7 @@ func (p *planner) LookupTableByID(
}
return catalog.TableEntry{}, err
}
// TODO (rohany): This shouldn't be needed once the descs.Collection always
// returns descriptors with hydrated types.
hydratedDesc, err := p.maybeHydrateTypesInDescriptor(ctx, table)
if err != nil {
return catalog.TableEntry{}, err
}
return catalog.TableEntry{Desc: hydratedDesc.(*sqlbase.ImmutableTableDescriptor)}, nil
return catalog.TableEntry{Desc: table}, nil
}

// TypeAsString enforces (not hints) that the given expression typechecks as a
Expand Down
43 changes: 0 additions & 43 deletions pkg/sql/resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand All @@ -25,7 +24,6 @@ import (
"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/protoutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -128,15 +126,6 @@ func (p *planner) LookupObject(
sc := p.LogicalSchemaAccessor()
lookupFlags.CommonLookupFlags = p.CommonLookupFlags(false /* required */)
objDesc, err := sc.GetObjectDesc(ctx, p.txn, p.ExecCfg().Settings, p.ExecCfg().Codec, dbName, scName, tbName, lookupFlags)

// The returned object may contain types.T that need hydrating.
if objDesc != nil {
objDesc, err = p.maybeHydrateTypesInDescriptor(ctx, objDesc)
if err != nil {
return false, nil, err
}
}

return objDesc != nil, objDesc, err
}

Expand Down Expand Up @@ -205,38 +194,6 @@ func (p *planner) ResolveTypeByID(ctx context.Context, id uint32) (*types.T, err
return desc.MakeTypesT(ctx, name, p)
}

// maybeHydrateTypesInDescriptor hydrates any types.T's in the input descriptor.
// TODO (rohany): Once we lease types, this should be pushed down into the
// leased object collection.
func (p *planner) maybeHydrateTypesInDescriptor(
ctx context.Context, objDesc catalog.Descriptor,
) (catalog.Descriptor, error) {
// As of now, only {Mutable,Immutable}TableDescriptor have types.T that
// need to be hydrated.
switch t := objDesc.(type) {
case *sqlbase.MutableTableDescriptor:
// MutableTableDescriptors are safe to modify in place.
if err := sqlbase.HydrateTypesInTableDescriptor(ctx, t.TableDesc(), p); err != nil {
return nil, err
}
return objDesc, nil
case *sqlbase.ImmutableTableDescriptor:
// ImmutableTableDescriptors need to be copied before hydration. If there
// aren't any user defined types in the descriptor, then just return.
if !t.ContainsUserDefinedTypes() {
return objDesc, nil
}
// Make a copy of the underlying TableDescriptor.
desc := protoutil.Clone(t.TableDesc()).(*sqlbase.TableDescriptor)
if err := sqlbase.HydrateTypesInTableDescriptor(ctx, desc, p); err != nil {
return nil, err
}
return sqlbase.NewImmutableTableDescriptor(*desc), nil
default:
return objDesc, nil
}
}

// ObjectLookupFlags is part of the resolver.SchemaResolver interface.
func (p *planner) ObjectLookupFlags(required, requireMutable bool) tree.ObjectLookupFlags {
return tree.ObjectLookupFlags{
Expand Down

0 comments on commit 91a0c1b

Please sign in to comment.