Skip to content

Commit

Permalink
sql/catalog/descs: remove allocations from hot path
Browse files Browse the repository at this point in the history
The lookup by ID path gets called constantly. This was over 1% of objects
allocated in some workloads. Here's a microbenchmark:

```
name                                                                    old time/op    new time/op    delta
ResolveExistingObject/CREATE_SCHEMA_sc;CREATE_TABLE_sc.foo_()sc.foo-16    2.62µs ± 1%    2.18µs ± 1%   -16.63%  (p=0.000 n=10+8)

name                                                                    old alloc/op   new alloc/op   delta
ResolveExistingObject/CREATE_SCHEMA_sc;CREATE_TABLE_sc.foo_()sc.foo-16      150B ± 0%        4B ± 0%   -97.33%  (p=0.001 n=8+9)

name                                                                    old allocs/op  new allocs/op  delta
ResolveExistingObject/CREATE_SCHEMA_sc;CREATE_TABLE_sc.foo_()sc.foo-16      12.0 ± 0%       0.0       -100.00%  (p=0.000 n=10+10)
```

Release note: None
  • Loading branch information
ajwerner committed Oct 4, 2022
1 parent 0eaeeb7 commit c0b535c
Showing 1 changed file with 80 additions and 27 deletions.
107 changes: 80 additions & 27 deletions pkg/sql/catalog/descs/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -73,11 +74,24 @@ func (tc *Collection) GetImmutableDescriptorsByID(
func (tc *Collection) GetImmutableDescriptorByID(
ctx context.Context, txn *kv.Txn, id descpb.ID, flags tree.CommonLookupFlags,
) (catalog.Descriptor, error) {
descs, err := tc.GetImmutableDescriptorsByID(ctx, txn, flags, id)
if err != nil {
flags.RequireMutable = false
return tc.getDescriptorByID(ctx, txn, flags, id)
}

// getDescriptorsByID returns a descriptor by ID according to the provided
// lookup flags.
//
// The Required flag is ignored and always overridden.
func (tc *Collection) getDescriptorByID(
ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, ids ...descpb.ID,
) (catalog.Descriptor, error) {
var arr [1]catalog.Descriptor
if err := getDescriptorsByID(
ctx, tc, txn, flags, arr[:], ids...,
); err != nil {
return nil, err
}
return descs[0], nil
return arr[0], nil
}

// getDescriptorsByID returns a slice of descriptors by ID according to the
Expand All @@ -86,12 +100,38 @@ func (tc *Collection) GetImmutableDescriptorByID(
// The Required flag is ignored and always overridden.
func (tc *Collection) getDescriptorsByID(
ctx context.Context, txn *kv.Txn, flags tree.CommonLookupFlags, ids ...descpb.ID,
) (descs []catalog.Descriptor, err error) {
) ([]catalog.Descriptor, error) {
descs := make([]catalog.Descriptor, len(ids))
if err := getDescriptorsByID(
ctx, tc, txn, flags, descs, ids...,
); err != nil {
return nil, err
}
return descs, nil
}

// getDescriptorsByID implements the Collection method of the same name.
// It takes a slice into which the retrieved descriptors will be stored.
// That slice must be the same length as the ids. This allows callers
// seeking to get just one descriptor to avoid an allocation by using a
// fixed-size array.
func getDescriptorsByID(
ctx context.Context,
tc *Collection,
txn *kv.Txn,
flags tree.CommonLookupFlags,
descs []catalog.Descriptor,
ids ...descpb.ID,
) (err error) {
// Override flags.
flags.Required = true
log.VEventf(ctx, 2, "looking up descriptors for ids %v", ids)
descs = make([]catalog.Descriptor, len(ids))
vls := make([]catalog.ValidationLevel, len(ids))
if log.ExpensiveLogEnabled(ctx, 2) {
// Copy the ids to a new slice to prevent the backing array from
// escaping and forcing IDs to escape on this hot path.
idsForLog := append(make([]descpb.ID, 0, len(ids)), ids...)
log.VEventf(ctx, 2, "looking up descriptors for ids %v", idsForLog)
}
var vls validationLevelMap
{
// Look up the descriptors in all layers except the storage layer on a
// best-effort basis.
Expand All @@ -101,7 +141,10 @@ func (tc *Collection) getDescriptorsByID(
tc: tc,
flags: flags,
}
for _, fn := range []func(id descpb.ID) (catalog.Descriptor, catalog.ValidationLevel, error){
type lookupFunc = func(
id descpb.ID,
) (catalog.Descriptor, catalog.ValidationLevel, error)
for _, fn := range []lookupFunc{
q.lookupVirtual,
q.lookupTemporary,
q.lookupSynthetic,
Expand All @@ -115,13 +158,13 @@ func (tc *Collection) getDescriptorsByID(
}
desc, vl, err := fn(id)
if err != nil {
return nil, err
return err
}
if desc == nil {
continue
}
descs[i] = desc
vls[i] = vl
vls.set(i, vl)
}
}
}
Expand All @@ -135,29 +178,29 @@ func (tc *Collection) getDescriptorsByID(
}
if !readIDs.Empty() {
if err = tc.stored.EnsureFromStorageByIDs(ctx, txn, readIDs, catalog.Any); err != nil {
return nil, err
return err
}
for i, id := range ids {
if descs[i] == nil {
descs[i] = tc.stored.GetCachedByID(id)
vls[i] = tc.stored.GetValidationLevelByID(id)
vls.set(i, tc.stored.GetValidationLevelByID(id))
}
}
}

// At this point, all descriptors are in the slice, finalize and hydrate them.
if err := tc.finalizeDescriptors(ctx, txn, flags, descs, vls); err != nil {
return nil, err
if err := tc.finalizeDescriptors(ctx, txn, flags, descs, &vls); err != nil {
return err
}
if err := tc.hydrateDescriptors(ctx, txn, flags, descs); err != nil {
return nil, err
return err
}
for _, desc := range descs {
if err := catalog.FilterDescriptorState(desc, flags); err != nil {
return nil, err
return err
}
}
return descs, nil
return nil
}

// byIDLookupContext is a helper struct for getDescriptorsByID which contains
Expand Down Expand Up @@ -277,7 +320,8 @@ func (tc *Collection) getDescriptorByName(
// When looking up descriptors by name, then descriptors in the adding state
// must be uncommitted to be visible (among other things).
flags.AvoidCommittedAdding = true
descs, err := tc.getDescriptorsByID(ctx, txn, flags, id)

desc, err := tc.getDescriptorByID(ctx, txn, flags, id)
if err != nil {
// Swallow error if the descriptor is dropped.
if errors.Is(err, catalog.ErrDescriptorDropped) {
Expand All @@ -294,7 +338,6 @@ func (tc *Collection) getDescriptorByName(
}
return nil, err
}
desc := descs[0]
if desc.GetName() != name && !(desc.DescriptorType() == catalog.Schema && isTemporarySchema(name)) {
// TODO(postamar): make Collection aware of name ops
//
Expand Down Expand Up @@ -484,13 +527,8 @@ func (tc *Collection) finalizeDescriptors(
txn *kv.Txn,
flags tree.CommonLookupFlags,
descs []catalog.Descriptor,
validationLevels []catalog.ValidationLevel,
validationLevels *validationLevelMap,
) error {
if len(validationLevels) != len(descs) {
return errors.AssertionFailedf(
"len(validationLevels) = %d should be equal to len(descs) = %d",
len(validationLevels), len(descs))
}
// Add the descriptors to the uncommitted layer if we want them to be mutable.
if flags.RequireMutable {
for i, desc := range descs {
Expand All @@ -507,8 +545,8 @@ func (tc *Collection) finalizeDescriptors(
requiredLevel = validate.ImmutableRead
}
var toValidate []catalog.Descriptor
for i, vl := range validationLevels {
if vl < requiredLevel {
for i := range descs {
if validationLevels.get(i) < requiredLevel {
toValidate = append(toValidate, descs[i])
}
}
Expand All @@ -533,3 +571,18 @@ func (tc *Collection) deadlineHolder(txn *kv.Txn) deadlineHolder {
func isTemporarySchema(name string) bool {
return strings.HasPrefix(name, catconstants.PgTempSchemaName)
}

// validationLevelMap is used to map indexes in a slice to their required
// validation level. This avoids any heap allocations as compared to a
// slice containing the correspondence between an index to a validation level.
type validationLevelMap struct {
m util.FastIntMap
}

func (vlm *validationLevelMap) get(idx int) catalog.ValidationLevel {
return catalog.ValidationLevel(vlm.m.GetDefault(idx))
}

func (vlm *validationLevelMap) set(idx int, l catalog.ValidationLevel) {
vlm.m.Set(idx, int(l))
}

0 comments on commit c0b535c

Please sign in to comment.