diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 50950ae8b5c2..ab4c96514a79 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -274,7 +274,7 @@ rangeLoop: func WriteDescriptors( ctx context.Context, txn *kv.Txn, - databases []*sqlbase.ImmutableDatabaseDescriptor, + databases []sqlbase.DatabaseDescriptor, tables []sqlbase.TableDescriptor, types []sqlbase.TypeDescriptor, descCoverage tree.DescriptorCoverage, @@ -285,13 +285,19 @@ func WriteDescriptors( defer tracing.FinishSpan(span) err := func() error { b := txn.NewBatch() - wroteDBs := make(map[descpb.ID]*sqlbase.ImmutableDatabaseDescriptor) - for _, desc := range databases { + wroteDBs := make(map[descpb.ID]sqlbase.DatabaseDescriptor) + for i := range databases { + desc := databases[i] // If the restore is not a full cluster restore we cannot know that // the users on the restoring cluster match the ones that were on the // cluster that was backed up. So we wipe the privileges on the database. if descCoverage != tree.AllDescriptors { - desc.Privileges = descpb.NewDefaultPrivilegeDescriptor(security.AdminRole) + if mut, ok := desc.(*sqlbase.MutableDatabaseDescriptor); ok { + mut.Privileges = descpb.NewDefaultPrivilegeDescriptor(security.AdminRole) + } else { + log.Fatalf(ctx, "wrong type for table %d, %T, expected MutableTableDescriptor", + desc.GetID(), desc) + } } wroteDBs[desc.GetID()] = desc if err := catalogkv.WriteNewDescToBatch(ctx, false /* kvTrace */, settings, b, keys.SystemSQLCodec, desc.GetID(), desc); err != nil { @@ -653,7 +659,7 @@ func loadBackupSQLDescs( type restoreResumer struct { job *jobs.Job settings *cluster.Settings - databases []*sqlbase.ImmutableDatabaseDescriptor + databases []sqlbase.DatabaseDescriptor tables []sqlbase.TableDescriptor // writtenTypes is the set of types that are restored from the backup into // the database. Note that this is not always the set of types within the @@ -736,7 +742,7 @@ func remapRelevantStatistics( func isDatabaseEmpty( ctx context.Context, db *kv.DB, - dbDesc *sqlbase.ImmutableDatabaseDescriptor, + dbDesc sqlbase.DatabaseDescriptor, ignoredTables map[descpb.ID]struct{}, ) (bool, error) { var allDescs []sqlbase.Descriptor @@ -766,7 +772,7 @@ func isDatabaseEmpty( func createImportingDescriptors( ctx context.Context, p sql.PlanHookState, sqlDescs []sqlbase.Descriptor, r *restoreResumer, ) ( - databases []*sqlbase.ImmutableDatabaseDescriptor, + databases []sqlbase.DatabaseDescriptor, tables []sqlbase.TableDescriptor, oldTableIDs []descpb.ID, writtenTypes []sqlbase.TypeDescriptor, diff --git a/pkg/ccl/backupccl/targets_test.go b/pkg/ccl/backupccl/targets_test.go index fd3f6389a0d2..30199f44082b 100644 --- a/pkg/ccl/backupccl/targets_test.go +++ b/pkg/ccl/backupccl/targets_test.go @@ -48,7 +48,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) { return desc } mkDB := func(id descpb.ID, name string) *sqlbase.ImmutableDatabaseDescriptor { - return sqlbase.NewInitialDatabaseDescriptor(id, name, security.AdminRole) + return &sqlbase.NewInitialDatabaseDescriptor(id, name, security.AdminRole).ImmutableDatabaseDescriptor } mkTyp := func(desc typDesc) *sqlbase.ImmutableTypeDescriptor { return sqlbase.NewImmutableTypeDescriptor(desc) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index d90f91ae8799..2c8827e23846 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -323,7 +323,7 @@ func importPlanHook( } else { // No target table means we're importing whatever we find into the session // database, so it must exist. - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, p.SessionData().Database, true /*required*/) + dbDesc, err := p.ResolveImmutableDatabaseDescriptor(ctx, p.SessionData().Database, true /*required*/) if err != nil { return pgerror.Wrap(err, pgcode.UndefinedObject, "could not resolve current database") diff --git a/pkg/kv/kvserver/reports/constraint_stats_report_test.go b/pkg/kv/kvserver/reports/constraint_stats_report_test.go index c6e684ce58b6..15e38f2da90c 100644 --- a/pkg/kv/kvserver/reports/constraint_stats_report_test.go +++ b/pkg/kv/kvserver/reports/constraint_stats_report_test.go @@ -760,7 +760,7 @@ func compileTestCase(tc baseReportTestCase) (compiledTestCase, error) { } } sysCfgBuilder.addDBDesc(dbID, - sqlbase.NewInitialDatabaseDescriptor(descpb.ID(dbID), db.name, security.AdminRole)) + &sqlbase.NewInitialDatabaseDescriptor(descpb.ID(dbID), db.name, security.AdminRole).ImmutableDatabaseDescriptor) for _, table := range db.tables { tableID := objectCounter diff --git a/pkg/sql/catalog/accessors/physical_schema_accessors.go b/pkg/sql/catalog/accessors/physical_schema_accessors.go index 40550f043e71..28da0275ea78 100644 --- a/pkg/sql/catalog/accessors/physical_schema_accessors.go +++ b/pkg/sql/catalog/accessors/physical_schema_accessors.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/errors" @@ -55,6 +54,7 @@ func NewCachedAccessor( // CachedPhysicalAccessor adds a cache on top of any Accessor. type CachedPhysicalAccessor struct { catalog.Accessor + // TODO (lucy): Rename. tc *descs.Collection // Used to avoid allocations. tableName tree.TableName @@ -71,38 +71,18 @@ func (a *CachedPhysicalAccessor) GetDatabaseDesc( name string, flags tree.DatabaseLookupFlags, ) (desc sqlbase.DatabaseDescriptor, err error) { - isSystemDB := name == sqlbase.SystemDatabaseName - if !(flags.AvoidCached || isSystemDB || lease.TestingTableLeasesAreDisabled()) { - refuseFurtherLookup, dbID, err := a.tc.GetUncommittedDatabaseID(name, flags.Required) - if refuseFurtherLookup || err != nil { + if flags.RequireMutable { + db, err := a.tc.GetMutableDatabaseDescriptor(ctx, txn, name, flags) + if db == nil { return nil, err } - - if dbID != descpb.InvalidID { - // Some database ID was found in the list of uncommitted DB changes. - // Use that to get the descriptor. - desc, err := a.tc.DatabaseCache().GetDatabaseDescByID(ctx, txn, dbID) - if desc == nil && flags.Required { - return nil, sqlbase.NewUndefinedDatabaseError(name) - } else if desc == nil { - // NB: We must return the actual value nil here as a typed nil will not - // be easily detectable by the caller. - return nil, nil - } - return desc, err - } - - // The database was not known in the uncommitted list. Have the db - // cache look it up by name for us. - desc, err := a.tc.DatabaseCache().GetDatabaseDesc(ctx, a.tc.LeaseManager().DB().Txn, name, flags.Required) - if desc == nil || err != nil { - return nil, err - } - return desc, nil + return db, err } - - // We avoided the cache. Go lower. - return a.Accessor.GetDatabaseDesc(ctx, txn, codec, name, flags) + typ, err := a.tc.GetDatabaseVersion(ctx, txn, name, flags) + if typ == nil { + return nil, err + } + return typ, err } // GetSchema implements the Accessor interface. diff --git a/pkg/sql/catalog/catalogkv/physical_accessor.go b/pkg/sql/catalog/catalogkv/physical_accessor.go index 6c1ead3ee5ee..6711e0eb3111 100644 --- a/pkg/sql/catalog/catalogkv/physical_accessor.go +++ b/pkg/sql/catalog/catalogkv/physical_accessor.go @@ -46,10 +46,11 @@ func (a UncachedPhysicalAccessor) GetDatabaseDesc( flags tree.DatabaseLookupFlags, ) (desc sqlbase.DatabaseDescriptor, err error) { if name == sqlbase.SystemDatabaseName { - // We can't return a direct reference to SystemDB, because the - // caller expects a private object that can be modified in-place. - sysDB := sqlbase.MakeSystemDatabaseDesc() - return sysDB, nil + if flags.RequireMutable { + return sqlbase.NewMutableExistingDatabaseDescriptor( + *sqlbase.MakeSystemDatabaseDesc().DatabaseDesc()), nil + } + return sqlbase.SystemDB, nil } found, descID, err := LookupDatabaseID(ctx, txn, codec, name) @@ -59,19 +60,22 @@ func (a UncachedPhysicalAccessor) GetDatabaseDesc( if flags.Required { return nil, sqlbase.NewUndefinedDatabaseError(name) } + log.Errorf(ctx, "db %q not found", name) return nil, nil } // NB: Take care to actually return nil here rather than a typed nil which // will not compare to nil when wrapped in the returned interface. - desc, err = GetDatabaseDescByID(ctx, txn, codec, descID) + untypedDesc, err := GetAnyDescriptorByID(ctx, txn, codec, descID, Mutability(flags.RequireMutable)) if err != nil { return nil, err } - if desc == nil { + db, ok := untypedDesc.(sqlbase.DatabaseDescriptor) + if !ok { + log.Errorf(ctx, "db not a DatabaseDescriptor", untypedDesc) return nil, nil } - return desc, err + return db, nil } // GetSchema implements the Accessor interface. diff --git a/pkg/sql/catalog/database/database.go b/pkg/sql/catalog/database/database.go deleted file mode 100644 index d3e6263800af..000000000000 --- a/pkg/sql/catalog/database/database.go +++ /dev/null @@ -1,246 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -// Package database primarily provides the incoherent database cache and -// related interfaces. -// -// TODO(ajwerner): Lease database descriptors like all other schema objects and -// eliminate this package. -package database - -import ( - "context" - "sync" - - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" - "github.com/cockroachdb/cockroach/pkg/util/log" -) - -// Cache holds a cache from database name to database ID. It is -// populated as database IDs are requested and a new cache is created whenever -// the system config changes. As such, no attempt is made to limit its size -// which is naturally limited by the number of database descriptors in the -// system the periodic reset whenever the system config is gossiped. -type Cache struct { - // databases is really a map of string -> descpb.ID - databases sync.Map - - // codec is used to encode and decode sql keys. - codec keys.SQLCodec - - // systemConfig holds a copy of the latest system config since the last - // call to resetForBatch. - systemConfig *config.SystemConfig -} - -// NewCache constructs a new Cache. -func NewCache(codec keys.SQLCodec, cfg *config.SystemConfig) *Cache { - return &Cache{ - codec: codec, - systemConfig: cfg, - } -} - -func (dc *Cache) getID(name string) descpb.ID { - val, ok := dc.databases.Load(name) - if !ok { - return descpb.InvalidID - } - return val.(descpb.ID) -} - -func (dc *Cache) setID(name string, id descpb.ID) { - dc.databases.Store(name, id) -} - -// getCachedDatabaseDesc looks up the database descriptor from the descriptor cache, -// given its name. Returns nil and no error if the name is not present in the -// cache. -func (dc *Cache) getCachedDatabaseDesc(name string) (*sqlbase.ImmutableDatabaseDescriptor, error) { - dbID, err := dc.GetCachedDatabaseID(name) - if dbID == descpb.InvalidID || err != nil { - return nil, err - } - - return dc.getCachedDatabaseDescByID(dbID) -} - -// getCachedDatabaseDescByID looks up the database descriptor from the descriptor cache, -// given its ID. -func (dc *Cache) getCachedDatabaseDescByID( - id descpb.ID, -) (*sqlbase.ImmutableDatabaseDescriptor, error) { - if id == keys.SystemDatabaseID { - // We can't return a direct reference to SystemDB, because the - // caller expects a private object that can be modified in-place. - sysDB := sqlbase.MakeSystemDatabaseDesc() - return sysDB, nil - } - - descKey := sqlbase.MakeDescMetadataKey(dc.codec, id) - descVal := dc.systemConfig.GetValue(descKey) - if descVal == nil { - return nil, nil - } - - desc := &descpb.Descriptor{} - if err := descVal.GetProto(desc); err != nil { - return nil, err - } - - dbDesc := desc.GetDatabase() - if dbDesc == nil { - return nil, pgerror.Newf(pgcode.WrongObjectType, "[%d] is not a database", id) - } - database := sqlbase.NewImmutableDatabaseDescriptor(*dbDesc) - if err := database.Validate(); err != nil { - return nil, err - } - // TODO(ajwerner): Set ModificationTime. - return database, nil -} - -// GetDatabaseDesc returns the database descriptor given its name -// if it exists in the cache, otherwise falls back to KV operations. -func (dc *Cache) GetDatabaseDesc( - ctx context.Context, - txnRunner func(context.Context, func(context.Context, *kv.Txn) error) error, - name string, - required bool, -) (*sqlbase.ImmutableDatabaseDescriptor, error) { - // Lookup the database in the cache first, falling back to the KV store if it - // isn't present. The cache might cause the usage of a recently renamed - // database, but that's a race that could occur anyways. - // The cache lookup may fail. - desc, err := dc.getCachedDatabaseDesc(name) - if err != nil { - return nil, err - } - if desc == nil { - if err := txnRunner(ctx, func(ctx context.Context, txn *kv.Txn) error { - // Run the descriptor read as high-priority, thereby pushing any intents out - // of its way. We don't want schema changes to prevent database lookup; - // we'd rather force them to refresh. Also this prevents deadlocks in cases - // where the name resolution is triggered by the transaction doing the - // schema change itself. - if err := txn.SetUserPriority(roachpb.MaxUserPriority); err != nil { - return err - } - a := catalogkv.UncachedPhysicalAccessor{} - descI, err := a.GetDatabaseDesc(ctx, txn, dc.codec, name, - tree.DatabaseLookupFlags{Required: required}) - if err != nil { - return err - } - if descI != nil { - desc = descI.(*sqlbase.ImmutableDatabaseDescriptor) - } - return nil - }); err != nil { - return nil, err - } - } - if desc != nil { - dc.setID(name, desc.GetID()) - } - return desc, err -} - -// GetDatabaseDescByID returns the database descriptor given its ID -// if it exists in the cache, otherwise falls back to KV operations. -func (dc *Cache) GetDatabaseDescByID( - ctx context.Context, txn *kv.Txn, id descpb.ID, -) (*sqlbase.ImmutableDatabaseDescriptor, error) { - desc, err := dc.getCachedDatabaseDescByID(id) - if desc == nil || err != nil { - if err != nil { - log.VEventf(ctx, 3, "error getting database descriptor from cache: %s", err) - } - desc, err = catalogkv.MustGetDatabaseDescByID(ctx, txn, dc.codec, id) - } - return desc, err -} - -// GetDatabaseID returns the ID of a database given its name. It -// uses the descriptor cache if possible, otherwise falls back to KV -// operations. -func (dc *Cache) GetDatabaseID( - ctx context.Context, - txnRunner func(context.Context, func(context.Context, *kv.Txn) error) error, - name string, - required bool, -) (descpb.ID, error) { - dbID, err := dc.GetCachedDatabaseID(name) - if err != nil { - return dbID, err - } - if dbID == descpb.InvalidID { - if err := txnRunner(ctx, func(ctx context.Context, txn *kv.Txn) error { - // Run the namespace read as high-priority, thereby pushing any intents out - // of its way. We don't want schema changes to prevent database acquisitions; - // we'd rather force them to refresh. Also this prevents deadlocks in cases - // where the name resolution is triggered by the transaction doing the - // schema change itself. - if err := txn.SetUserPriority(roachpb.MaxUserPriority); err != nil { - return err - } - var err error - dbID, err = catalogkv.GetDatabaseID(ctx, txn, dc.codec, name, required) - return err - }); err != nil { - return descpb.InvalidID, err - } - } - dc.setID(name, dbID) - return dbID, nil -} - -// GetCachedDatabaseID returns the ID of a database given its name -// from the cache. This method never goes to the store to resolve -// the name to id mapping. Returns InvalidID if the name to id mapping or -// the database descriptor are not in the cache. -func (dc *Cache) GetCachedDatabaseID(name string) (descpb.ID, error) { - if id := dc.getID(name); id != descpb.InvalidID { - return id, nil - } - - if name == sqlbase.SystemDB.GetName() { - return sqlbase.SystemDB.GetID(), nil - } - - var nameKey sqlbase.DescriptorKey = sqlbase.NewDatabaseKey(name) - nameVal := dc.systemConfig.GetValue(nameKey.Key(dc.codec)) - if nameVal == nil { - // Try the deprecated system.namespace before returning InvalidID. - // TODO(solon): This can be removed in 20.2. - nameKey = sqlbase.NewDeprecatedDatabaseKey(name) - nameVal = dc.systemConfig.GetValue(nameKey.Key(dc.codec)) - if nameVal == nil { - return descpb.InvalidID, nil - } - } - - id, err := nameVal.GetInt() - return descpb.ID(id), err -} - -// Codec returns the cache's codec. -func (dc *Cache) Codec() keys.SQLCodec { - return dc.codec -} diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index ba686ee240e1..9c6b182b07d9 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -14,6 +14,7 @@ package descs import ( "context" + "fmt" "sort" "sync" @@ -22,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/database" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" @@ -36,15 +36,6 @@ import ( "github.com/cockroachdb/errors" ) -// UncommittedDatabase is a database that has been created/dropped -// within the current transaction using the Collection. A rename -// is a drop of the old name and creation of the new name. -type UncommittedDatabase struct { - name string - id descpb.ID - dropped bool -} - // uncommittedDescriptor is a descriptor that has been modified in the current // transaction. type uncommittedDescriptor struct { @@ -124,23 +115,16 @@ func (ld *leasedDescriptors) numDescriptors() int { } // MakeCollection constructs a Collection. -func MakeCollection( - leaseMgr *lease.Manager, - settings *cluster.Settings, - dbCache *database.Cache, - dbCacheSubscriber DatabaseCacheSubscriber, -) Collection { +func MakeCollection(leaseMgr *lease.Manager, settings *cluster.Settings) Collection { return Collection{ - leaseMgr: leaseMgr, - settings: settings, - databaseCache: dbCache, - dbCacheSubscriber: dbCacheSubscriber, + leaseMgr: leaseMgr, + settings: settings, } } // NewCollection constructs a new *Collection. func NewCollection(leaseMgr *lease.Manager, settings *cluster.Settings) *Collection { - tc := MakeCollection(leaseMgr, settings, nil, nil) + tc := MakeCollection(leaseMgr, settings) return &tc } @@ -165,12 +149,6 @@ type Collection struct { // Collection and invisible to other transactions. uncommittedDescriptors []uncommittedDescriptor - // databaseCache is used as a cache for database names. - // This field is nil when the field is initialized for an internalPlanner. - // TODO(andrei): get rid of it and replace it with a leasing system for - // database descriptors. - databaseCache *database.Cache - // schemaCache maps {databaseID, schemaName} -> (schemaID, if exists, otherwise nil). // TODO(sqlexec): replace with leasing system with custom schemas. // This is currently never cleared, because there should only be unique schemas @@ -178,14 +156,6 @@ type Collection struct { // made, and you cannot read from other schema caches. schemaCache sync.Map - // DatabaseCacheSubscriber is used to block until the node's database cache has been - // updated when ReleaseAll is called. - dbCacheSubscriber DatabaseCacheSubscriber - - // Same as uncommittedDescriptors applying to databases modified within - // an uncommitted transaction. - uncommittedDatabases []UncommittedDatabase - // allDescriptors is a slice of all available descriptors. The descriptors // are cached to avoid repeated lookups by users like virtual tables. The // cache is purged whenever events would cause a scan of all descriptors to @@ -212,6 +182,43 @@ type Collection struct { settings *cluster.Settings } +func (tc *Collection) GetMutableDatabaseDescriptor( + ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, +) (*sqlbase.MutableDatabaseDescriptor, error) { + if log.V(2) { + log.Infof(ctx, "reading mutable descriptor on '%s'", name) + } + + // First, try the uncommitted descriptors. + if refuseFurtherLookup, desc, err := tc.getUncommittedDescriptor( + keys.RootNamespaceID, keys.RootNamespaceID, name, flags.Required, + ); refuseFurtherLookup || err != nil { + return nil, err + } else if mut := desc.mutable; mut != nil { + db, ok := mut.(*sqlbase.MutableDatabaseDescriptor) + if !ok { + return nil, nil + } + log.VEventf(ctx, 2, "found uncommitted descriptor %d", db.GetID()) + return db, nil + } + + phyAccessor := catalogkv.UncachedPhysicalAccessor{} + db, err := phyAccessor.GetDatabaseDesc(ctx, txn, tc.codec(), name, flags) + if err != nil || db == nil { + return nil, err + } + mutDesc, ok := db.(*sqlbase.MutableDatabaseDescriptor) + if !ok { + // TODO (lucy): Here and elsewhere in the Collection, we return a nil + // descriptor with a nil error if the type cast doesn't succeed, regardless + // of whether flags.Required is true. This seems like a potential source + // of bugs. + return nil, nil + } + return mutDesc, nil +} + // GetMutableTableDescriptor returns a mutable table descriptor. // // If flags.required is false, GetMutableTableDescriptor() will gracefully @@ -242,40 +249,30 @@ func (tc *Collection) getMutableObjectDescriptor( log.Infof(ctx, "reading mutable descriptor on '%s'", name) } - refuseFurtherLookup, dbID, err := tc.GetUncommittedDatabaseID(name.Catalog(), flags.Required) - if refuseFurtherLookup || err != nil { + // Resolve the database. + db, err := tc.GetDatabaseVersion(ctx, txn, name.Catalog(), + tree.DatabaseLookupFlags{CommonLookupFlags: flags.CommonLookupFlags}) + if err != nil || db == nil { return nil, err } + dbID := db.GetID() - if dbID == descpb.InvalidID && tc.DatabaseCache() != nil { - // Resolve the database from the database cache when the transaction - // hasn't modified the database. - dbID, err = tc.DatabaseCache().GetDatabaseID(ctx, tc.leaseMgr.DB().Txn, name.Catalog(), flags.Required) - if err != nil || dbID == descpb.InvalidID { - // dbID can still be invalid if required is false and the database is not found. - return nil, err - } + // Resolve the schema to the ID of the schema. + foundSchema, resolvedSchema, err := tc.ResolveSchema(ctx, txn, dbID, name.Schema()) + if err != nil || !foundSchema { + return nil, err } - // The following checks only work if the dbID is not invalid. - if dbID != descpb.InvalidID { - // Resolve the schema to the ID of the schema. - foundSchema, resolvedSchema, err := tc.ResolveSchema(ctx, txn, dbID, name.Schema()) - if err != nil || !foundSchema { - return nil, err - } - - if refuseFurtherLookup, desc, err := tc.getUncommittedDescriptor( - dbID, - resolvedSchema.ID, - name.Object(), - flags.Required, - ); refuseFurtherLookup || err != nil { - return nil, err - } else if mut := desc.mutable; mut != nil { - log.VEventf(ctx, 2, "found uncommitted descriptor %d", mut.GetID()) - return mut, nil - } + if refuseFurtherLookup, desc, err := tc.getUncommittedDescriptor( + dbID, + resolvedSchema.ID, + name.Object(), + flags.Required, + ); refuseFurtherLookup || err != nil { + return nil, err + } else if mut := desc.mutable; mut != nil { + log.VEventf(ctx, 2, "found uncommitted descriptor %d", mut.GetID()) + return mut, nil } phyAccessor := catalogkv.UncachedPhysicalAccessor{} @@ -331,6 +328,97 @@ func (tc *Collection) ResolveSchema( return exists, resolved, err } +func (tc *Collection) GetDatabaseVersion( + ctx context.Context, txn *kv.Txn, name string, flags tree.DatabaseLookupFlags, +) (*sqlbase.ImmutableDatabaseDescriptor, error) { + + readFromStore := func() (*sqlbase.ImmutableDatabaseDescriptor, error) { + phyAccessor := catalogkv.UncachedPhysicalAccessor{} + desc, err := phyAccessor.GetDatabaseDesc(ctx, txn, tc.codec(), name, flags) + if err != nil || desc == nil { + return nil, err + } + return desc.(*sqlbase.ImmutableDatabaseDescriptor), nil + } + + if refuseFurtherLookup, desc, err := tc.getUncommittedDescriptor( + keys.RootNamespaceID, + keys.RootNamespaceID, + name, + flags.Required, + ); refuseFurtherLookup || err != nil { + return nil, err + } else if immut := desc.immutable; immut != nil { + log.VEventf(ctx, 2, "found uncommitted descriptor %d", immut.GetID()) + db, ok := immut.(*sqlbase.ImmutableDatabaseDescriptor) + if !ok { + if flags.Required { + return nil, sqlbase.NewUndefinedDatabaseError(name) + } + return nil, nil + } + return db, nil + } + + avoidCache := flags.AvoidCached || lease.TestingTableLeasesAreDisabled() || + name == sqlbase.SystemDatabaseName + if avoidCache { + return readFromStore() + } + + // First, look to see if we already have the descriptor. + // This ensures that, once a SQL transaction resolved name N to id X, it will + // continue to use N to refer to X even if N is renamed during the + // transaction. + if desc := tc.leasedDescriptors.getByName(keys.RootNamespaceID, keys.RootNamespaceID, name); desc != nil { + log.VEventf(ctx, 2, "found descriptor in collection for '%s'", name) + db, ok := desc.(*sqlbase.ImmutableDatabaseDescriptor) + if !ok { + if flags.Required { + return nil, sqlbase.NewUndefinedDatabaseError(name) + } + return nil, nil + } + return db, nil + } + + readTimestamp := txn.ReadTimestamp() + desc, expiration, err := tc.leaseMgr.AcquireByName(ctx, readTimestamp, keys.RootNamespaceID, keys.RootNamespaceID, name) + if err != nil { + // Read the descriptor from the store in the face of some specific errors + // because of a known limitation of AcquireByName. See the known + // limitations of AcquireByName for details. + if catalog.HasInactiveDescriptorError(err) || + errors.Is(err, sqlbase.ErrDescriptorNotFound) { + return readFromStore() + } + // Lease acquisition failed with some other error. This we don't + // know how to deal with, so propagate the error. + return nil, err + } + db, ok := desc.(*sqlbase.ImmutableDatabaseDescriptor) + if !ok { + if flags.Required { + return nil, sqlbase.NewUndefinedDatabaseError(name) + } + return nil, nil + } + + if expiration.LessEq(readTimestamp) { + log.Fatalf(ctx, "bad descriptor for T=%s, expiration=%s", readTimestamp, expiration) + } + + tc.leasedDescriptors.add(desc) + log.VEventf(ctx, 2, "added descriptor '%s' to collection", name) + + // If the descriptor we just acquired expires before the txn's deadline, + // reduce the deadline. We use ReadTimestamp() that doesn't return the commit + // timestamp, so we need to set a deadline on the transaction to prevent it + // from committing beyond the version's expiration time. + txn.UpdateDeadlineMaybe(ctx, expiration) + return db, nil +} + // 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(). @@ -384,25 +472,13 @@ func (tc *Collection) getObjectVersion( ) } - refuseFurtherLookup, dbID, err := tc.GetUncommittedDatabaseID(name.Catalog(), flags.Required) - if refuseFurtherLookup || err != nil { + // Resolve the database. + db, err := tc.GetDatabaseVersion(ctx, txn, name.Catalog(), + tree.DatabaseLookupFlags{CommonLookupFlags: flags.CommonLookupFlags}) + if err != nil || db == nil { return nil, err } - - if dbID == descpb.InvalidID && tc.DatabaseCache() != nil { - // Resolve the database from the database cache when the transaction - // hasn't modified the database. - dbID, err = tc.DatabaseCache().GetDatabaseID(ctx, tc.leaseMgr.DB().Txn, name.Catalog(), flags.Required) - if err != nil || dbID == descpb.InvalidID { - // dbID can still be invalid if required is false and the database is not found. - return nil, err - } - } - - // If at this point we have an InvalidID, we should immediately try read from store. - if dbID == descpb.InvalidID { - return readObjectFromStore() - } + dbID := db.GetID() // Resolve the schema to the ID of the schema. foundSchema, resolvedSchema, err := tc.ResolveSchema(ctx, txn, dbID, name.Schema()) @@ -484,11 +560,28 @@ func (tc *Collection) getObjectVersion( return desc, nil } +func (tc *Collection) GetDatabaseVersionByID( + ctx context.Context, txn *kv.Txn, dbID descpb.ID, flags tree.DatabaseLookupFlags, +) (*sqlbase.ImmutableDatabaseDescriptor, error) { + desc, err := tc.getDescriptorVersionByID(ctx, txn, dbID, flags.CommonLookupFlags, true /* setTxnDeadline */) + if err != nil { + if errors.Is(err, sqlbase.ErrDescriptorNotFound) { + return nil, sqlbase.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) + } + return nil, err + } + db, ok := desc.(*sqlbase.ImmutableDatabaseDescriptor) + if !ok { + return nil, sqlbase.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) + } + return db, 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 descpb.ID, flags tree.ObjectLookupFlags, ) (*sqlbase.ImmutableTableDescriptor, error) { - desc, err := tc.getDescriptorVersionByID(ctx, txn, tableID, flags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, tableID, flags.CommonLookupFlags, true /* setTxnDeadline */) if err != nil { if errors.Is(err, sqlbase.ErrDescriptorNotFound) { return nil, sqlbase.NewUndefinedRelationError( @@ -509,7 +602,7 @@ func (tc *Collection) GetTableVersionByID( } func (tc *Collection) getDescriptorVersionByID( - ctx context.Context, txn *kv.Txn, id descpb.ID, flags tree.ObjectLookupFlags, setTxnDeadline bool, + ctx context.Context, txn *kv.Txn, id descpb.ID, flags tree.CommonLookupFlags, setTxnDeadline bool, ) (catalog.Descriptor, error) { if flags.AvoidCached || lease.TestingTableLeasesAreDisabled() { desc, err := catalogkv.GetDescriptorByID(ctx, txn, tc.codec(), id, catalogkv.Immutable, @@ -617,8 +710,8 @@ func (tc *Collection) hydrateTypesInTableDesc( if err != nil { return tree.TypeName{}, nil, err } - // TODO (lucy): This database access should go through the collection. - dbDesc, err := catalogkv.MustGetDatabaseDescByID(ctx, txn, tc.codec(), desc.ParentID) + dbDesc, err := tc.GetDatabaseVersionByID(ctx, txn, desc.ParentID, + tree.DatabaseLookupFlags{CommonLookupFlags: tree.CommonLookupFlags{Required: true}}) if err != nil { return tree.TypeName{}, nil, err } @@ -645,8 +738,8 @@ func (tc *Collection) hydrateTypesInTableDesc( if err != nil { return tree.TypeName{}, nil, err } - // TODO (lucy): This database access should go through the collection. - dbDesc, err := catalogkv.MustGetDatabaseDescByID(ctx, txn, tc.codec(), desc.ParentID) + dbDesc, err := tc.GetDatabaseVersionByID(ctx, txn, desc.ParentID, + tree.DatabaseLookupFlags{CommonLookupFlags: tree.CommonLookupFlags{Required: true}}) if err != nil { return tree.TypeName{}, nil, err } @@ -696,43 +789,9 @@ func (tc *Collection) ReleaseLeases(ctx context.Context) { func (tc *Collection) ReleaseAll(ctx context.Context) { tc.ReleaseLeases(ctx) tc.uncommittedDescriptors = nil - tc.uncommittedDatabases = nil tc.releaseAllDescriptors() } -// WaitForCacheToDropDatabases waits until the database cache has been updated -// to properly reflect all dropped databases, so that future commands on the -// same gateway node observe the dropped databases. -func (tc *Collection) WaitForCacheToDropDatabases(ctx context.Context) { - for _, uc := range tc.uncommittedDatabases { - if !uc.dropped { - continue - } - // Wait until the database cache has been updated to properly - // reflect a dropped database, so that future commands on the - // same gateway node observe the dropped database. - tc.dbCacheSubscriber.WaitForCacheState( - func(dc *database.Cache) bool { - // Resolve the database name from the database cache. - dbID, err := dc.GetCachedDatabaseID(uc.name) - if err != nil || dbID == descpb.InvalidID { - // dbID can still be 0 if required is false and - // the database is not found. Swallowing error here - // because it was felt there was no value in returning - // it to a higher layer only to be swallow there. This - // entire codepath is only called from one place so - // it's better to swallow it here. - return true - } - - // If the database name still exists but it now references another - // db with a more recent id, we're good - it means that the database - // name has been reused. - return dbID > uc.id - }) - } -} - // HasUncommittedTables returns true if the Collection contains uncommitted // tables. func (tc *Collection) HasUncommittedTables() bool { @@ -862,7 +921,7 @@ func (tc *Collection) GetTypeVersion( func (tc *Collection) GetTypeVersionByID( ctx context.Context, txn *kv.Txn, typeID descpb.ID, flags tree.ObjectLookupFlags, ) (*sqlbase.ImmutableTypeDescriptor, error) { - desc, err := tc.getDescriptorVersionByID(ctx, txn, typeID, flags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, typeID, flags.CommonLookupFlags, true /* setTxnDeadline */) if err != nil { if errors.Is(err, sqlbase.ErrDescriptorNotFound) { return nil, pgerror.Newf( @@ -878,47 +937,6 @@ func (tc *Collection) GetTypeVersionByID( return typ, nil } -// DBAction is an operation to an uncommitted database. -type DBAction bool - -const ( - // DBCreated notes that the database has been created. - DBCreated DBAction = false - // DBDropped notes that the database has been dropped. - DBDropped DBAction = true -) - -// AddUncommittedDatabase stages the database action for the relevant database. -func (tc *Collection) AddUncommittedDatabase(name string, id descpb.ID, action DBAction) { - db := UncommittedDatabase{name: name, id: id, dropped: action == DBDropped} - tc.uncommittedDatabases = append(tc.uncommittedDatabases, db) - tc.releaseAllDescriptors() -} - -// GetUncommittedDatabaseID returns a database ID for the requested tablename -// if the requested tablename is for a database modified within the transaction -// affiliated with the LeaseCollection. -func (tc *Collection) GetUncommittedDatabaseID( - requestedDbName string, required bool, -) (c bool, res descpb.ID, err error) { - // Walk latest to earliest so that a DROP DATABASE followed by a - // CREATE DATABASE with the same name will result in the CREATE DATABASE - // being seen. - for i := len(tc.uncommittedDatabases) - 1; i >= 0; i-- { - db := tc.uncommittedDatabases[i] - if requestedDbName == db.name { - if db.dropped { - if required { - return true, descpb.InvalidID, sqlbase.NewUndefinedDatabaseError(requestedDbName) - } - return true, descpb.InvalidID, nil - } - return false, db.id, nil - } - } - return false, descpb.InvalidID, nil -} - // getUncommittedDescriptor returns a descriptor for the requested name // if the requested name is for a descriptor modified within the transaction // affiliated with the LeaseCollection. @@ -1121,7 +1139,6 @@ func (tc *Collection) CopyModifiedObjects(to *Collection) { return } to.uncommittedDescriptors = tc.uncommittedDescriptors - to.uncommittedDatabases = tc.uncommittedDatabases // Do not copy the leased descriptors because we do not want // the leased descriptors to be released by the "to" Collection. // The "to" Collection can re-lease the same descriptors. @@ -1142,16 +1159,6 @@ func (tc *Collection) LeaseManager() *lease.Manager { return tc.leaseMgr } -// DatabaseCache returns the database.Cache. -func (tc *Collection) DatabaseCache() *database.Cache { - return tc.databaseCache -} - -// ResetDatabaseCache resets the table collection's database.Cache. -func (tc *Collection) ResetDatabaseCache(dbCache *database.Cache) { - tc.databaseCache = dbCache -} - // MigrationSchemaChangeRequiredContext flags a schema change as necessary to // run even in a mixed-version 19.2/20.1 state where schema changes are normally // banned, because the schema change is being run in a startup migration. It's @@ -1176,14 +1183,6 @@ type migrationSchemaChangeRequiredHint struct{} // change was disallowed from running in a mixed-version var ErrSchemaChangeDisallowedInMixedState = errors.New("schema change cannot be initiated in this version until the version upgrade is finalized") -// DatabaseCacheSubscriber allows the connExecutor to wait for a callback. -type DatabaseCacheSubscriber interface { - // WaitForCacheState takes a callback depending on the cache state and blocks - // until the callback declares success. The callback is repeatedly called as - // the cache is updated. - WaitForCacheState(cond func(*database.Cache) bool) -} - // DistSQLTypeResolverFactory is an object that constructs TypeResolver objects // that are bound under a transaction. These TypeResolvers access descriptors // through the descs.Collection and eventually the lease.Manager. It cannot be @@ -1252,7 +1251,7 @@ func (dt *DistSQLTypeResolver) GetTypeDescriptor( ctx, dt.txn, id, - tree.ObjectLookupFlagsWithRequired(), + tree.CommonLookupFlags{Required: true}, false, /* setTxnDeadline */ ) if err != nil { diff --git a/pkg/sql/comment_on_database.go b/pkg/sql/comment_on_database.go index 601a00accc60..ac4c27f99269 100644 --- a/pkg/sql/comment_on_database.go +++ b/pkg/sql/comment_on_database.go @@ -31,7 +31,7 @@ type commentOnDatabaseNode struct { func (p *planner) CommentOnDatabase( ctx context.Context, n *tree.CommentOnDatabase, ) (planNode, error) { - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, string(n.Name), true) + dbDesc, err := p.ResolveImmutableDatabaseDescriptor(ctx, string(n.Name), true) if err != nil { return nil, err } diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index ae004450743f..f7b471cfb54b 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -20,7 +20,6 @@ import ( "time" "unicode/utf8" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -28,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/database" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -256,10 +254,6 @@ type Server struct { // InternalMetrics is used to account internal queries. InternalMetrics Metrics - - // dbCache is a cache for database descriptors, maintained through Gossip - // updates. - dbCache *databaseCacheHolder } // Metrics collects timeseries data about SQL activity. @@ -282,17 +276,14 @@ type Metrics struct { // NewServer creates a new Server. Start() needs to be called before the Server // is used. func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server { - systemCfg := config.NewSystemConfig(cfg.DefaultZoneConfig) return &Server{ cfg: cfg, Metrics: makeMetrics(false /*internal*/), InternalMetrics: makeMetrics(true /*internal*/), - // dbCache will be updated on Start(). - dbCache: newDatabaseCacheHolder(database.NewCache(cfg.Codec, systemCfg)), - pool: pool, - sqlStats: sqlStats{st: cfg.Settings, apps: make(map[string]*appStats)}, - reportedStats: sqlStats{st: cfg.Settings, apps: make(map[string]*appStats)}, - reCache: tree.NewRegexpCache(512), + pool: pool, + sqlStats: sqlStats{st: cfg.Settings, apps: make(map[string]*appStats)}, + reportedStats: sqlStats{st: cfg.Settings, apps: make(map[string]*appStats)}, + reCache: tree.NewRegexpCache(512), } } @@ -326,21 +317,6 @@ func makeMetrics(internal bool) Metrics { // Start starts the Server's background processing. func (s *Server) Start(ctx context.Context, stopper *stop.Stopper) { - if s.cfg.Codec.ForSystemTenant() { - gossipUpdateC := s.cfg.SystemConfig.RegisterSystemConfigChannel() - stopper.RunWorker(ctx, func(ctx context.Context) { - for { - select { - case <-gossipUpdateC: - sysCfg := s.cfg.SystemConfig.GetSystemConfig() - s.dbCache.updateSystemConfig(sysCfg) - case <-stopper.ShouldStop(): - return - } - } - }) - } - // Start a loop to clear SQL stats at the max reset interval. This is // to ensure that we always have some worker clearing SQL stats to avoid // continually allocating space for the SQL stats. Additionally, spawn @@ -641,8 +617,7 @@ func (s *Server) newConnExecutor( portals: make(map[string]PreparedPortal), } ex.extraTxnState.prepStmtsNamespaceMemAcc = ex.sessionMon.MakeBoundAccount() - ex.extraTxnState.descCollection = descs.MakeCollection(s.cfg.LeaseManager, - s.cfg.Settings, s.dbCache.getDatabaseCache(), s.dbCache) + ex.extraTxnState.descCollection = descs.MakeCollection(s.cfg.LeaseManager, s.cfg.Settings) ex.extraTxnState.txnRewindPos = -1 ex.extraTxnState.schemaChangeJobsCache = make(map[descpb.ID]*jobs.Job) ex.mu.ActiveQueries = make(map[ClusterWideID]*queryMeta) @@ -831,7 +806,7 @@ func (ex *connExecutor) close(ctx context.Context, closeType closeType) { ex.state.finishExternalTxn() } - if err := ex.resetExtraTxnState(ctx, ex.server.dbCache, txnEv); err != nil { + if err := ex.resetExtraTxnState(ctx, txnEv); err != nil { log.Warningf(ctx, "error while cleaning up connExecutor: %s", err) } @@ -1194,9 +1169,7 @@ func (ns *prepStmtNamespace) resetTo( // resetExtraTxnState resets the fields of ex.extraTxnState when a transaction // commits, rolls back or restarts. -func (ex *connExecutor) resetExtraTxnState( - ctx context.Context, dbCacheHolder *databaseCacheHolder, ev txnEvent, -) error { +func (ex *connExecutor) resetExtraTxnState(ctx context.Context, ev txnEvent) error { ex.extraTxnState.jobs = nil for k := range ex.extraTxnState.schemaChangeJobsCache { @@ -1205,8 +1178,6 @@ func (ex *connExecutor) resetExtraTxnState( ex.extraTxnState.descCollection.ReleaseAll(ctx) - ex.extraTxnState.descCollection.ResetDatabaseCache(dbCacheHolder.getDatabaseCache()) - // Close all portals. for name, p := range ex.extraTxnState.prepStmtsNamespace.portals { p.decRef(ctx, &ex.extraTxnState.prepStmtsNamespaceMemAcc, name) @@ -1795,7 +1766,7 @@ func (ex *connExecutor) execCopyIn( } else { txnOpt = copyTxnOpt{ resetExtraTxnState: func(ctx context.Context) error { - return ex.resetExtraTxnState(ctx, ex.server.dbCache, noEvent) + return ex.resetExtraTxnState(ctx, noEvent) }, } } @@ -2230,12 +2201,9 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( handleErr(err) } - // Wait for the cache to reflect the dropped databases if any. - ex.extraTxnState.descCollection.WaitForCacheToDropDatabases(ex.Ctx()) - fallthrough case txnRestart, txnRollback: - if err := ex.resetExtraTxnState(ex.Ctx(), ex.server.dbCache, advInfo.txnEvent); err != nil { + if err := ex.resetExtraTxnState(ex.Ctx(), advInfo.txnEvent); err != nil { return advanceInfo{}, err } default: diff --git a/pkg/sql/create_database.go b/pkg/sql/create_database.go index 9b21fc94c2d0..a0c066f3050a 100644 --- a/pkg/sql/create_database.go +++ b/pkg/sql/create_database.go @@ -15,7 +15,6 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -100,8 +99,6 @@ func (n *createDatabaseNode) startExec(params runParams) error { ); err != nil { return err } - params.extendedEvalCtx.Descs.AddUncommittedDatabase( - desc.GetName(), desc.GetID(), descs.DBCreated) } return nil } diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go index 5fbe5493ed23..00e348b38a80 100644 --- a/pkg/sql/create_schema.go +++ b/pkg/sql/create_schema.go @@ -61,19 +61,18 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema "cannot create schema without being connected to a database") } - // TODO (lucy): We need a MutableDatabaseDescriptor resolution function. - db, err := p.ResolveUncachedDatabaseByName(params.ctx, p.CurrentDatabase(), true /* required */) + db, err := p.ResolveMutableDatabaseDescriptor(params.ctx, p.CurrentDatabase(), true /* required */) if err != nil { return err } // Users cannot create schemas within the system database. - if db.ID == keys.SystemDatabaseID { + if db.GetID() == keys.SystemDatabaseID { return pgerror.New(pgcode.InvalidObjectDefinition, "cannot create schemas in the system database") } // Ensure there aren't any name collisions. - exists, err := p.schemaExists(params.ctx, db.ID, n.Schema) + exists, err := p.schemaExists(params.ctx, db.GetID(), n.Schema) if err != nil { return err } @@ -117,29 +116,28 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema // Create the SchemaDescriptor. desc := sqlbase.NewMutableCreatedSchemaDescriptor(descpb.SchemaDescriptor{ - ParentID: db.ID, + ParentID: db.GetID(), Name: n.Schema, ID: id, Privileges: privs, }) // Update the parent database with this schema information. - mutDB := sqlbase.NewMutableExistingDatabaseDescriptor(*db.DatabaseDesc()) - if mutDB.Schemas == nil { - mutDB.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) + if db.Schemas == nil { + db.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) } - mutDB.Schemas[desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ + db.Schemas[desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ ID: desc.ID, Dropped: false, } - if err := p.writeDatabaseChange(params.ctx, mutDB); err != nil { + if err := p.writeDatabaseChange(params.ctx, db); err != nil { return err } // Finally create the schema on disk. return p.createDescriptorWithID( params.ctx, - sqlbase.NewSchemaKey(db.ID, n.Schema).Key(p.ExecCfg().Codec), + sqlbase.NewSchemaKey(db.GetID(), n.Schema).Key(p.ExecCfg().Codec), id, desc, params.ExecCfg().Settings, diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 49ed071f586c..e96d47016238 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -33,7 +33,7 @@ type createSequenceNode struct { func (p *planner) CreateSequence(ctx context.Context, n *tree.CreateSequence) (planNode, error) { un := n.Name.ToUnresolvedObjectName() - dbDesc, prefix, err := p.ResolveUncachedDatabase(ctx, un) + dbDesc, prefix, err := p.ResolveTargetForObject(ctx, un) if err != nil { return nil, err } diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go index 15790b19159d..45dd818f695b 100644 --- a/pkg/sql/create_type.go +++ b/pkg/sql/create_type.go @@ -54,7 +54,7 @@ func resolveNewTypeName( params runParams, name *tree.UnresolvedObjectName, ) (*tree.TypeName, *sqlbase.ImmutableDatabaseDescriptor, error) { // Resolve the target schema and database. - db, prefix, err := params.p.ResolveUncachedDatabase(params.ctx, name) + db, prefix, err := params.p.ResolveTargetForObject(params.ctx, name) if err != nil { return nil, nil, err } diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 267dfb00755b..5cf493be7807 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -13,9 +13,10 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -30,17 +31,11 @@ import ( // suitable; consider instead schema_accessors.go and resolver.go. // -// renameDatabase implements the DatabaseDescEditor interface. func (p *planner) renameDatabase( - ctx context.Context, oldDesc *sqlbase.ImmutableDatabaseDescriptor, newName string, + ctx context.Context, desc *sqlbase.MutableDatabaseDescriptor, newName string, ) error { - oldName := oldDesc.GetName() - newDesc := sqlbase.NewMutableExistingDatabaseDescriptor(*oldDesc.DatabaseDesc()) - newDesc.Version++ - newDesc.SetName(newName) - if err := newDesc.Validate(); err != nil { - return err - } + oldName := desc.GetName() + desc.SetName(newName) if exists, _, err := catalogkv.LookupDatabaseID(ctx, p.txn, p.ExecCfg().Codec, newName); err == nil && exists { return pgerror.Newf(pgcode.DuplicateDatabase, @@ -49,40 +44,47 @@ func (p *planner) renameDatabase( return err } + // Add the new namespace entry. newKey := catalogkv.MakeDatabaseNameKey(ctx, p.ExecCfg().Settings, newName).Key(p.ExecCfg().Codec) - - descID := newDesc.GetID() - descKey := sqlbase.MakeDescMetadataKey(p.ExecCfg().Codec, descID) - descDesc := newDesc.DescriptorProto() - b := &kv.Batch{} if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "CPut %s -> %d", newKey, descID) - log.VEventf(ctx, 2, "Put %s -> %s", descKey, descDesc) + log.VEventf(ctx, 2, "CPut %s -> %d", newKey, desc.ID) } - b.CPut(newKey, descID, nil) - b.Put(descKey, descDesc) - err := catalogkv.RemoveDatabaseNamespaceEntry( - ctx, p.txn, p.ExecCfg().Codec, oldName, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - ) - if err != nil { + b.CPut(newKey, desc.ID, nil) + if err := p.txn.Run(ctx, b); err != nil { return err } - p.Descriptors().AddUncommittedDatabase(oldName, descID, descs.DBDropped) - p.Descriptors().AddUncommittedDatabase(newName, descID, descs.DBCreated) + // Add the old name to the draining names on the database descriptor. + renameDetails := descpb.NameInfo{ + ParentID: keys.RootNamespaceID, + ParentSchemaID: keys.RootNamespaceID, + Name: oldName, + } + desc.DrainingNames = append(desc.DrainingNames, renameDetails) - return p.txn.Run(ctx, b) + if err := p.createDropDatabaseJob( + // TODO (lucy): !!! update the job description from the AST node + ctx, desc.GetID(), nil /* DroppedTableDetails */, nil /* typesToDrop */, "rename database", + ); err != nil { + return err + } + return p.writeDatabaseChange(ctx, desc) } +// writeDatabaseChange writes a MutableDatabaseDescriptor's changes to the +// store. Unlike with tables, callers are responsible for queuing the +// accompanying job. func (p *planner) writeDatabaseChange( ctx context.Context, desc *sqlbase.MutableDatabaseDescriptor, ) error { desc.MaybeIncrementVersion() - // TODO (rohany, lucy): This usage of descs.DBCreated is awkward, but since - // we are getting rid of this anyway, I'll just leave it for now to be - // cleaned up as part of the database cache removal process. - p.Descriptors().AddUncommittedDatabase(desc.Name, desc.ID, descs.DBCreated) + if err := desc.Validate(); err != nil { + return err + } + if err := p.Descriptors().AddUncommittedDescriptor(desc); err != nil { + return err + } b := p.txn.NewBatch() if err := catalogkv.WriteDescToBatch( ctx, diff --git a/pkg/sql/database_test.go b/pkg/sql/database_test.go index d106231388b1..f71e973dfd4c 100644 --- a/pkg/sql/database_test.go +++ b/pkg/sql/database_test.go @@ -15,12 +15,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/config" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/database" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -41,10 +38,7 @@ func TestDatabaseAccessors(t *testing.T) { if _, err := catalogkv.MustGetDatabaseDescByID(ctx, txn, keys.SystemSQLCodec, keys.SystemDatabaseID); err != nil { return err } - - databaseCache := database.NewCache(keys.SystemSQLCodec, config.NewSystemConfig(zonepb.DefaultZoneConfigRef())) - _, err := databaseCache.GetDatabaseDescByID(ctx, txn, keys.SystemDatabaseID) - return err + return nil }); err != nil { t.Fatal(err) } diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index fd940a661c5c..b269680d48e8 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -51,7 +52,7 @@ var ( // createDatabase implements the DatabaseDescEditor interface. func (p *planner) createDatabase( ctx context.Context, database *tree.CreateDatabase, jobDesc string, -) (*sqlbase.ImmutableDatabaseDescriptor, bool, error) { +) (*sqlbase.MutableDatabaseDescriptor, bool, error) { dbName := string(database.Name) shouldCreatePublicSchema := true @@ -78,10 +79,6 @@ func (p *planner) createDatabase( return nil, false, err } - // TODO(ajwerner): Consider whether this should be returning a - // MutableDatabaseDescriptor and where/how this will interact with the - // descs.Collection (now it happens well above this call, which is probably - // fine). desc := sqlbase.NewInitialDatabaseDescriptor(id, string(database.Name), p.SessionData().User) if err := p.createDescriptorWithID(ctx, dKey.Key(p.ExecCfg().Codec), id, desc, nil, jobDesc); err != nil { return nil, true, err @@ -142,23 +139,39 @@ func (p *planner) createDescriptorWithID( return err } - if mutType, ok := descriptor.(*sqlbase.MutableTypeDescriptor); ok { - if err := mutType.Validate(ctx, p.txn, p.ExecCfg().Codec); err != nil { + mutDesc, ok := descriptor.(catalog.MutableDescriptor) + if !ok { + log.Fatalf(ctx, "unexpected type %T when creating descriptor", descriptor) + } + isTable := false + switch desc := mutDesc.(type) { + case *sqlbase.MutableTypeDescriptor: + if err := desc.Validate(ctx, p.txn, p.ExecCfg().Codec); err != nil { return err } - if err := p.Descriptors().AddUncommittedDescriptor(mutType); err != nil { + if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { return err } - } - - mutDesc, isTable := descriptor.(*sqlbase.MutableTableDescriptor) - if isTable { - if err := mutDesc.ValidateTable(); err != nil { + case *sqlbase.MutableTableDescriptor: + isTable = true + if err := desc.ValidateTable(); err != nil { + return err + } + if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { + return err + } + case *sqlbase.MutableDatabaseDescriptor: + if err := desc.Validate(); err != nil { return err } if err := p.Descriptors().AddUncommittedDescriptor(mutDesc); err != nil { return err } + case *sqlbase.MutableSchemaDescriptor: + // TODO (lucy): Integrate with schema leasing. At that point, we can pull + // the AddUncommittedDescriptor calls out of the individual cases above. + default: + log.Fatalf(ctx, "unexpected type %T when creating descriptor", mutDesc) } if err := p.txn.Run(ctx, b); err != nil { @@ -168,7 +181,7 @@ func (p *planner) createDescriptorWithID( // Queue a schema change job to eventually make the table public. if err := p.createOrUpdateSchemaChangeJob( ctx, - mutDesc, + mutDesc.(*MutableTableDescriptor), jobDesc, descpb.InvalidMutationID); err != nil { return err diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index bfd3bd8bd827..e75ed30c966b 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -13,15 +13,12 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -35,7 +32,7 @@ import ( type dropDatabaseNode struct { n *tree.DropDatabase - dbDesc *sqlbase.ImmutableDatabaseDescriptor + dbDesc *sqlbase.MutableDatabaseDescriptor td []toDelete schemasToDelete []string allTableObjectsToDelete []*sqlbase.MutableTableDescriptor @@ -60,7 +57,7 @@ func (p *planner) DropDatabase(ctx context.Context, n *tree.DropDatabase) (planN } // Check that the database exists. - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, string(n.Name), !n.IfExists) + dbDesc, err := p.ResolveMutableDatabaseDescriptor(ctx, string(n.Name), !n.IfExists) if err != nil { return nil, err } @@ -256,14 +253,6 @@ func (n *dropDatabaseNode) startExec(params runParams) error { tbNameStrings = append(tbNameStrings, toDel.tn.FQString()) } - descKey := sqlbase.MakeDescMetadataKey(p.ExecCfg().Codec, n.dbDesc.GetID()) - - b := &kv.Batch{} - if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "Del %s", descKey) - } - b.Del(descKey) - for _, schemaToDelete := range n.schemasToDelete { if err := catalogkv.RemoveSchemaNamespaceEntry( ctx, @@ -276,32 +265,20 @@ func (n *dropDatabaseNode) startExec(params runParams) error { } } - err := catalogkv.RemoveDatabaseNamespaceEntry( - ctx, p.txn, p.ExecCfg().Codec, n.dbDesc.GetName(), p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - ) - if err != nil { + if err := p.removeDbComment(ctx, n.dbDesc.GetID()); err != nil { return err } - // No job was created because no tables were dropped, so zone config can be - // immediately removed, if applicable. - if len(n.allTableObjectsToDelete) == 0 && params.ExecCfg().Codec.ForSystemTenant() { - zoneKeyPrefix := config.MakeZoneKeyPrefix(config.SystemTenantObjectID(n.dbDesc.GetID())) - if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "DelRange %s", zoneKeyPrefix) - } - // Delete the zone config entry for this database. - b.DelRange(zoneKeyPrefix, zoneKeyPrefix.PrefixEnd(), false /* returnKeys */) - } - - p.Descriptors().AddUncommittedDatabase(n.dbDesc.GetName(), n.dbDesc.GetID(), descs.DBDropped) - - if err := p.txn.Run(ctx, b); err != nil { - return err + // Update the database descriptor itself to reflect its dropped state. + dropDetails := descpb.NameInfo{ + ParentID: keys.RootNamespaceID, + ParentSchemaID: keys.RootNamespaceID, + Name: n.dbDesc.GetName(), } - - if err := p.removeDbComment(ctx, n.dbDesc.GetID()); err != nil { - return err + n.dbDesc.DrainingNames = append(n.dbDesc.DrainingNames, dropDetails) + n.dbDesc.State = descpb.DatabaseDescriptor_DROP + if err := p.writeDatabaseChange(ctx, n.dbDesc); err != nil { + return nil } // Log Drop Database event. This is an auditable log event and is recorded diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index e21dcf82b9c8..e4159a4b4077 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -389,52 +389,6 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a'); } } -// Tests that SHOW TABLES works correctly when a database is recreated -// during the time the underlying tables are still being deleted. -func TestShowTablesAfterRecreateDatabase(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() - // Turn off the application of schema changes so that tables do not - // get completely dropped. - params.Knobs = base.TestingKnobs{ - SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ - SchemaChangeJobNoOp: func() bool { - return true - }, - }, - } - s, sqlDB, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(context.Background()) - - if _, err := sqlDB.Exec(` -CREATE DATABASE t; -CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR); -INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd'); -`); err != nil { - t.Fatal(err) - } - - if _, err := sqlDB.Exec(` -DROP DATABASE t CASCADE; -CREATE DATABASE t; -`); err != nil { - t.Fatal(err) - } - - rows, err := sqlDB.Query(` -SET DATABASE=t; -SHOW TABLES; -`) - if err != nil { - t.Fatal(err) - } - defer rows.Close() - if rows.Next() { - t.Fatal("table should be invisible through SHOW TABLES") - } -} - func TestDropIndex(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index a74c85f4196f..0ab4a4641dd4 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -20,7 +20,6 @@ import ( "regexp" "sort" "strings" - "sync" "time" "github.com/cockroachdb/apd/v2" @@ -43,7 +42,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/accessors" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/database" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/colexec" @@ -851,63 +849,6 @@ func (k *TenantTestingKnobs) CanSetClusterSettings() bool { return k != nil && k.ClusterSettingsUpdater != nil } -// databaseCacheHolder is a thread-safe container for a *Cache. -// It also allows clients to block until the cache is updated to a desired -// state. -// -// NOTE(andrei): The way in which we handle the database cache is funky: there's -// this top-level holder, which gets updated on gossip updates. Then, each -// session gets its *Cache, which is updated from the holder after every -// transaction - the SystemConfig is updated and the lazily computer map of db -// names to ids is wiped. So many session are sharing and contending on a -// mutable cache, but nobody's sharing this holder. We should make up our mind -// about whether we like the sharing or not and, if we do, share the holder too. -// Also, we could use the SystemConfigDeltaFilter to limit the updates to -// databases that chaged. One of the problems with the existing architecture -// is if a transaction is completed on a session and the session remains dormant -// for a long time, the next transaction will see a rather old database cache. -type databaseCacheHolder struct { - mu struct { - syncutil.Mutex - c *database.Cache - cv *sync.Cond - } -} - -func newDatabaseCacheHolder(c *database.Cache) *databaseCacheHolder { - dc := &databaseCacheHolder{} - dc.mu.c = c - dc.mu.cv = sync.NewCond(&dc.mu.Mutex) - return dc -} - -func (dc *databaseCacheHolder) getDatabaseCache() *database.Cache { - dc.mu.Lock() - defer dc.mu.Unlock() - return dc.mu.c -} - -// WaitForCacheState implements the DatabaseCacheSubscriber interface. -func (dc *databaseCacheHolder) WaitForCacheState(cond func(*database.Cache) bool) { - dc.mu.Lock() - defer dc.mu.Unlock() - for done := cond(dc.mu.c); !done; done = cond(dc.mu.c) { - dc.mu.cv.Wait() - } -} - -// databaseCacheHolder implements the DatabaseCacheSubscriber interface. -var _ descs.DatabaseCacheSubscriber = &databaseCacheHolder{} - -// updateSystemConfig is called whenever a new system config gossip entry is -// received. -func (dc *databaseCacheHolder) updateSystemConfig(cfg *config.SystemConfig) { - dc.mu.Lock() - dc.mu.c = database.NewCache(dc.mu.c.Codec(), cfg) - dc.mu.cv.Broadcast() - dc.mu.Unlock() -} - func shouldDistributeGivenRecAndMode( rec distRecommendation, mode sessiondata.DistSQLExecMode, ) bool { diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go index 8a205aa61630..d456fc58d99f 100644 --- a/pkg/sql/grant_revoke.go +++ b/pkg/sql/grant_revoke.go @@ -15,7 +15,6 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -145,19 +144,15 @@ func (n *changePrivilegesNode) startExec(params runParams) error { } switch d := descriptor.(type) { - case *sqlbase.ImmutableDatabaseDescriptor: - if err := d.Validate(); err != nil { + case *sqlbase.MutableDatabaseDescriptor: + // TODO (lucy): Ideally this would happen in the same batch as for the + // table descriptors. + if err := p.createDropDatabaseJob(ctx, d.GetID(), nil, nil, + fmt.Sprintf("updating privileges for database %d", d.GetID()), + ); err != nil { return err } - if err := catalogkv.WriteDescToBatch( - ctx, - p.extendedEvalCtx.Tracing.KVTracingEnabled(), - p.ExecCfg().Settings, - b, - p.ExecCfg().Codec, - descriptor.GetID(), - descriptor, - ); err != nil { + if err := p.writeDatabaseChange(ctx, d); err != nil { return err } diff --git a/pkg/sql/logictest/testdata/logic_test/jobs b/pkg/sql/logictest/testdata/logic_test/jobs index 0a89de74a3a2..1d9bd1b7795c 100644 --- a/pkg/sql/logictest/testdata/logic_test/jobs +++ b/pkg/sql/logictest/testdata/logic_test/jobs @@ -13,23 +13,17 @@ GRANT ALL ON DATABASE test TO testuser statement ok CREATE TABLE t(x INT); INSERT INTO t(x) VALUES (1); CREATE INDEX ON t(x) -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT -SELECT job_type, description, user_name FROM [SHOW JOBS] WHERE description != 'updating privileges' +SELECT job_type, description, user_name FROM [SHOW JOBS] ---- -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT -SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE description != 'updating privileges' +SELECT job_type, description, user_name FROM crdb_internal.jobs ---- -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root user testuser @@ -49,19 +43,11 @@ statement ok CREATE TABLE u(x INT); INSERT INTO u(x) VALUES (1); CREATE INDEX ON u(x); -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT SELECT job_type, description, user_name FROM [SHOW JOBS] WHERE description != 'updating privileges' ---- SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE description != 'updating privileges' ---- @@ -71,22 +57,16 @@ SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser user root -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT SELECT job_type, description, user_name FROM [SHOW JOBS] WHERE description != 'updating privileges' ---- -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser -# The "updating privileges" clause in the SELECT statement is for excluding jobs -# run by an unrelated startup migration. -# TODO (lucy): Update this if/when we decide to change how these jobs queued by -# the startup migration are handled. query TTT SELECT job_type, description, user_name FROM crdb_internal.jobs WHERE description != 'updating privileges' ---- -SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root -SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser +SCHEMA CHANGE updating privileges for database 52 root +SCHEMA CHANGE CREATE INDEX ON test.public.t (x) root +SCHEMA CHANGE CREATE INDEX ON test.public.u (x) testuser diff --git a/pkg/sql/logictest/testdata/logic_test/rename_database b/pkg/sql/logictest/testdata/logic_test/rename_database index 5f17af122c22..24a3b3042031 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_database +++ b/pkg/sql/logictest/testdata/logic_test/rename_database @@ -211,3 +211,23 @@ DROP TABLE db2.a; CREATE TABLE db2.a (a int default nextval('a_seq') + nextval(' statement ok USE defaultdb; DROP DATABASE db1 CASCADE + +# Test that names cannot be reused within the same transaction. + +statement ok +CREATE DATABASE oldname + +statement ok +BEGIN + +statement ok +ALTER DATABASE oldname RENAME TO newname + +statement error pgcode 42P04 database "oldname" already exists +CREATE DATABASE oldname + +statement ok +ROLLBACK + +statement ok +DROP DATABASE oldname diff --git a/pkg/sql/logictest/testdata/logic_test/rename_table b/pkg/sql/logictest/testdata/logic_test/rename_table index 1f8d28de11fc..721843b346ab 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_table +++ b/pkg/sql/logictest/testdata/logic_test/rename_table @@ -210,25 +210,6 @@ INSERT INTO d.kv (k,v) VALUES ('g', 'h') statement ok ROLLBACK -# The reuse of a name is allowed. -statement ok -BEGIN - -statement ok -ALTER DATABASE d RENAME TO dnew - -statement ok -CREATE DATABASE d - -statement ok -CREATE TABLE d.kv (k CHAR PRIMARY KEY, v CHAR) - -statement ok -INSERT INTO d.kv (k,v) VALUES ('a', 'b') - -statement ok -COMMIT - # Check that on a rollback a database name cannot be used. statement ok BEGIN diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 9ab3e9003b7d..0354ac6e80b1 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1000,7 +1000,7 @@ type oneAtATimeSchemaResolver struct { func (r oneAtATimeSchemaResolver) getDatabaseByID( id descpb.ID, ) (*sqlbase.ImmutableDatabaseDescriptor, error) { - return r.p.Descriptors().DatabaseCache().GetDatabaseDescByID(r.ctx, r.p.txn, id) + return r.p.Descriptors().GetDatabaseVersionByID(r.ctx, r.p.txn, id, tree.DatabaseLookupFlags{}) } func (r oneAtATimeSchemaResolver) getTableByID(id descpb.ID) (sqlbase.TableDescriptor, error) { diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index e0f6afd6659b..e8e78e09f5e2 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -90,8 +90,8 @@ type PlanHookState interface { GetAllRoles(ctx context.Context) (map[string]bool, error) BumpRoleMembershipTableVersion(ctx context.Context) error EvalAsOfTimestamp(ctx context.Context, asOf tree.AsOfClause) (hlc.Timestamp, error) - ResolveUncachedDatabaseByName( - ctx context.Context, dbName string, required bool) (*UncachedDatabaseDescriptor, error) + ResolveImmutableDatabaseDescriptor( + ctx context.Context, dbName string, required bool) (*ImmutableDatabaseDescriptor, error) ResolveMutableTableDescriptor( ctx context.Context, tn *TableName, required bool, requiredType tree.RequiredTableKind, ) (table *MutableTableDescriptor, err error) diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index 97d54504c17c..c66f9cb6722e 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -28,7 +28,7 @@ import ( ) type renameDatabaseNode struct { - dbDesc *sqlbase.ImmutableDatabaseDescriptor + dbDesc *sqlbase.MutableDatabaseDescriptor newName string } @@ -47,7 +47,7 @@ func (p *planner) RenameDatabase(ctx context.Context, n *tree.RenameDatabase) (p return nil, pgerror.DangerousStatementf("RENAME DATABASE on current database") } - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, string(n.Name), true /*required*/) + dbDesc, err := p.ResolveMutableDatabaseDescriptor(ctx, string(n.Name), true /*required*/) if err != nil { return nil, err } diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index d8d5f3d6e5ea..5acfa40d6740 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -90,14 +90,14 @@ func (n *renameTableNode) startExec(params runParams) error { oldTn := n.oldTn prevDBID := tableDesc.ParentID - var targetDbDesc *UncachedDatabaseDescriptor + var targetDbDesc *ImmutableDatabaseDescriptor // If the target new name has no qualifications, then assume that the table // is intended to be renamed into the same database and schema. newTn := n.newTn if !newTn.ExplicitSchema && !newTn.ExplicitCatalog { newTn.ObjectNamePrefix = oldTn.ObjectNamePrefix var err error - targetDbDesc, err = p.ResolveUncachedDatabaseByName(ctx, string(oldTn.CatalogName), true /* required */) + targetDbDesc, err = p.ResolveImmutableDatabaseDescriptor(ctx, string(oldTn.CatalogName), true /* required */) if err != nil { return err } @@ -118,7 +118,7 @@ func (n *renameTableNode) startExec(params runParams) error { newUn := newTn.ToUnresolvedObjectName() var prefix tree.ObjectNamePrefix var err error - targetDbDesc, prefix, err = p.ResolveUncachedDatabase(ctx, newUn) + targetDbDesc, prefix, err = p.ResolveTargetForObject(ctx, newUn) if err != nil { return err } diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 8d6f8930a895..3985090e9172 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -32,20 +32,33 @@ import ( var _ resolver.SchemaResolver = &planner{} -// ResolveUncachedDatabaseByName looks up a database name from the store. -func (p *planner) ResolveUncachedDatabaseByName( +func (p *planner) ResolveImmutableDatabaseDescriptor( ctx context.Context, dbName string, required bool, -) (res *sqlbase.ImmutableDatabaseDescriptor, err error) { - p.runWithOptions(resolveFlags{skipCache: true}, func() { - var desc sqlbase.DatabaseDescriptor - desc, err = p.LogicalSchemaAccessor().GetDatabaseDesc( - ctx, p.txn, p.ExecCfg().Codec, dbName, p.CommonLookupFlags(required), - ) - if desc != nil { - res = desc.(*sqlbase.ImmutableDatabaseDescriptor) - } - }) - return res, err +) (*sqlbase.ImmutableDatabaseDescriptor, error) { + var desc sqlbase.DatabaseDescriptor + desc, err := p.LogicalSchemaAccessor().GetDatabaseDesc( + ctx, p.txn, p.ExecCfg().Codec, dbName, tree.DatabaseLookupFlags{ + CommonLookupFlags: p.CommonLookupFlags(required), + }, + ) + if err != nil || desc == nil { + return nil, err + } + return desc.(*sqlbase.ImmutableDatabaseDescriptor), nil +} + +func (p *planner) ResolveMutableDatabaseDescriptor( + ctx context.Context, name string, required bool, +) (*sqlbase.MutableDatabaseDescriptor, error) { + desc, err := p.LogicalSchemaAccessor().GetDatabaseDesc( + ctx, p.txn, p.ExecCfg().Codec, name, tree.DatabaseLookupFlags{ + CommonLookupFlags: p.CommonLookupFlags(required), + RequireMutable: true, + }) + if err != nil || desc == nil { + return nil, err + } + return desc.(*sqlbase.MutableDatabaseDescriptor), nil } // runWithOptions sets the provided resolution flags for the @@ -97,13 +110,13 @@ func (p *planner) ResolveUncachedTableDescriptor( return table, err } -func (p *planner) ResolveUncachedDatabase( +func (p *planner) ResolveTargetForObject( ctx context.Context, un *tree.UnresolvedObjectName, -) (res *UncachedDatabaseDescriptor, namePrefix tree.ObjectNamePrefix, err error) { - var prefix *catalog.ResolvedObjectPrefix - p.runWithOptions(resolveFlags{skipCache: true}, func() { - prefix, namePrefix, err = resolver.ResolveTargetObject(ctx, p, un) - }) +) (res *ImmutableDatabaseDescriptor, namePrefix tree.ObjectNamePrefix, err error) { + prefix, namePrefix, err := resolver.ResolveTargetObject(ctx, p, un) + if err != nil { + return nil, tree.ObjectNamePrefix{}, err + } return prefix.Database, namePrefix, err } @@ -112,7 +125,8 @@ func (p *planner) LookupSchema( ctx context.Context, dbName, scName string, ) (found bool, scMeta tree.SchemaMeta, err error) { sc := p.LogicalSchemaAccessor() - dbDesc, err := sc.GetDatabaseDesc(ctx, p.txn, p.ExecCfg().Codec, dbName, p.CommonLookupFlags(false /*required*/)) + dbDesc, err := sc.GetDatabaseDesc(ctx, p.txn, p.ExecCfg().Codec, dbName, + tree.DatabaseLookupFlags{CommonLookupFlags: p.CommonLookupFlags(false /*required*/)}) if err != nil || dbDesc == nil { return false, nil, err } @@ -154,9 +168,8 @@ func (p *planner) GetTypeDescriptor( if err != nil { return tree.TypeName{}, nil, err } - // TODO (lucy): This database access should go through the collection. - // When I try to use the DatabaseCache() here, a nil pointer deref occurs. - dbDesc, err := catalogkv.MustGetDatabaseDescByID(ctx, p.txn, p.ExecCfg().Codec, desc.ParentID) + dbDesc, err := p.Descriptors().GetDatabaseVersionByID(ctx, p.txn, desc.ParentID, + tree.DatabaseLookupFlags{CommonLookupFlags: tree.CommonLookupFlags{Required: true}}) if err != nil { return tree.TypeName{}, nil, err } @@ -224,7 +237,7 @@ func getDescriptorsFromTargetList( } descs := make([]sqlbase.Descriptor, 0, len(targets.Databases)) for _, database := range targets.Databases { - descriptor, err := p.ResolveUncachedDatabaseByName(ctx, string(database), true /*required*/) + descriptor, err := p.ResolveMutableDatabaseDescriptor(ctx, string(database), true /*required*/) if err != nil { return nil, err } @@ -311,7 +324,8 @@ func findTableContainingIndex( lookupFlags tree.CommonLookupFlags, ) (result *tree.TableName, desc *MutableTableDescriptor, err error) { sa := sc.LogicalSchemaAccessor() - dbDesc, err := sa.GetDatabaseDesc(ctx, txn, codec, dbName, lookupFlags) + dbDesc, err := sa.GetDatabaseDesc(ctx, txn, codec, dbName, + tree.DatabaseLookupFlags{CommonLookupFlags: lookupFlags}) if dbDesc == nil || err != nil { return nil, nil, err } diff --git a/pkg/sql/schema_accessors.go b/pkg/sql/schema_accessors.go index 94e84d3fe668..8c6b680ef467 100644 --- a/pkg/sql/schema_accessors.go +++ b/pkg/sql/schema_accessors.go @@ -39,9 +39,12 @@ type ( // DatabaseDescriptor is provided for convenience and to make the // interface definitions below more intuitive. DatabaseDescriptor = descpb.DatabaseDescriptor - // UncachedDatabaseDescriptor is provided for convenience and to make the + // MutableDatabaseDescriptor is provided for convenience and to make the // interface definitions below more intuitive. - UncachedDatabaseDescriptor = sqlbase.ImmutableDatabaseDescriptor + MutableDatabaseDescriptor = sqlbase.MutableDatabaseDescriptor + // ImmutableDatabaseDescriptor is provided for convenience and to make the + // interface definitions below more intuitive. + ImmutableDatabaseDescriptor = sqlbase.ImmutableDatabaseDescriptor // MutableTableDescriptor is provided for convenience and to make the // interface definitions below more intuitive. MutableTableDescriptor = sqlbase.MutableTableDescriptor diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index ce04c76b9781..56e0b5acc8d3 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -18,6 +18,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1775,63 +1776,141 @@ func (r schemaChangeResumer) Resume( return scErr } - // For an empty database, the zone config for it was already GC'ed and there's - // nothing left to do. - if details.DroppedDatabaseID != descpb.InvalidID && - len(details.DroppedTables) == 0 && - len(details.DroppedTypes) == 0 { + // If we're dropping a database, the database may have children to also drop. + // We also leave open the possibility of jobs involving dropping multiple + // tables with no database, for 19.2 compatibility. + databaseHasChildrenToDrop := len(details.DroppedTables) > 0 || len(details.DroppedTypes) > 0 + + if details.DroppedDatabaseID == descpb.InvalidID && + details.TableID == descpb.InvalidID && + !databaseHasChildrenToDrop { + return errors.AssertionFailedf("schema change has no specified database or table(s)") + } + + // Single-table schema change. + if details.TableID != descpb.InvalidID { + return execSchemaChange(details.TableID, details.MutationID, details.DroppedDatabaseID) + } + + // TODO (lucy): !!! add retries + if details.DroppedDatabaseID != descpb.InvalidID { + if err := execDatabaseChanges(ctx, details, p); err != nil { + return err + } + } + + if !databaseHasChildrenToDrop { return nil } - // If a database is being dropped, handle this separately by draining names - // for all the tables and types. - // - // This also covers other cases where we have a leftover 19.2 job that drops - // multiple tables in a single job (e.g., TRUNCATE on multiple tables), so - // it's possible for DroppedDatabaseID to be unset. - if details.DroppedDatabaseID != descpb.InvalidID || len(details.DroppedTables) > 1 { - // Drop all of the types in the database. - for i := range details.DroppedTypes { - ts := &typeSchemaChanger{ - typeID: details.DroppedTypes[i], - execCfg: p.ExecCfg(), - } - if err := ts.execWithRetry(ctx); err != nil { + // Drop all of the types in the database. + for i := range details.DroppedTypes { + ts := &typeSchemaChanger{ + typeID: details.DroppedTypes[i], + execCfg: p.ExecCfg(), + } + if err := ts.execWithRetry(ctx); err != nil { + return err + } + } + + // Drop the tables now. + for i := range details.DroppedTables { + droppedTable := &details.DroppedTables[i] + if err := execSchemaChange(droppedTable.ID, descpb.InvalidMutationID, details.DroppedDatabaseID); err != nil { + return err + } + } + dropTime := timeutil.Now().UnixNano() + tablesToGC := make([]jobspb.SchemaChangeGCDetails_DroppedID, len(details.DroppedTables)) + for i, table := range details.DroppedTables { + tablesToGC[i] = jobspb.SchemaChangeGCDetails_DroppedID{ID: table.ID, DropTime: dropTime} + } + multiTableGCDetails := jobspb.SchemaChangeGCDetails{ + Tables: tablesToGC, + ParentID: details.DroppedDatabaseID, + } + + return startGCJob( + ctx, + p.ExecCfg().DB, + p.ExecCfg().JobRegistry, + r.job.Payload().Username, + r.job.Payload().Description, + multiTableGCDetails, + ) +} + +func execDatabaseChanges( + ctx context.Context, details jobspb.SchemaChangeDetails, p PlanHookState, +) error { + // If there's a database specified that's undergoing a schema change, check + // for draining names. This can be the case if the database is either being + // renamed, or being dropped. In the latter case, we also delete the database + // descriptor. If there are no child tables or types to be dropped, we also + // remove the zone config for the database. + if details.DroppedDatabaseID != descpb.InvalidID { + var dbDesc *sqlbase.ImmutableDatabaseDescriptor + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + dbDesc, err = catalogkv.MustGetDatabaseDescByID(ctx, txn, p.ExecCfg().Codec, details.DroppedDatabaseID) + return err + }); err != nil { + return err + } + + if len(dbDesc.GetDrainingNames()) > 0 { + log.Infof(ctx, "draining names on database %d", dbDesc.GetID()) + if err := drainNamesForDescriptor( + ctx, + dbDesc.GetID(), + p.LeaseMgr(), + p.ExecCfg().Codec, + nil, /* beforeDrainNames */ + ); err != nil { return err } } - // Drop the tables now. - for i := range details.DroppedTables { - droppedTable := &details.DroppedTables[i] - if err := execSchemaChange(droppedTable.ID, descpb.InvalidMutationID, details.DroppedDatabaseID); err != nil { + if dbDesc.Dropped() { + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + if err := txn.SetSystemConfigTrigger(p.ExecCfg().Codec.ForSystemTenant()); err != nil { + return err + } + b := &kv.Batch{} + + descKey := sqlbase.MakeDescMetadataKey(p.ExecCfg().Codec, dbDesc.GetID()) + b.Del(descKey) + + // Remove the zone config if applicable. + databaseHasChildrenToDrop := len(details.DroppedTables) > 0 || len(details.DroppedTypes) > 0 + if !databaseHasChildrenToDrop && p.ExecCfg().Codec.ForSystemTenant() { + zoneKeyPrefix := config.MakeZoneKeyPrefix(config.SystemTenantObjectID(dbDesc.GetID())) + if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { + log.VEventf(ctx, 2, "DelRange %s", zoneKeyPrefix) + } + // Delete the zone config entry for this database. + b.DelRange(zoneKeyPrefix, zoneKeyPrefix.PrefixEnd(), false /* returnKeys */) + } + + return txn.Run(ctx, b) + }); err != nil { return err } } - dropTime := timeutil.Now().UnixNano() - tablesToGC := make([]jobspb.SchemaChangeGCDetails_DroppedID, len(details.DroppedTables)) - for i, table := range details.DroppedTables { - tablesToGC[i] = jobspb.SchemaChangeGCDetails_DroppedID{ID: table.ID, DropTime: dropTime} - } - multiTableGCDetails := jobspb.SchemaChangeGCDetails{ - Tables: tablesToGC, - ParentID: details.DroppedDatabaseID, - } - return startGCJob( - ctx, - p.ExecCfg().DB, - p.ExecCfg().JobRegistry, - r.job.Payload().Username, - r.job.Payload().Description, - multiTableGCDetails, - ) - } - if details.TableID == descpb.InvalidID { - return errors.AssertionFailedf("schema change has no specified database or table(s)") + if err := WaitToUpdateLeases(ctx, p.LeaseMgr(), dbDesc.GetID()); err != nil { + if errors.Is(err, sqlbase.ErrDescriptorNotFound) { + // This happens when we dropped the database. + return nil + } + log.Warningf(ctx, "waiting to update leases: %+v", err) + // As we are dismissing the error, go through the recording motions. + // This ensures that any important error gets reported to Sentry, etc. + sqltelemetry.RecordError(ctx, err, &p.ExecCfg().Settings.SV) + } } - return execSchemaChange(details.TableID, details.MutationID, details.DroppedDatabaseID) + return nil } // OnFailOrCancel is part of the jobs.Resumer interface. diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 2fa96e531452..dda04ebd5ce1 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -156,7 +156,7 @@ func (n *scrubNode) Close(ctx context.Context) { func (n *scrubNode) startScrubDatabase(ctx context.Context, p *planner, name *tree.Name) error { // Check that the database exists. database := string(*name) - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, database, true /*required*/) + dbDesc, err := p.ResolveImmutableDatabaseDescriptor(ctx, database, true /*required*/) if err != nil { return err } diff --git a/pkg/sql/sem/tree/name_resolution.go b/pkg/sql/sem/tree/name_resolution.go index c9d8f9ef5522..8fd71c493afa 100644 --- a/pkg/sql/sem/tree/name_resolution.go +++ b/pkg/sql/sem/tree/name_resolution.go @@ -568,7 +568,10 @@ type CommonLookupFlags struct { } // DatabaseLookupFlags is the flag struct suitable for GetDatabaseDesc(). -type DatabaseLookupFlags = CommonLookupFlags +type DatabaseLookupFlags struct { + CommonLookupFlags + RequireMutable bool +} // DatabaseListFlags is the flag struct suitable for GetObjectNames(). type DatabaseListFlags struct { diff --git a/pkg/sql/serial.go b/pkg/sql/serial.go index d55b8a7bfe7d..c0bfba2aa23b 100644 --- a/pkg/sql/serial.go +++ b/pkg/sql/serial.go @@ -117,13 +117,13 @@ func (p *planner) processSerialInColumnDef( tree.Name(tableName.Table() + "_" + string(d.Name) + "_seq")) // The first step in the search is to prepare the seqName to fill in - // the catalog/schema parent. This is what ResolveUncachedDatabase does. + // the catalog/schema parent. This is what ResolveTargetForObject does. // // Here and below we skip the cache because name resolution using // the cache does not work (well) if the txn retries and the // descriptor was written already in an early txn attempt. un := seqName.ToUnresolvedObjectName() - dbDesc, prefix, err := p.ResolveUncachedDatabase(ctx, un) + dbDesc, prefix, err := p.ResolveTargetForObject(ctx, un) if err != nil { return nil, nil, nil, nil, err } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 15e0cb116f75..9dc7aa13de2a 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -190,7 +190,7 @@ func checkPrivilegeForSetZoneConfig(ctx context.Context, p *planner, zs tree.Zon if zs.Database == "system" { return p.RequireAdminRole(ctx, "alter the system database") } - dbDesc, err := p.ResolveUncachedDatabaseByName(ctx, string(zs.Database), true) + dbDesc, err := p.ResolveImmutableDatabaseDescriptor(ctx, string(zs.Database), true) if err != nil { return err } diff --git a/pkg/sql/show_zone_config.go b/pkg/sql/show_zone_config.go index aa6d63af1c42..d9302878afc4 100644 --- a/pkg/sql/show_zone_config.go +++ b/pkg/sql/show_zone_config.go @@ -106,7 +106,7 @@ func getShowZoneConfigRow( return nil, err } } else if zoneSpecifier.Database != "" { - database, err := p.ResolveUncachedDatabaseByName( + database, err := p.ResolveImmutableDatabaseDescriptor( ctx, string(zoneSpecifier.Database), true, /* required */ diff --git a/pkg/sql/sqlbase/database_desc.go b/pkg/sql/sqlbase/database_desc.go index 29693283c9f6..2069c92fdb48 100644 --- a/pkg/sql/sqlbase/database_desc.go +++ b/pkg/sql/sqlbase/database_desc.go @@ -54,7 +54,7 @@ type MutableDatabaseDescriptor struct { // initial version from an id and name. func NewInitialDatabaseDescriptor( id descpb.ID, name string, owner string, -) *ImmutableDatabaseDescriptor { +) *MutableDatabaseDescriptor { return NewInitialDatabaseDescriptorWithPrivileges(id, name, descpb.NewDefaultPrivilegeDescriptor(owner)) } @@ -63,8 +63,8 @@ func NewInitialDatabaseDescriptor( // initial version from an id and name. func NewInitialDatabaseDescriptorWithPrivileges( id descpb.ID, name string, privileges *descpb.PrivilegeDescriptor, -) *ImmutableDatabaseDescriptor { - return NewImmutableDatabaseDescriptor(descpb.DatabaseDescriptor{ +) *MutableDatabaseDescriptor { + return NewMutableCreatedDatabaseDescriptor(descpb.DatabaseDescriptor{ Name: name, ID: id, Version: 1, @@ -82,6 +82,17 @@ func NewImmutableDatabaseDescriptor(desc descpb.DatabaseDescriptor) *ImmutableDa return &ret } +// NewMutableCreatedDatabaseDescriptor returns a MutableDatabaseDescriptor from +// the given database descriptor with a nil cluster version. This is for a +// database that is created in the same transaction. +func NewMutableCreatedDatabaseDescriptor( + desc descpb.DatabaseDescriptor, +) *MutableDatabaseDescriptor { + return &MutableDatabaseDescriptor{ + ImmutableDatabaseDescriptor: makeImmutableDatabaseDescriptor(desc), + } +} + // NewMutableExistingDatabaseDescriptor returns a MutableDatabaseDescriptor from the // given database descriptor with the cluster version also set to the descriptor. // This is for databases that already exist. diff --git a/pkg/sql/sqlbase/errors.go b/pkg/sql/sqlbase/errors.go index 51914e201f30..4d5a8d6fc778 100644 --- a/pkg/sql/sqlbase/errors.go +++ b/pkg/sql/sqlbase/errors.go @@ -11,11 +11,15 @@ package sqlbase import ( + "context" + "runtime/debug" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -116,6 +120,7 @@ func NewUndefinedTypeError(name tree.NodeFormatter) error { // NewUndefinedRelationError creates an error that represents a missing database table or view. func NewUndefinedRelationError(name tree.NodeFormatter) error { + log.Error(context.TODO(), string(debug.Stack())) return pgerror.Newf(pgcode.UndefinedTable, "relation %q does not exist", tree.ErrString(name)) } diff --git a/pkg/sql/sqlbase/system.go b/pkg/sql/sqlbase/system.go index d8be57a90c33..787790674ac9 100644 --- a/pkg/sql/sqlbase/system.go +++ b/pkg/sql/sqlbase/system.go @@ -358,13 +358,14 @@ const SystemDatabaseName = "system" // MakeSystemDatabaseDesc constructs a copy of the system database // descriptor. func MakeSystemDatabaseDesc() *ImmutableDatabaseDescriptor { - return NewInitialDatabaseDescriptorWithPrivileges( - keys.SystemDatabaseID, - SystemDatabaseName, + return NewImmutableDatabaseDescriptor(descpb.DatabaseDescriptor{ + Name: SystemDatabaseName, + ID: keys.SystemDatabaseID, + Version: 1, // Assign max privileges to root user. - descpb.NewCustomSuperuserPrivilegeDescriptor( + Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor( descpb.SystemAllowedPrivileges[keys.SystemDatabaseID], security.NodeUser), - ) + }) } // These system config descpb.TableDescriptor literals should match the descriptor diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 7dc15a6cddf1..845c64aa1caf 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -241,7 +241,9 @@ var varGen = map[string]sessionVar{ if len(dbName) != 0 { // Verify database descriptor exists. if _, err := evalCtx.schemaAccessors.logical.GetDatabaseDesc( - ctx, evalCtx.Txn, evalCtx.Codec, dbName, tree.DatabaseLookupFlags{Required: true}, + ctx, evalCtx.Txn, evalCtx.Codec, dbName, tree.DatabaseLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{Required: true}, + }, ); err != nil { return "", err } diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index b01a9a747a97..77dada8f24d2 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -453,7 +453,11 @@ func (e virtualDefEntry) getPlanInfo( var dbDesc *sqlbase.ImmutableDatabaseDescriptor if dbName != "" { dbDescI, err := p.LogicalSchemaAccessor().GetDatabaseDesc(ctx, p.txn, p.ExecCfg().Codec, - dbName, tree.DatabaseLookupFlags{Required: true, AvoidCached: p.avoidCachedDescriptors}) + dbName, tree.DatabaseLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, AvoidCached: p.avoidCachedDescriptors, + }, + }) if err != nil { return nil, err } @@ -665,7 +669,12 @@ var publicSelectPrivileges = descpb.NewPrivilegeDescriptor( ) func initVirtualDatabaseDesc(id descpb.ID, name string) *sqlbase.ImmutableDatabaseDescriptor { - return sqlbase.NewInitialDatabaseDescriptorWithPrivileges(id, name, publicSelectPrivileges) + return sqlbase.NewImmutableDatabaseDescriptor(descpb.DatabaseDescriptor{ + Name: name, + ID: id, + Version: 1, + Privileges: publicSelectPrivileges, + }) } // getEntries is part of the VirtualTabler interface. diff --git a/pkg/sql/virtual_table.go b/pkg/sql/virtual_table.go index 58ab05854ae3..5b913a25c8ac 100644 --- a/pkg/sql/virtual_table.go +++ b/pkg/sql/virtual_table.go @@ -247,7 +247,11 @@ func (v *vTableLookupJoinNode) startExec(params runParams) error { params.p.txn, params.p.ExecCfg().Codec, v.dbName, - tree.DatabaseLookupFlags{Required: true, AvoidCached: params.p.avoidCachedDescriptors}, + tree.DatabaseLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, AvoidCached: params.p.avoidCachedDescriptors, + }, + }, ) if err != nil { return err