diff --git a/pkg/sql/catalog/descs/uncommitted_descriptors.go b/pkg/sql/catalog/descs/uncommitted_descriptors.go index d353071a9c3f..337626774ad1 100644 --- a/pkg/sql/catalog/descs/uncommitted_descriptors.go +++ b/pkg/sql/catalog/descs/uncommitted_descriptors.go @@ -11,6 +11,7 @@ package descs import ( + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -100,11 +101,16 @@ type uncommittedDescriptors struct { // // TODO(postamar): better uncommitted namespace changes handling after 22.1. descNames nstree.Set + + // addedSystemDatabase is used to mark whether the optimization to add the + // system database to the set of uncommitted descriptors has occurred. + addedSystemDatabase bool } func (ud *uncommittedDescriptors) reset() { ud.descs.Clear() ud.descNames.Clear() + ud.addedSystemDatabase = false } // add adds a descriptor to the set of uncommitted descriptors and returns @@ -124,7 +130,9 @@ func (ud *uncommittedDescriptors) add(mut catalog.MutableDescriptor) (catalog.De // checkOut checks out an uncommitted mutable descriptor for use in the // transaction. This descriptor should later be checked in again. func (ud *uncommittedDescriptors) checkOut(id descpb.ID) (catalog.MutableDescriptor, error) { - ud.maybeInitialize() + if id == keys.SystemDatabaseID { + ud.maybeAddSystemDatabase() + } entry := ud.descs.GetByID(id) if entry == nil { return nil, errors.NewAssertionErrorWithWrappedErrf( @@ -182,7 +190,9 @@ func maybeRefreshCachedFieldsOnTypeDescriptor( // getByID looks up an uncommitted descriptor by ID. func (ud *uncommittedDescriptors) getByID(id descpb.ID) catalog.Descriptor { - ud.maybeInitialize() + if id == keys.SystemDatabaseID && !ud.addedSystemDatabase { + ud.maybeAddSystemDatabase() + } entry := ud.descs.GetByID(id) if entry == nil { return nil @@ -200,12 +210,18 @@ func (ud *uncommittedDescriptors) getByID(id descpb.ID) catalog.Descriptor { func (ud *uncommittedDescriptors) getByName( dbID descpb.ID, schemaID descpb.ID, name string, ) (hasKnownRename bool, desc catalog.Descriptor) { - ud.maybeInitialize() + if dbID == 0 && schemaID == 0 && name == systemschema.SystemDatabaseName { + ud.maybeAddSystemDatabase() + } // Walk latest to earliest so that a DROP followed by a CREATE with the same // name will result in the CREATE being seen. if got := ud.descs.GetByName(dbID, schemaID, name); got != nil { return false, got.(*uncommittedDescriptor).immutable } + // Check whether the set is empty to avoid allocating the NameInfo. + if ud.descNames.Empty() { + return false, nil + } return ud.descNames.Contains(descpb.NameInfo{ ParentID: dbID, ParentSchemaID: schemaID, @@ -216,7 +232,6 @@ func (ud *uncommittedDescriptors) getByName( func (ud *uncommittedDescriptors) iterateNewVersionByID( fn func(entry catalog.NameEntry, originalVersion lease.IDVersion) error, ) error { - ud.maybeInitialize() return ud.descs.IterateByID(func(entry catalog.NameEntry) error { mut := entry.(*uncommittedDescriptor).mutable if mut == nil || mut.IsNew() || !mut.IsUncommittedVersion() { @@ -229,7 +244,6 @@ func (ud *uncommittedDescriptors) iterateNewVersionByID( func (ud *uncommittedDescriptors) iterateImmutableByID( fn func(imm catalog.Descriptor) error, ) error { - ud.maybeInitialize() return ud.descs.IterateByID(func(entry catalog.NameEntry) error { return fn(entry.(*uncommittedDescriptor).immutable) }) @@ -286,8 +300,9 @@ var systemUncommittedDatabase = &uncommittedDescriptor{ // value lazily when this is needed, which ought to be exceedingly rare. } -func (ud *uncommittedDescriptors) maybeInitialize() { - if ud.descs.Len() == 0 { +func (ud *uncommittedDescriptors) maybeAddSystemDatabase() { + if !ud.addedSystemDatabase { + ud.addedSystemDatabase = true ud.descs.Upsert(systemUncommittedDatabase) } } diff --git a/pkg/sql/catalog/lease/descriptor_state.go b/pkg/sql/catalog/lease/descriptor_state.go index a6c75aa7b7c3..75112018c413 100644 --- a/pkg/sql/catalog/lease/descriptor_state.go +++ b/pkg/sql/catalog/lease/descriptor_state.go @@ -94,6 +94,7 @@ type descriptorState struct { func (t *descriptorState) findForTimestamp( ctx context.Context, timestamp hlc.Timestamp, ) (*descriptorVersionState, bool, error) { + expensiveLogEnabled := log.ExpensiveLogEnabled(ctx, 2) t.mu.Lock() defer t.mu.Unlock() @@ -109,7 +110,7 @@ func (t *descriptorState) findForTimestamp( latest := i+1 == len(t.mu.active.data) if !desc.hasExpired(timestamp) { // Existing valid descriptor version. - desc.incRefCount(ctx) + desc.incRefCount(ctx, expensiveLogEnabled) return desc, latest, nil } @@ -220,29 +221,37 @@ func (t *descriptorState) release(ctx context.Context, s *descriptorVersionState // Decrements the refcount and returns true if the lease has to be removed // from the store. - decRefcount := func(s *descriptorVersionState) *storedLease { + expensiveLoggingEnabled := log.ExpensiveLogEnabled(ctx, 2) + decRefCount := func(s *descriptorVersionState) (shouldRemove bool) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.refcount-- + if expensiveLoggingEnabled { + log.Infof(ctx, "release: %s", s.stringLocked()) + } + return s.mu.refcount == 0 + } + maybeMarkRemoveStoredLease := func(s *descriptorVersionState) *storedLease { // Figure out if we'd like to remove the lease from the store asap (i.e. // when the refcount drops to 0). If so, we'll need to mark the lease as // invalid. - removeOnceDereferenced := t.m.removeOnceDereferenced() || + removeOnceDereferenced := // Release from the store if the descriptor has been dropped or taken // offline. t.mu.takenOffline || - // Release from the store if the lease is not for the latest - // version; only leases for the latest version can be acquired. - s != t.mu.active.findNewest() || - s.GetVersion() < t.mu.maxVersionSeen - + // Release from the store if the lease is not for the latest + // version; only leases for the latest version can be acquired. + s != t.mu.active.findNewest() || + s.GetVersion() < t.mu.maxVersionSeen || + t.m.removeOnceDereferenced() + if !removeOnceDereferenced { + return nil + } s.mu.Lock() defer s.mu.Unlock() - s.mu.refcount-- - if log.ExpensiveLogEnabled(ctx, 2) { - log.Infof(ctx, "release: %s", s.stringLocked()) - } if s.mu.refcount < 0 { panic(errors.AssertionFailedf("negative ref count: %s", s)) } - if s.mu.refcount == 0 && s.mu.lease != nil && removeOnceDereferenced { l := s.mu.lease s.mu.lease = nil @@ -251,9 +260,12 @@ func (t *descriptorState) release(ctx context.Context, s *descriptorVersionState return nil } maybeRemoveLease := func() *storedLease { + if shouldRemove := decRefCount(s); !shouldRemove { + return nil + } t.mu.Lock() defer t.mu.Unlock() - if l := decRefcount(s); l != nil { + if l := maybeMarkRemoveStoredLease(s); l != nil { t.mu.active.remove(s) return l } diff --git a/pkg/sql/catalog/lease/descriptor_version_state.go b/pkg/sql/catalog/lease/descriptor_version_state.go index eed9b57810d2..062e682db347 100644 --- a/pkg/sql/catalog/lease/descriptor_version_state.go +++ b/pkg/sql/catalog/lease/descriptor_version_state.go @@ -77,9 +77,7 @@ func (s *descriptorVersionState) Underlying() catalog.Descriptor { } func (s *descriptorVersionState) Expiration() hlc.Timestamp { - s.mu.Lock() - defer s.mu.Unlock() - return s.mu.expiration + return s.getExpiration() } func (s *descriptorVersionState) SafeMessage() string { @@ -113,15 +111,15 @@ func (s *descriptorVersionState) hasExpiredLocked(timestamp hlc.Timestamp) bool return s.mu.expiration.LessEq(timestamp) } -func (s *descriptorVersionState) incRefCount(ctx context.Context) { +func (s *descriptorVersionState) incRefCount(ctx context.Context, expensiveLogEnabled bool) { s.mu.Lock() defer s.mu.Unlock() - s.incRefCountLocked(ctx) + s.incRefCountLocked(ctx, expensiveLogEnabled) } -func (s *descriptorVersionState) incRefCountLocked(ctx context.Context) { +func (s *descriptorVersionState) incRefCountLocked(ctx context.Context, expensiveLogEnabled bool) { s.mu.refcount++ - if log.ExpensiveLogEnabled(ctx, 2) { + if expensiveLogEnabled { log.VEventf(ctx, 2, "descriptorVersionState.incRefCount: %s", s.stringLocked()) } } diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 31134e1c45d4..809ad1ecf8b4 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -666,9 +666,9 @@ func NewLeaseManager( func NameMatchesDescriptor( desc catalog.Descriptor, parentID descpb.ID, parentSchemaID descpb.ID, name string, ) bool { - return desc.GetParentID() == parentID && - desc.GetParentSchemaID() == parentSchemaID && - desc.GetName() == name + return desc.GetName() == name && + desc.GetParentID() == parentID && + desc.GetParentSchemaID() == parentSchemaID } // findNewest returns the newest descriptor version state for the ID. diff --git a/pkg/sql/catalog/lease/name_cache.go b/pkg/sql/catalog/lease/name_cache.go index a65a26960d2f..ef676ff37481 100644 --- a/pkg/sql/catalog/lease/name_cache.go +++ b/pkg/sql/catalog/lease/name_cache.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" ) @@ -29,7 +30,7 @@ func makeNameCache() nameCache { // from the store. The cache maintains the latest version for each name. // All methods are thread-safe. type nameCache struct { - mu syncutil.Mutex + mu syncutil.RWMutex descriptors nstree.Map } @@ -47,14 +48,15 @@ func (c *nameCache) get( name string, timestamp hlc.Timestamp, ) *descriptorVersionState { - c.mu.Lock() + c.mu.RLock() desc, ok := c.descriptors.GetByName( parentID, parentSchemaID, name, ).(*descriptorVersionState) - c.mu.Unlock() + c.mu.RUnlock() if !ok { return nil } + expensiveLogEnabled := log.ExpensiveLogEnabled(ctx, 2) desc.mu.Lock() if desc.mu.lease == nil { desc.mu.Unlock() @@ -80,7 +82,7 @@ func (c *nameCache) get( return nil } - desc.incRefCountLocked(ctx) + desc.incRefCountLocked(ctx, expensiveLogEnabled) return desc } diff --git a/pkg/sql/catalog/nstree/set.go b/pkg/sql/catalog/nstree/set.go index 06592e8e87b2..d0bab68ada65 100644 --- a/pkg/sql/catalog/nstree/set.go +++ b/pkg/sql/catalog/nstree/set.go @@ -47,6 +47,11 @@ func (s *Set) Clear() { *s = Set{} } +// Empty returns true if the set has no entries. +func (s *Set) Empty() bool { + return !s.initialized() || s.t.Len() == 0 +} + func (s *Set) maybeInitialize() { if s.initialized() { return