Skip to content

Commit

Permalink
sql: descriptor validation overhaul
Browse files Browse the repository at this point in the history
Previously, descriptor validation suffered from a few shortcomings:
1. The methods to call were inconsistent across descriptor types, and
   the error messages they return are inconsistent as well.
2. Those methods made ad-hoc use of catalog.DescGetter to walk through
   the descriptor reference graph, and could sometimes walk surprisingly
   far. This is particularly true for type descriptors, which hold
   references to other types.
3. To complicate things further, DescGetter implementations which read
   descriptors from storage themselves perform validation as well.
   Although it is perfectly sensible to systematically validate
   descriptors when read, there is a circularity problem at hand which
   would benefit from being solved without melting the maintainer's
   brain in the process.
4. The validation methods return an error type, making it awkward to
   return multiple errors. Tools like doctor would be made more useful
   if they could report more than only the first encountered error.

Recently we introduced a change that added descriptor validation on
write. This change involves validating all uncommitted descriptors in
bulk and this in turn favours a bulk approach to reading all their
referenced descriptors from storage.

With this in mind, this commit adds a GetReferencedDescIDs method to
catalog.Descriptor which returns the IDs of all descriptors referenced
by the receiver. By calling this method on all descriptors we intend to
validate and by reading all referenced descriptors in one kv.Batch and
stuffing them in a catalog.MapDescGetter, we can now validate using this
in-memory DescGetter instead.

Turning validation into this multi-step process means it can no longer
be invoked as a method call on the target descriptor. This commit moves
the entry point to descriptor validation to the catalog.Validate
function. The descriptor objects themselves still define the validation
checks themselves but these are effectively made inaccessible from
outside the catalog package. The rationale behind this is to enforce
order in the sequence of checks performed and to enforce some uniformity
in the formatting of the error messages. The checks are grouped into
three tiers: internal consistency checks, cross-reference consistency
checks, and transactional consistency checks.

All this also helps reduce the number of call sites of descriptor
validations as well as increase the scope of the validations. This has
uncovered a few bugs related to schemas and temporary tables. This
effort has also helped identify validation checks which have since been
made redundant, either by other existing rules or by the new
validation-on-write behaviour. Finally, this has allowed us to strip
doctor of its own duplicated (and dodgy) validation logic and simply
report all validation errors instead.

Release justification: This commit is safe for this release because it
consists of bug fixes and changes involving descriptor validation. If
anything these changes will help uncover descriptor corruptions which
were previously unnoticed, both because we're extending the validation
suite and because the doctor tool produces richer output.

Release note (cli change): The doctor tool can now report multiple
descriptor validation failures per descriptor.
  • Loading branch information
Marius Posta committed Feb 24, 2021
1 parent 3bae09b commit d443b82
Show file tree
Hide file tree
Showing 70 changed files with 4,174 additions and 3,609 deletions.
3 changes: 0 additions & 3 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,9 +128,6 @@ type TestServerArgs struct {

// IF set, the demo login endpoint will be enabled.
EnableDemoLoginEndpoint bool

// If set, testing specific descriptor validation will be disabled. even if the server
DisableTestingDescriptorValidation bool
}

// TestClusterArgs contains the parameters one can set when creating a test
Expand Down
19 changes: 6 additions & 13 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,23 +433,16 @@ func WriteDescriptors(
}
return err
}
// TODO(ajwerner): Utilize validation inside of the descs.Collection
// rather than reaching into the store.
dg := catalogkv.NewOneLevelUncachedDescGetter(txn, codec)

bdg := catalogkv.NewOneLevelUncachedDescGetter(txn, codec)
descs := make([]catalog.Descriptor, 0, len(databases)+len(tables))
for _, table := range tables {
if err := table.Validate(ctx, dg); err != nil {
return errors.Wrapf(err,
"validate table %d", errors.Safe(table.GetID()))
}
descs = append(descs, table)
}

for _, db := range databases {
if err := db.Validate(ctx, dg); err != nil {
return errors.Wrapf(err,
"validate database %d", errors.Safe(db.GetID()))
}
descs = append(descs, db)
}
return nil
return catalog.ValidateSelfAndCrossReferences(ctx, bdg, descs...)
}()
return errors.Wrapf(err, "restoring table desc and namespace entries")
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/avro_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) {
if err != nil {
return nil, err
}
return mutDesc, mutDesc.ValidateSelf(ctx)
return mutDesc, catalog.ValidateSelf(mutDesc)
}

func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.EncDatumRow, error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/multi_region
Original file line number Diff line number Diff line change
Expand Up @@ -911,7 +911,7 @@ SHOW ENUMS FROM drop_region_db
schema name values owner
public crdb_internal_region {ap-southeast-2,ca-central-1} root

statement error pq: region "us-east-1" has not been added to database "drop_region_db"
statement error pq: relation "t" \([0-9]+\): invalid locality config: region "us-east-1" has not been added to database "drop_region_db"
CREATE TABLE drop_region_db.public.t(a int) LOCALITY REGIONAL BY TABLE IN "us-east-1"

statement ok
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ go_test(
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/tabledesc",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
Expand Down Expand Up @@ -139,7 +140,7 @@ func (pt *partitioningTest) parse() error {
return err
}
pt.parsed.tableDesc = mutDesc
if err := pt.parsed.tableDesc.ValidateSelf(ctx); err != nil {
if err := catalog.ValidateSelf(pt.parsed.tableDesc); err != nil {
return err
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/testcluster
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,6 @@ doctor cluster
----
debug doctor cluster
Examining 35 descriptors and 36 namespace entries...
Table 53: ParentID 50, ParentSchemaID 29, Name 'foo': not being dropped but no namespace entry found
ParentID 50, ParentSchemaID 29: relation "foo" (53): not being dropped but no namespace entry found
Examining 1 running jobs...
ERROR: validation failed
12 changes: 6 additions & 6 deletions pkg/cli/testdata/doctor/testzipdir
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,12 @@ doctor zipdir
----
debug doctor zipdir testdata/doctor/debugzip
Examining 38 descriptors and 43 namespace entries...
Table 53: ParentID 52, ParentSchemaID 29, Name 'users': desc 53: parentID 52 does not exist
Table 54: ParentID 52, ParentSchemaID 29, Name 'vehicles': desc 54: parentID 52 does not exist
Table 55: ParentID 52, ParentSchemaID 29, Name 'rides': desc 55: parentID 52 does not exist
Table 56: ParentID 52, ParentSchemaID 29, Name 'vehicle_location_histories': desc 56: parentID 52 does not exist
Table 57: ParentID 52, ParentSchemaID 29, Name 'promo_codes': desc 57: parentID 52 does not exist
Table 58: ParentID 52, ParentSchemaID 29, Name 'user_promo_codes': desc 58: parentID 52 does not exist
ParentID 52, ParentSchemaID 29: relation "users" (53): referenced database descriptor 52 not found
ParentID 52, ParentSchemaID 29: relation "vehicles" (54): referenced database descriptor 52 not found
ParentID 52, ParentSchemaID 29: relation "rides" (55): referenced database descriptor 52 not found
ParentID 52, ParentSchemaID 29: relation "vehicle_location_histories" (56): referenced database descriptor 52 not found
ParentID 52, ParentSchemaID 29: relation "promo_codes" (57): referenced database descriptor 52 not found
ParentID 52, ParentSchemaID 29: relation "user_promo_codes" (58): referenced database descriptor 52 not found
Descriptor 52: has namespace row(s) [{ParentID:0 ParentSchemaID:0 Name:movr}] but no descriptor
Examining 1 running jobs...
job 587337426984566785: schema change GC refers to missing table descriptor(s) [59]
Expand Down
3 changes: 0 additions & 3 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -273,9 +273,6 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config {
if params.Knobs.SQLExecutor == nil {
cfg.TestingKnobs.SQLExecutor = &sql.ExecutorTestingKnobs{}
}
if !params.DisableTestingDescriptorValidation {
cfg.TestingKnobs.SQLExecutor.(*sql.ExecutorTestingKnobs).TestingDescriptorValidation = true
}

// For test servers, leave interleaved tables enabled by default. We'll remove
// this when we remove interleaved tables altogether.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_column_type_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ ALTER TABLE t.test ALTER COLUMN x TYPE STRING;

<-childJobStartNotification

expected := "pq: unimplemented: cannot perform a schema change operation while an ALTER COLUMN TYPE schema change is in progress"
expected := `pq: relation "test" \(53\): unimplemented: cannot perform a schema change operation while an ALTER COLUMN TYPE schema change is in progress`
sqlDB.ExpectErr(t, expected, `
ALTER TABLE t.test ADD COLUMN y INT;
`)
Expand Down
10 changes: 1 addition & 9 deletions pkg/sql/alter_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ 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/typedesc"
Expand Down Expand Up @@ -222,8 +221,7 @@ func (n *alterDatabaseAddRegionNode) startExec(params runParams) error {

// Validate the type descriptor after the changes. We have to do this explicitly here, because
// we're using an internal call to addEnumValue above which doesn't perform validation.
dg := catalogkv.NewOneLevelUncachedDescGetter(params.p.txn, params.ExecCfg().Codec)
if err := typeDesc.Validate(params.ctx, dg); err != nil {
if err := validateDescriptor(params.ctx, params.p, typeDesc); err != nil {
return err
}

Expand Down Expand Up @@ -536,12 +534,6 @@ func (n *alterDatabasePrimaryRegionNode) switchPrimaryRegion(params runParams) e
return err
}

// Validate the type descriptor after the changes.
dg := catalogkv.NewOneLevelUncachedDescGetter(params.p.txn, params.ExecCfg().Codec)
if err := typeDesc.Validate(params.ctx, dg); err != nil {
return err
}

// Update the database's zone configuration.
if err := ApplyZoneConfigFromDatabaseRegionConfig(
params.ctx,
Expand Down
22 changes: 9 additions & 13 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"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/catalog/resolver"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
Expand Down Expand Up @@ -281,7 +280,7 @@ func (n *alterTableNode) startExec(params runParams) error {
case *tree.CheckConstraintTableDef:
var err error
params.p.runWithOptions(resolveFlags{contextDatabaseID: n.tableDesc.ParentID}, func() {
info, infoErr := n.tableDesc.GetConstraintInfo(params.ctx, nil)
info, infoErr := n.tableDesc.GetConstraintInfo()
if infoErr != nil {
err = infoErr
return
Expand Down Expand Up @@ -663,14 +662,13 @@ func (n *alterTableNode) startExec(params runParams) error {
return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState,
"column %q in the middle of being added, try again later", t.Column)
}
if err := n.tableDesc.Validate(
params.ctx, catalogkv.NewOneLevelUncachedDescGetter(params.p.Txn(), params.ExecCfg().Codec),
); err != nil {

if err := validateDescriptor(params.ctx, params.p, n.tableDesc); err != nil {
return err
}

case *tree.AlterTableDropConstraint:
info, err := n.tableDesc.GetConstraintInfo(params.ctx, nil)
info, err := n.tableDesc.GetConstraintInfo()
if err != nil {
return err
}
Expand All @@ -692,14 +690,12 @@ func (n *alterTableNode) startExec(params runParams) error {
return err
}
descriptorChanged = true
if err := n.tableDesc.Validate(
params.ctx, catalogkv.NewOneLevelUncachedDescGetter(params.p.Txn(), params.ExecCfg().Codec),
); err != nil {
if err := validateDescriptor(params.ctx, params.p, n.tableDesc); err != nil {
return err
}

case *tree.AlterTableValidateConstraint:
info, err := n.tableDesc.GetConstraintInfo(params.ctx, nil)
info, err := n.tableDesc.GetConstraintInfo()
if err != nil {
return err
}
Expand Down Expand Up @@ -889,7 +885,7 @@ func (n *alterTableNode) startExec(params runParams) error {
descriptorChanged = descriptorChanged || descChanged

case *tree.AlterTableRenameConstraint:
info, err := n.tableDesc.GetConstraintInfo(params.ctx, nil)
info, err := n.tableDesc.GetConstraintInfo()
if err != nil {
return err
}
Expand Down Expand Up @@ -1106,7 +1102,7 @@ func applyColumnMutation(
}
}

info, err := tableDesc.GetConstraintInfo(params.ctx, nil)
info, err := tableDesc.GetConstraintInfo()
if err != nil {
return err
}
Expand Down Expand Up @@ -1141,7 +1137,7 @@ func applyColumnMutation(
"constraint in the middle of being dropped")
}
}
info, err := tableDesc.GetConstraintInfo(params.ctx, nil)
info, err := tableDesc.GetConstraintInfo()
if err != nil {
return err
}
Expand Down
23 changes: 6 additions & 17 deletions pkg/sql/alter_table_locality.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -135,7 +134,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityGlobalToRegionalByTable(

// Finalize the alter by writing a new table descriptor and updating the zone
// configuration.
if err := n.validateAndWriteNewTableLocalityAndZoneConfig(
if err := n.writeNewTableLocalityAndZoneConfig(
params,
n.dbDesc,
); err != nil {
Expand Down Expand Up @@ -167,7 +166,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityRegionalByTableToGlobal(

// Finalize the alter by writing a new table descriptor and updating the zone
// configuration.
if err := n.validateAndWriteNewTableLocalityAndZoneConfig(
if err := n.writeNewTableLocalityAndZoneConfig(
params,
n.dbDesc,
); err != nil {
Expand Down Expand Up @@ -208,7 +207,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityRegionalByTableToRegionalB
}

// Finalize the alter by writing a new table descriptor and updating the zone configuration.
if err := n.validateAndWriteNewTableLocalityAndZoneConfig(
if err := n.writeNewTableLocalityAndZoneConfig(
params,
n.dbDesc,
); err != nil {
Expand Down Expand Up @@ -539,21 +538,11 @@ func (n *alterTableSetLocalityNode) startExec(params runParams) error {
})
}

// validateAndWriteNewTableLocalityAndZoneConfig validates the newly updated
// LocalityConfig in a table descriptor, writes that table descriptor, and
// writes a new zone configuration for the given table.
func (n *alterTableSetLocalityNode) validateAndWriteNewTableLocalityAndZoneConfig(
// writeNewTableLocalityAndZoneConfig writes the table descriptor with the newly
// updated LocalityConfig and writes a new zone configuration for the table.
func (n *alterTableSetLocalityNode) writeNewTableLocalityAndZoneConfig(
params runParams, dbDesc *dbdesc.Immutable,
) error {
// Validate the new locality before updating the table descriptor.
dg := catalogkv.NewOneLevelUncachedDescGetter(params.p.txn, params.EvalContext().Codec)
if err := n.tableDesc.ValidateTableLocalityConfig(
params.ctx,
dg,
); err != nil {
return err
}

// Write out the table descriptor update.
if err := params.p.writeSchemaChange(
params.ctx,
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/alter_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,12 +126,6 @@ func (n *alterTypeNode) startExec(params runParams) error {
return err
}

// Validate the type descriptor after the changes.
dg := catalogkv.NewOneLevelUncachedDescGetter(params.p.txn, params.ExecCfg().Codec)
if err := n.desc.Validate(params.ctx, dg); err != nil {
return err
}

if !eventLogDone {
// Write a log event.
if err := params.p.logEvent(params.ctx,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"errors.go",
"table_col_map.go",
"table_col_set.go",
"validate.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog",
visibility = ["//visibility:public"],
Expand Down
44 changes: 21 additions & 23 deletions pkg/sql/catalog/catalogkv/catalogkv.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,7 @@ func requiredError(kind DescriptorKind, id descpb.ID) error {
// Txn. It will use the transaction to resolve mutable descriptors using
// GetDescriptorByID but will pass a nil DescGetter into those lookup calls to
// ensure that the entire graph of dependencies is not traversed.
func NewOneLevelUncachedDescGetter(txn *kv.Txn, codec keys.SQLCodec) catalog.DescGetter {
func NewOneLevelUncachedDescGetter(txn *kv.Txn, codec keys.SQLCodec) catalog.BatchDescGetter {
return &oneLevelUncachedDescGetter{
txn: txn,
codec: codec,
Expand Down Expand Up @@ -312,7 +312,11 @@ func unwrapDescriptor(
return nil, nil
}
if validate {
if err := unwrapped.Validate(ctx, dg); err != nil {
var level catalog.ValidationLevel
if dg != nil {
level = catalog.ValidationLevelSelfAndCrossReferences
}
if err := catalog.Validate(ctx, dg, level, unwrapped).CombinedError(); err != nil {
return nil, err
}
}
Expand All @@ -329,29 +333,25 @@ func unwrapDescriptorMutable(
table, database, typ, schema :=
descpb.TableFromDescriptor(desc, hlc.Timestamp{}),
desc.GetDatabase(), desc.GetType(), desc.GetSchema()
var err error
var mut catalog.MutableDescriptor
switch {
case table != nil:
mutTable, err := tabledesc.NewFilledInExistingMutable(ctx, dg, false /* skipFKsWithMissingTable */, table)
if err != nil {
return nil, err
}
if err := mutTable.ValidateSelf(ctx); err != nil {
return nil, err
}
return mutTable, nil
mut, err = tabledesc.NewFilledInExistingMutable(ctx, dg, false /* skipFKsWithMissingTable */, table)
case database != nil:
dbDesc := dbdesc.NewExistingMutable(*database)
if err := dbDesc.Validate(ctx, dg); err != nil {
return nil, err
}
return dbDesc, nil
mut, err = dbdesc.NewExistingMutable(*database), nil
case typ != nil:
return typedesc.NewExistingMutable(*typ), nil
mut, err = typedesc.NewExistingMutable(*typ), nil
case schema != nil:
return schemadesc.NewMutableExisting(*schema), nil
default:
return nil, nil
mut, err = schemadesc.NewMutableExisting(*schema), nil
}
if mut != nil && err == nil {
err = catalog.ValidateSelf(mut)
}
if err != nil {
return nil, err
}
return mut, nil
}

// CountUserDescriptors returns the number of descriptors present that were
Expand Down Expand Up @@ -416,10 +416,8 @@ func GetAllDescriptors(
for _, desc := range descs {
dg[desc.GetID()] = desc
}
for _, desc := range descs {
if err := desc.Validate(ctx, dg); err != nil {
return nil, err
}
if err := catalog.ValidateSelfAndCrossReferences(ctx, dg, descs...); err != nil {
return nil, err
}
return descs, nil
}
Expand Down
Loading

0 comments on commit d443b82

Please sign in to comment.