Skip to content

Commit

Permalink
descs,*: refactor by-ID descriptor lookups
Browse files Browse the repository at this point in the history
This commit refactors the by-ID getters and builders API with sane
defaults for leasing. Access to leased descriptors is now explicitly
opted in.

Informs #87753.

Release note: None
  • Loading branch information
Marius Posta committed Jan 5, 2023
1 parent 9fc8c24 commit a39452b
Show file tree
Hide file tree
Showing 110 changed files with 412 additions and 409 deletions.
8 changes: 4 additions & 4 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10387,7 +10387,7 @@ $$;
require.NoError(t, err)
require.Equal(t, 110, int(tbDesc.GetID()))

fnDesc, err := col.ByID(txn).WithoutNonPublic().Immutable().Function(ctx, descpb.ID(udfID))
fnDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Function(ctx, descpb.ID(udfID))
require.NoError(t, err)
require.Equal(t, 111, int(fnDesc.GetID()))
require.Equal(t, 104, int(fnDesc.GetParentID()))
Expand Down Expand Up @@ -10443,7 +10443,7 @@ $$;
require.NoError(t, err)
require.Equal(t, 118, int(tbDesc.GetID()))

fnDesc, err := col.ByID(txn).WithoutNonPublic().Immutable().Function(ctx, descpb.ID(udfID))
fnDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Function(ctx, descpb.ID(udfID))
require.NoError(t, err)
require.Equal(t, 119, int(fnDesc.GetID()))
require.Equal(t, 112, int(fnDesc.GetParentID()))
Expand Down Expand Up @@ -10534,7 +10534,7 @@ $$;
require.NoError(t, err)
require.Equal(t, 110, int(tbDesc.GetID()))

fnDesc, err := col.ByID(txn).WithoutNonPublic().Immutable().Function(ctx, descpb.ID(udfID))
fnDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Function(ctx, descpb.ID(udfID))
require.NoError(t, err)
require.Equal(t, 111, int(fnDesc.GetID()))
require.Equal(t, 104, int(fnDesc.GetParentID()))
Expand Down Expand Up @@ -10592,7 +10592,7 @@ $$;
require.NoError(t, err)
require.Equal(t, 129, int(tbDesc.GetID()))

fnDesc, err := col.ByID(txn)..WithPublic().Immutable().Function(ctx, descpb.ID(udfID))
fnDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Function(ctx, descpb.ID(udfID))
require.NoError(t, err)
require.Equal(t, 130, int(fnDesc.GetID()))
require.Equal(t, 107, int(fnDesc.GetParentID()))
Expand Down
32 changes: 16 additions & 16 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -1073,7 +1073,7 @@ func createImportingDescriptors(
// Write the updated databases.
for dbID, schemas := range existingDBsWithNewSchemas {
log.Infof(ctx, "writing %d schema entries to database %d", len(schemas), dbID)
desc, err := descsCol.ByID(txn).Mutable().Desc(ctx, dbID)
desc, err := descsCol.MutableByID(txn).Desc(ctx, dbID)
if err != nil {
return err
}
Expand All @@ -1093,7 +1093,7 @@ func createImportingDescriptors(
// to the new tables being restored.
for _, table := range mutableTables {
// Collect all types used by this table.
dbDesc, err := descsCol.ByID(txn).WithoutDropped().WithoutLeased().Immutable().Database(ctx, table.GetParentID())
dbDesc, err := descsCol.ByID(txn).WithoutDropped().Get().Database(ctx, table.GetParentID())
if err != nil {
return err
}
Expand All @@ -1115,7 +1115,7 @@ func createImportingDescriptors(
continue
}
// Otherwise, add a backreference to this table.
typDesc, err := descsCol.ByID(txn).Mutable().Type(ctx, id)
typDesc, err := descsCol.MutableByID(txn).Type(ctx, id)
if err != nil {
return err
}
Expand All @@ -1139,7 +1139,7 @@ func createImportingDescriptors(
if details.DescriptorCoverage != tree.AllDescriptors {
for _, table := range tableDescs {
if lc := table.GetLocalityConfig(); lc != nil {
desc, err := descsCol.ByID(txn).WithoutDropped().WithoutLeased().Immutable().Database(ctx, table.ParentID)
desc, err := descsCol.ByID(txn).WithoutDropped().Get().Database(ctx, table.ParentID)
if err != nil {
return err
}
Expand All @@ -1149,7 +1149,7 @@ func createImportingDescriptors(
table.ID, table.ParentID)
}

mutTable, err := descsCol.ByID(txn).Mutable().Table(ctx, table.GetID())
mutTable, err := descsCol.MutableByID(txn).Table(ctx, table.GetID())
if err != nil {
return err
}
Expand Down Expand Up @@ -2228,7 +2228,7 @@ func prefetchDescriptors(
// and we're going to write them to KV very soon as part of a
// single batch).
ids := allDescIDs.Ordered()
got, err := descsCol.ByID(txn).Mutable().Descs(ctx, ids)
got, err := descsCol.MutableByID(txn).Descs(ctx, ids)
if err != nil {
return nstree.Catalog{}, errors.Wrap(err, "prefetch descriptors")
}
Expand Down Expand Up @@ -2360,7 +2360,7 @@ func (r *restoreResumer) dropDescriptors(
mutableTables := make([]*tabledesc.Mutable, len(details.TableDescs))
for i := range details.TableDescs {
var err error
mutableTables[i], err = descsCol.ByID(txn).Mutable().Table(ctx, details.TableDescs[i].ID)
mutableTables[i], err = descsCol.MutableByID(txn).Table(ctx, details.TableDescs[i].ID)
if err != nil {
return err
}
Expand Down Expand Up @@ -2439,7 +2439,7 @@ func (r *restoreResumer) dropDescriptors(
// TypeDescriptors don't have a GC job process, so we can just write them
// as dropped here.
typDesc := details.TypeDescs[i]
mutType, err := descsCol.ByID(txn).Mutable().Type(ctx, typDesc.ID)
mutType, err := descsCol.MutableByID(txn).Type(ctx, typDesc.ID)
if err != nil {
return err
}
Expand All @@ -2455,7 +2455,7 @@ func (r *restoreResumer) dropDescriptors(

for i := range details.FunctionDescs {
fnDesc := details.FunctionDescs[i]
mutFn, err := descsCol.ByID(txn).Mutable().Function(ctx, fnDesc.ID)
mutFn, err := descsCol.MutableByID(txn).Function(ctx, fnDesc.ID)
if err != nil {
return err
}
Expand Down Expand Up @@ -2527,13 +2527,13 @@ func (r *restoreResumer) dropDescriptors(
continue
}

mutSchema, err := descsCol.ByID(txn).Mutable().Desc(ctx, schemaDesc.GetID())
mutSchema, err := descsCol.MutableByID(txn).Desc(ctx, schemaDesc.GetID())
if err != nil {
return err
}
entry, hasEntry := dbsWithDeletedSchemas[schemaDesc.GetParentID()]
if !hasEntry {
mutParent, err := descsCol.ByID(txn).Mutable().Desc(ctx, schemaDesc.GetParentID())
mutParent, err := descsCol.MutableByID(txn).Desc(ctx, schemaDesc.GetParentID())
if err != nil {
return err
}
Expand Down Expand Up @@ -2604,7 +2604,7 @@ func (r *restoreResumer) dropDescriptors(
continue
}

db, err := descsCol.ByID(txn).Mutable().Desc(ctx, dbDesc.GetID())
db, err := descsCol.MutableByID(txn).Desc(ctx, dbDesc.GetID())
if err != nil {
return err
}
Expand Down Expand Up @@ -2658,12 +2658,12 @@ func setGCTTLForDroppingTable(
log.VInfof(ctx, 2, "lowering TTL for table %q (%d)", tableToDrop.GetName(), tableToDrop.GetID())
// We get a mutable descriptor here because we are going to construct a
// synthetic descriptor collection in which they are online.
dbDesc, err := descsCol.ByID(txn).WithoutLeased().Immutable().Database(ctx, tableToDrop.GetParentID())
dbDesc, err := descsCol.ByID(txn).Get().Database(ctx, tableToDrop.GetParentID())
if err != nil {
return err
}

schemaDesc, err := descsCol.ByID(txn).WithoutLeased().Immutable().Schema(ctx, tableToDrop.GetParentSchemaID())
schemaDesc, err := descsCol.ByID(txn).Get().Schema(ctx, tableToDrop.GetParentSchemaID())
if err != nil {
return err
}
Expand Down Expand Up @@ -2722,15 +2722,15 @@ func (r *restoreResumer) removeExistingTypeBackReferences(
return restored, nil
}
// Finally, look it up using the transaction.
typ, err := descsCol.ByID(txn).Mutable().Type(ctx, id)
typ, err := descsCol.MutableByID(txn).Type(ctx, id)
if err != nil {
return nil, err
}
existingTypes[typ.GetID()] = typ
return typ, nil
}

dbDesc, err := descsCol.ByID(txn).WithoutDropped().WithoutLeased().Immutable().Database(ctx, tbl.GetParentID())
dbDesc, err := descsCol.ByID(txn).WithoutDropped().Get().Database(ctx, tbl.GetParentID())
if err != nil {
return err
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@ func allocateDescriptorRewrites(
} else {
// If we found an existing schema, then we need to remap all references
// to this schema to the existing one.
desc, err := col.ByID(txn).WithoutLeased().Immutable().Schema(ctx, id)
desc, err := col.ByID(txn).Get().Schema(ctx, id)
if err != nil {
return err
}
Expand Down Expand Up @@ -381,7 +381,7 @@ func allocateDescriptorRewrites(
}

// Check privileges.
parentDB, err := col.ByID(txn).WithoutLeased().Immutable().Database(ctx, parentID)
parentDB, err := col.ByID(txn).Get().Database(ctx, parentID)
if err != nil {
return errors.Wrapf(err,
"failed to lookup parent DB %d", errors.Safe(parentID))
Expand Down Expand Up @@ -446,7 +446,7 @@ func allocateDescriptorRewrites(
targetDB, typ.Name)
}
// Check privileges on the parent DB.
parentDB, err := col.ByID(txn).WithoutLeased().Immutable().Database(ctx, parentID)
parentDB, err := col.ByID(txn).Get().Database(ctx, parentID)
if err != nil {
return errors.Wrapf(err,
"failed to lookup parent DB %d", errors.Safe(parentID))
Expand Down Expand Up @@ -694,7 +694,7 @@ func getDatabaseIDAndDesc(
return dbID, nil, errors.Errorf("a database named %q needs to exist", targetDB)
}
// Check privileges on the parent DB.
dbDesc, err = col.ByID(txn).WithoutLeased().Immutable().Database(ctx, dbID)
dbDesc, err = col.ByID(txn).Get().Database(ctx, dbID)
if err != nil {
return 0, nil, errors.Wrapf(err,
"failed to lookup parent DB %d", errors.Safe(dbID))
Expand Down Expand Up @@ -2034,7 +2034,7 @@ func renameTargetDatabaseDescriptor(
}
db, ok := restoreDBs[0].(*dbdesc.Mutable)
if !ok {
return errors.AssertionFailedf("expected *dbdesc.Mutable but found %T", db)
return errors.AssertionFailedf("expected *dbdesc.mutable but found %T", db)
}
db.SetName(newDBName)
return nil
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -595,7 +595,7 @@ func checkMultiRegionCompatible(
// For REGION BY TABLE IN <region> tables, allow the restore if the
// database has the region.
regionEnumID := database.GetRegionConfig().RegionEnumID
regionEnum, err := col.ByID(txn).WithoutLeased().Immutable().Type(ctx, regionEnumID)
regionEnum, err := col.ByID(txn).Get().Type(ctx, regionEnumID)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/cdcevent/rowfetcher_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ func refreshUDT(
if err != nil {
return err
}
tableDesc, err = collection.ByID(txn).WithoutNonPublic().Immutable().Table(ctx, tableID)
tableDesc, err = collection.ByIDWithLeased(txn).WithoutNonPublic().Get().Table(ctx, tableID)
return err
}); err != nil {
// Manager can return all kinds of errors during chaos, but based on
Expand Down Expand Up @@ -189,7 +189,7 @@ func (c *rowFetcherCache) RowFetcherForColumnFamily(
// version and the desired version to use the cache. It is safe to use
// UserDefinedTypeColsHaveSameVersion if we have a hit because we are
// guaranteed that the tables have the same version. Additionally, these
// fetchers are always initialized with a single tabledesc.Immutable.
// fetchers are always initialized with a single tabledesc.Get.
if safe, err := catalog.UserDefinedTypeColsInFamilyHaveSameVersion(tableDesc, f.tableDesc, family); err != nil {
return nil, nil, err
} else if safe {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ func fetchTableDescriptors(
// and lie within the primary index span. Deduplication is important
// here as requesting the same span twice will deadlock.
return targets.EachTableID(func(id catid.DescID) error {
tableDesc, err := descriptors.ByID(txn).WithoutNonPublic().WithoutLeased().Immutable().Table(ctx, id)
tableDesc, err := descriptors.ByID(txn).WithoutNonPublic().Get().Table(ctx, id)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -1177,11 +1177,11 @@ func getQualifiedTableNameObj(
ctx context.Context, execCfg *sql.ExecutorConfig, txn *kv.Txn, desc catalog.TableDescriptor,
) (tree.TableName, error) {
col := execCfg.CollectionFactory.NewCollection(ctx)
db, err := col.ByID(txn).WithoutLeased().Immutable().Database(ctx, desc.GetParentID())
db, err := col.ByID(txn).Get().Database(ctx, desc.GetParentID())
if err != nil {
return tree.TableName{}, err
}
sc, err := col.ByID(txn).WithoutLeased().Immutable().Schema(ctx, desc.GetParentSchemaID())
sc, err := col.ByID(txn).Get().Schema(ctx, desc.GetParentSchemaID())
if err != nil {
return tree.TableName{}, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/schemafeed/schema_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,7 @@ func (tf *schemaFeed) primeInitialTableDescs(ctx context.Context) error {
}
// Note that all targets are currently guaranteed to be tables.
return tf.targets.EachTableID(func(id descpb.ID) error {
tableDesc, err := descriptors.ByID(txn).WithoutNonPublic().WithoutLeased().Immutable().Table(ctx, id)
tableDesc, err := descriptors.ByID(txn).WithoutNonPublic().Get().Table(ctx, id)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/multiregionccl/region_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,11 +123,11 @@ func getEnumMembers(
t.Helper()
enumMembers := make(map[string][]byte)
err := sql.TestingDescsTxn(ctx, ts, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error {
dbDesc, err := descsCol.ByID(txn).WithoutNonPublic().Immutable().Database(ctx, dbID)
dbDesc, err := descsCol.ByIDWithLeased(txn).WithoutNonPublic().Get().Database(ctx, dbID)
require.NoError(t, err)
regionEnumID, err := dbDesc.MultiRegionEnumID()
require.NoError(t, err)
regionEnumDesc, err := descsCol.ByID(txn).WithoutNonPublic().Immutable().Type(ctx, regionEnumID)
regionEnumDesc, err := descsCol.ByIDWithLeased(txn).WithoutNonPublic().Get().Type(ctx, regionEnumID)
require.NoError(t, err)
for ord := 0; ord < regionEnumDesc.NumEnumMembers(); ord++ {
enumMembers[regionEnumDesc.GetMemberLogicalRepresentation(ord)] = regionEnumDesc.GetMemberPhysicalRepresentation(ord)
Expand Down
4 changes: 2 additions & 2 deletions pkg/scheduledjobs/schedulebase/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,13 +191,13 @@ func FullyQualifyTables(
}

// Resolve the database.
dbDesc, err := col.ByID(txn).WithoutNonPublic().Immutable().Database(ctx, tableDesc.GetParentID())
dbDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Database(ctx, tableDesc.GetParentID())
if err != nil {
return err
}

// Resolve the schema.
schemaDesc, err := col.ByID(txn).WithoutNonPublic().Immutable().Schema(ctx, tableDesc.GetParentSchemaID())
schemaDesc, err := col.ByIDWithLeased(txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID())
if err != nil {
return err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -2469,7 +2469,7 @@ func (s *statusServer) HotRangesV2(
} else {
if err = s.sqlServer.distSQLServer.InternalExecutorFactory.DescsTxnWithExecutor(
ctx, s.db, nil, func(ctx context.Context, txn *kv.Txn, col *descs.Collection, ie sqlutil.InternalExecutor) error {
desc, err := col.ByID(txn).WithoutNonPublic().WithoutLeased().Immutable().Table(ctx, descpb.ID(tableID))
desc, err := col.ByID(txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tableID))
if err != nil {
return errors.Wrapf(err, "cannot get table descriptor with tableID: %d, %s", tableID, r.Desc)
}
Expand All @@ -2487,13 +2487,13 @@ func (s *statusServer) HotRangesV2(
}
}

if dbDesc, err := col.ByID(txn).WithoutNonPublic().WithoutLeased().Immutable().Database(ctx, desc.GetParentID()); err != nil {
if dbDesc, err := col.ByID(txn).WithoutNonPublic().Get().Database(ctx, desc.GetParentID()); err != nil {
log.Warningf(ctx, "cannot get database by descriptor ID: %s: %v", r.Desc, err)
} else {
dbName = dbDesc.GetName()
}

if schemaDesc, err := col.ByID(txn).WithoutNonPublic().WithoutLeased().Immutable().Schema(ctx, desc.GetParentSchemaID()); err != nil {
if schemaDesc, err := col.ByID(txn).WithoutNonPublic().Get().Schema(ctx, desc.GetParentSchemaID()); err != nil {
log.Warningf(ctx, "cannot get schema name for range descriptor: %s: %v", r.Desc, err)
} else {
schemaName = schemaDesc.GetName()
Expand Down
2 changes: 1 addition & 1 deletion pkg/spanconfig/spanconfigreconciler/reconciler.go
Original file line number Diff line number Diff line change
Expand Up @@ -642,7 +642,7 @@ func (r *incrementalReconciler) filterForMissingTableIDs(
continue // nothing to do
}

desc, err := descsCol.ByID(txn).WithoutLeased().Immutable().Desc(ctx, descriptorUpdate.ID)
desc, err := descsCol.ByID(txn).Get().Desc(ctx, descriptorUpdate.ID)

considerAsMissing := false
if errors.Is(err, catalog.ErrDescriptorNotFound) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,7 @@ func (s *SQLTranslator) generateSpanConfigurations(
}

// We're dealing with a SQL object.
desc, err := descsCol.ByID(txn).WithoutLeased().Immutable().Desc(ctx, id)
desc, err := descsCol.ByID(txn).Get().Desc(ctx, id)
if err != nil {
if errors.Is(err, catalog.ErrDescriptorNotFound) {
return nil, nil // the descriptor has been deleted; nothing to do here
Expand Down Expand Up @@ -529,7 +529,7 @@ func (s *SQLTranslator) findDescendantLeafIDs(
func (s *SQLTranslator) findDescendantLeafIDsForDescriptor(
ctx context.Context, id descpb.ID, txn *kv.Txn, descsCol *descs.Collection,
) (descpb.IDs, error) {
desc, err := descsCol.ByID(txn).WithoutLeased().Immutable().Desc(ctx, id)
desc, err := descsCol.ByID(txn).Get().Desc(ctx, id)
if err != nil {
if errors.Is(err, catalog.ErrDescriptorNotFound) {
return nil, nil // the descriptor has been deleted; nothing to do here
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ func (s *Tenant) LookupTableDescriptorByID(
ctx context.Context, txn *kv.Txn, descsCol *descs.Collection,
) error {
var err error
desc, err = descsCol.ByID(txn).WithoutLeased().Immutable().Table(ctx, id)
desc, err = descsCol.ByID(txn).Get().Table(ctx, id)
return err
}))
return desc
Expand Down
Loading

0 comments on commit a39452b

Please sign in to comment.