Skip to content

Commit

Permalink
catalog,catalogkv: add and use descriptor builder
Browse files Browse the repository at this point in the history
Release justification: bug fixes and low-risk updates to new
functionality.

This commit is, at heart, a refactor of how catalog.Descriptor objects
are created; either when read from storage via the catalogkv API
or when constructed from an in-memory descpb.*Descriptor struct.

The original motivation for this change was the fact that previously,
calling the descriptor validation methods could sometimes modify the
descriptor itself. Specifically, ValidateSelf would perform "privilege
surgery" and modify the privileges descriptor. This appears to have been
around since version 2 and is, in fact, responsible for a few bugs,
which this commit also fixes.

Taking a broader perspective it appears necessary that all such
descriptor surgeries should be performed together when read from
storage regardless of descriptor subtype (table, db, type, schema).
These surgeries are typically no-ops: table format upgrade, foreign key
representation upgrade, etc. Considering that catalog.Descriptor objects
can be created either from storage or from memory, this involves:
  1. unifying the constructors in the (table|db|type|schema)desc
     packages,
  2. unifying the descriptor unwrapping logic in catalogkv.

To address (1) this commit introduces catalog.DescriptorBuilder, a
builder interface which replaces the NewImmutable, NewExistingMutable,
NewCreatedMutable, etc. constructors. The builder does the same thing as
these except it uses a deep copy of the proto descriptor instead of
a shallow copy. This has, in itself, uncovered a few minor bugs.
Still, the main reason for the existence of this builder object is that
we can now perform all post-deserialization descriptor changes in a
method to be called before building the catalog.Descriptor object
proper.

This commit also changes how we update the descriptor modification time
using the MVCC timestamp. This is done at builder creation time, or, in
cases where we stick to descpb structs, using a new
descp.FromDescriptorWithMVCCTimestamp function which replaces
descpb.TableFromDescriptor and the descpb.Descriptor.GetTable,
GetDatabase, GetType and GetSchema methods. These methods are now all
forbidden from casual use by a new linter check (it used to be just
GetTable).

To address (2) this commit uses the new builder to unify all descriptor
wrapping logic in catalogkv into a new descriptorFromKeyValue function.
This function is what backs the catalogkv API now. This commit also
changes the API somewhat, the old GetDescriptorFromID function with its
many flag arguments is gone in favour of many lightweight public getter
functions (complementing existing ones) which all delegate to a common
private getDescriptorByID function. The most important side benefit to
doing all this is that we now no longer skip cross-reference validation
when getting mutable descriptors through the catalogkv API.
Cross-reference validation is now the default, and the exceptions to
this are few:
  - The catalogkv DescGetter implemementation, aka the
    oneLevelUncachedDescGetter, does not perform these checks, because
    they, in turn, require a DescGetter. In theory we could get around
    this infinite recursion with caching but right now there's no point.
  - GetAllDescriptorsUnvalidated explicitly performs no validation at
    all.

Doing all of this has uncovered a few bugs, the most serious of which is
that the protected_ts_meta system table didn't have the correct
permissions applied.

Furthermore this commit also unifies namespace collision detection logic
in catalogkv. A side-effect of this is that error messages for PG codes
42710 and 42P07 contain qualified names more often than before.

Release note (bug fix): Fixed privileges for system.protected_ts_meta.
  • Loading branch information
Marius Posta committed Mar 11, 2021
1 parent a86cf54 commit 15ee387
Show file tree
Hide file tree
Showing 167 changed files with 2,011 additions and 1,671 deletions.
5 changes: 2 additions & 3 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
Expand Down Expand Up @@ -186,7 +185,7 @@ func backup(
g := ctxgroup.WithContext(ctx)
pkIDs := make(map[uint64]bool)
for i := range backupManifest.Descriptors {
if t := descpb.TableFromDescriptor(&backupManifest.Descriptors[i], hlc.Timestamp{}); t != nil {
if t, _, _, _ := descpb.FromDescriptor(&backupManifest.Descriptors[i]); t != nil {
pkIDs[roachpb.BulkOpSummaryID(uint64(t.ID), uint64(t.PrimaryIndex.ID))] = true
}
}
Expand Down Expand Up @@ -327,7 +326,7 @@ func backup(
}
var tableStatistics []*stats.TableStatisticProto
for i := range backupManifest.Descriptors {
if tableDesc := descpb.TableFromDescriptor(&backupManifest.Descriptors[i], hlc.Timestamp{}); tableDesc != nil {
if tableDesc, _, _, _ := descpb.FromDescriptor(&backupManifest.Descriptors[i]); tableDesc != nil {
// Collect all the table stats for this table.
tableStatisticsAcc, err := statsCache.GetTableStats(ctx, tableDesc.GetID())
if err != nil {
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -332,9 +332,9 @@ func spansForAllTableIndexes(
// entire interval. DROPPED tables should never later become PUBLIC.
// TODO(pbardea): Consider and test the interaction between revision_history
// backups and OFFLINE tables.
rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{})
rawTbl, _, _, _ := descpb.FromDescriptor(rev.Desc)
if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP {
tbl := tabledesc.NewImmutable(*rawTbl)
tbl := tabledesc.NewBuilder(rawTbl).BuildImmutableTable()
revSpans, err := getLogicallyMergedTableSpans(tbl, added, execCfg.Codec, rev.Time,
checkForKVInBounds)
if err != nil {
Expand Down Expand Up @@ -996,7 +996,7 @@ func backupPlanHook(
dbsInPrev := make(map[descpb.ID]struct{})
rawDescs := prevBackups[len(prevBackups)-1].Descriptors
for i := range rawDescs {
if t := descpb.TableFromDescriptor(&rawDescs[i], hlc.Timestamp{}); t != nil {
if t, _, _, _ := descpb.FromDescriptor(&rawDescs[i]); t != nil {
tablesInPrev[t.ID] = struct{}{}
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6058,7 +6058,7 @@ func getMockTableDesc(
PrimaryIndex: pkIndex,
Indexes: indexes,
}
return tabledesc.NewImmutable(mockTableDescriptor)
return tabledesc.NewBuilder(&mockTableDescriptor).BuildImmutableTable()
}

// Unit tests for the getLogicallyMergedTableSpans() method.
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backupbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/dbdesc",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/schemadesc",
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backupbase/targets.go
Original file line number Diff line number Diff line change
Expand Up @@ -560,7 +560,7 @@ func FullClusterTargets(
for _, desc := range allDescs {
switch desc := desc.(type) {
case catalog.DatabaseDescriptor:
dbDesc := dbdesc.NewImmutable(*desc.DatabaseDesc())
dbDesc := dbdesc.NewBuilder(desc.DatabaseDesc()).BuildImmutableDatabase()
fullClusterDescs = append(fullClusterDescs, desc)
if dbDesc.GetID() != systemschema.SystemDB.GetID() {
// The only database that isn't being fully backed up is the system DB.
Expand Down
18 changes: 9 additions & 9 deletions pkg/ccl/backupccl/backupbase/targets_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc"
Expand Down Expand Up @@ -48,23 +49,22 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
type tbDesc = descpb.TableDescriptor
type typDesc = descpb.TypeDescriptor
ts1 := hlc.Timestamp{WallTime: 1}
mkTable := func(descriptor tbDesc) catalog.TableDescriptor {
desc := tabledesc.NewImmutable(descriptor)
desc.TableDesc().ModificationTime = ts1
return desc
mkTable := func(descriptor tbDesc) catalog.Descriptor {
descProto := tabledesc.NewBuilder(&descriptor).BuildImmutable().DescriptorProto()
return catalogkv.NewBuilderWithMVCCTimestamp(descProto, ts1).BuildImmutable()
}
mkDB := func(id descpb.ID, name string) *dbdesc.Immutable {
mkDB := func(id descpb.ID, name string) catalog.Descriptor {
return &dbdesc.NewInitial(id, name, security.AdminRoleName()).Immutable
}
mkTyp := func(desc typDesc) *typedesc.Immutable {
mkTyp := func(desc typDesc) catalog.Descriptor {
// Set a default parent schema for the type descriptors.
if desc.ParentSchemaID == descpb.InvalidID {
desc.ParentSchemaID = keys.PublicSchemaID
}
return typedesc.NewImmutable(desc)
return typedesc.NewBuilder(&desc).BuildImmutable()
}
mkSchema := func(desc scDesc) *schemadesc.Immutable {
return schemadesc.NewImmutable(desc)
mkSchema := func(desc scDesc) catalog.Descriptor {
return schemadesc.NewBuilder(&desc).BuildImmutable()
}
toOid := typedesc.TypeIDToOID
typeExpr := "'hello'::@100015 = 'hello'::@100015"
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -835,7 +835,7 @@ func loadSQLDescsFromBackupsAtTime(
unwrapDescriptors := func(raw []descpb.Descriptor) []catalog.Descriptor {
ret := make([]catalog.Descriptor, 0, len(raw))
for i := range raw {
ret = append(ret, catalogkv.UnwrapDescriptorRaw(context.TODO(), &raw[i]))
ret = append(ret, catalogkv.NewBuilder(&raw[i]).BuildExistingMutable())
}
return ret
}
Expand Down Expand Up @@ -869,7 +869,7 @@ func loadSQLDescsFromBackupsAtTime(
for _, raw := range byID {
// A revision may have been captured before it was in a DB that is
// backed up -- if the DB is missing, filter the object.
desc := catalogkv.UnwrapDescriptorRaw(context.TODO(), raw)
desc := catalogkv.NewBuilder(raw).BuildExistingMutable()
var isObject bool
switch desc.(type) {
case catalog.TableDescriptor, catalog.TypeDescriptor, catalog.SchemaDescriptor:
Expand Down
66 changes: 25 additions & 41 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -444,16 +444,7 @@ func WriteDescriptors(
}
return err
}

bdg := catalogkv.NewOneLevelUncachedDescGetter(txn, codec)
descs := make([]catalog.Descriptor, 0, len(databases)+len(tables))
for _, table := range tables {
descs = append(descs, table)
}
for _, db := range databases {
descs = append(descs, db)
}
return catalog.ValidateSelfAndCrossReferences(ctx, bdg, descs...)
return nil
}()
return errors.Wrapf(err, "restoring table desc and namespace entries")
}
Expand Down Expand Up @@ -882,9 +873,9 @@ func spansForAllRestoreTableIndexes(
// entire interval. DROPPED tables should never later become PUBLIC.
// TODO(pbardea): Consider and test the interaction between revision_history
// backups and OFFLINE tables.
rawTbl := descpb.TableFromDescriptor(rev.Desc, hlc.Timestamp{})
rawTbl, _, _, _ := descpb.FromDescriptor(rev.Desc)
if rawTbl != nil && rawTbl.State != descpb.DescriptorState_DROP {
tbl := tabledesc.NewImmutable(*rawTbl)
tbl := tabledesc.NewBuilder(rawTbl).BuildImmutableTable()
for _, idx := range tbl.NonDropIndexes() {
key := tableAndIndex{tableID: tbl.GetID(), indexID: idx.GetID()}
if !added[key] {
Expand Down Expand Up @@ -949,7 +940,7 @@ func createImportingDescriptors(
for _, desc := range sqlDescs {
switch desc := desc.(type) {
case catalog.TableDescriptor:
mut := tabledesc.NewCreatedMutable(*desc.TableDesc())
mut := tabledesc.NewBuilder(desc.TableDesc()).BuildCreatedMutableTable()
if shouldPreRestore(mut) {
preRestoreTables = append(preRestoreTables, mut)
} else {
Expand All @@ -960,15 +951,15 @@ func createImportingDescriptors(
oldTableIDs = append(oldTableIDs, mut.GetID())
case catalog.DatabaseDescriptor:
if _, ok := details.DescriptorRewrites[desc.GetID()]; ok {
mut := dbdesc.NewCreatedMutable(*desc.DatabaseDesc())
mut := dbdesc.NewBuilder(desc.DatabaseDesc()).BuildCreatedMutableDatabase()
databases = append(databases, mut)
mutableDatabases = append(mutableDatabases, mut)
}
case catalog.SchemaDescriptor:
mut := schemadesc.NewCreatedMutable(*desc.SchemaDesc())
mut := schemadesc.NewBuilder(desc.SchemaDesc()).BuildCreatedMutableSchema()
schemas = append(schemas, mut)
case catalog.TypeDescriptor:
mut := typedesc.NewCreatedMutable(*desc.TypeDesc())
mut := typedesc.NewBuilder(desc.TypeDesc()).BuildCreatedMutableType()
types = append(types, mut)
}
}
Expand Down Expand Up @@ -1946,9 +1937,9 @@ func (r *restoreResumer) dropDescriptors(

// Delete any schema descriptors that this restore created. Also collect the
// descriptors so we can update their parent databases later.
dbsWithDeletedSchemas := make(map[descpb.ID][]*descpb.SchemaDescriptor)
dbsWithDeletedSchemas := make(map[descpb.ID][]catalog.SchemaDescriptor)
for _, schemaDesc := range details.SchemaDescs {
sc := schemadesc.NewMutableExisting(*schemaDesc)
sc := schemadesc.NewBuilder(schemaDesc).BuildImmutableSchema()
// We need to ignore descriptors we just added since we haven't committed the txn that deletes these.
isSchemaEmpty, err := isSchemaEmpty(ctx, txn, sc.GetID(), allDescs, ignoredChildDescIDs)
if err != nil {
Expand All @@ -1963,19 +1954,19 @@ func (r *restoreResumer) dropDescriptors(
ctx,
b,
codec,
sc.ParentID,
sc.GetParentID(),
keys.RootNamespaceID,
sc.Name,
sc.GetName(),
false, /* kvTrace */
)
b.Del(catalogkeys.MakeDescMetadataKey(codec, sc.ID))
dbsWithDeletedSchemas[sc.GetParentID()] = append(dbsWithDeletedSchemas[sc.GetParentID()], sc.SchemaDesc())
b.Del(catalogkeys.MakeDescMetadataKey(codec, sc.GetID()))
dbsWithDeletedSchemas[sc.GetParentID()] = append(dbsWithDeletedSchemas[sc.GetParentID()], sc)
}

// Delete the database descriptors.
deletedDBs := make(map[descpb.ID]struct{})
for _, dbDesc := range details.DatabaseDescs {
db := dbdesc.NewExistingMutable(*dbDesc)
db := dbdesc.NewBuilder(dbDesc).BuildExistingMutable()
// We need to ignore descriptors we just added since we haven't committed the txn that deletes these.
isDBEmpty, err := isDatabaseEmpty(ctx, txn, db.GetID(), allDescs, ignoredChildDescIDs)
if err != nil {
Expand Down Expand Up @@ -2043,7 +2034,7 @@ func (r *restoreResumer) removeExistingTypeBackReferences(
existingTypes := make(map[descpb.ID]*typedesc.Mutable)
for i := range details.TypeDescs {
typ := details.TypeDescs[i]
restoredTypes[typ.ID] = typedesc.NewImmutable(*typ)
restoredTypes[typ.ID] = typedesc.NewBuilder(typ).BuildImmutableType()
}
for _, tbl := range restoredTables {
lookup := func(id descpb.ID) (catalog.TypeDescriptor, error) {
Expand Down Expand Up @@ -2114,30 +2105,21 @@ func getRestoringPrivileges(
user security.SQLUsername,
wroteDBs map[descpb.ID]catalog.DatabaseDescriptor,
descCoverage tree.DescriptorCoverage,
) (*descpb.PrivilegeDescriptor, error) {
// Don't update the privileges of descriptors if we're doing a cluster
// restore.
if descCoverage == tree.AllDescriptors {
return nil, nil
}

var updatedPrivileges *descpb.PrivilegeDescriptor

) (updatedPrivileges *descpb.PrivilegeDescriptor, err error) {
switch desc := desc.(type) {
case catalog.TableDescriptor, catalog.SchemaDescriptor:
if wrote, ok := wroteDBs[desc.GetParentID()]; ok {
// If we're creating a new database in this restore, the privileges of the
// table and schema should be that of the parent DB.
//
// Leave the privileges of the temp system tables as the default too.
if descCoverage != tree.AllDescriptors || wrote.GetName() == restoreTempSystemDB {
if descCoverage == tree.RequestedDescriptors || wrote.GetName() == restoreTempSystemDB {
updatedPrivileges = wrote.GetPrivileges()
}
} else {
} else if descCoverage == tree.RequestedDescriptors {
parentDB, err := catalogkv.MustGetDatabaseDescByID(ctx, txn, codec, desc.GetParentID())
if err != nil {
return nil, errors.Wrapf(err,
"failed to lookup parent DB %d", errors.Safe(desc.GetParentID()))
return nil, errors.Wrapf(err, "failed to lookup parent DB %d", errors.Safe(desc.GetParentID()))
}

// Default is to copy privs from restoring parent db, like CREATE {TABLE,
Expand All @@ -2147,10 +2129,12 @@ func getRestoringPrivileges(
updatedPrivileges = sql.CreateInheritedPrivilegesFromDBDesc(parentDB, user)
}
case catalog.TypeDescriptor, catalog.DatabaseDescriptor:
// If the restore is not a 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 type/database.
updatedPrivileges = descpb.NewDefaultPrivilegeDescriptor(user)
if descCoverage == tree.RequestedDescriptors {
// If the restore is not a 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 type/database.
updatedPrivileges = descpb.NewDefaultPrivilegeDescriptor(user)
}
}
return updatedPrivileges, nil
}
Expand Down
Loading

0 comments on commit 15ee387

Please sign in to comment.