Skip to content

Commit

Permalink
Merge pull request #108488 from rafiss/backport22.2-108347
Browse files Browse the repository at this point in the history
release-22.2: catalog: use NewReferencedDescriptorNotFoundError
  • Loading branch information
rafiss authored Aug 10, 2023
2 parents 27c5406 + 932bf15 commit 75768e5
Show file tree
Hide file tree
Showing 13 changed files with 36 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ ALTER TABLE t2 ADD COLUMN new_col INT
query idx=2
SELECT * FROM t2 AS OF SYSTEM TIME with_min_timestamp(now() - '10s', true) WHERE pk = 2
----
pq: referenced descriptor ID 105: descriptor not found
pq: referenced descriptor ID 105: looking up ID 105: descriptor not found
events (7 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
Expand All @@ -257,7 +257,7 @@ events (7 found):
query idx=2
SELECT * FROM t2 AS OF SYSTEM TIME with_min_timestamp(now() - '10s', true) WHERE pk = 2
----
pq: referenced descriptor ID 105: descriptor not found
pq: referenced descriptor ID 105: looking up ID 105: descriptor not found
events (7 found):
* event 1: colbatchscan trace on node_idx 2: local read
* event 2: transaction retry on node_idx: 2
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/test_examine_zipdir
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ Examining 37 descriptors and 42 namespace entries...
ParentID 52, ParentSchemaID 29: relation "vehicle_location_histories" (56): referenced database ID 52: referenced descriptor not found
ParentID 52, ParentSchemaID 29: relation "promo_codes" (57): referenced database ID 52: referenced descriptor not found
ParentID 52, ParentSchemaID 29: relation "user_promo_codes" (58): referenced database ID 52: referenced descriptor not found
ParentID 0, ParentSchemaID 0: namespace entry "movr" (52): descriptor not found
ParentID 0, ParentSchemaID 0: namespace entry "movr" (52): referenced schema ID 52: referenced descriptor not found
Examining 2 jobs...
job 587337426984566785: running schema change GC refers to missing table descriptor(s) [59]; existing descriptors that still need to be dropped []; job safe to delete: true.
ERROR: validation failed
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/test_examine_zipdir_verbose
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ Examining 37 descriptors and 42 namespace entries...
ParentID 52, ParentSchemaID 29: relation "user_promo_codes" (58): referenced database ID 52: referenced descriptor not found
ParentID 52, ParentSchemaID 29: relation "user_promo_codes" (58): processed
ParentID 0, ParentSchemaID 0: namespace entry "defaultdb" (50): processed
ParentID 0, ParentSchemaID 0: namespace entry "movr" (52): descriptor not found
ParentID 0, ParentSchemaID 0: namespace entry "movr" (52): referenced schema ID 52: referenced descriptor not found
ParentID 0, ParentSchemaID 0: namespace entry "postgres" (51): processed
ParentID 0, ParentSchemaID 0: namespace entry "system" (1): processed
ParentID 1, ParentSchemaID 0: namespace entry "public" (29): processed
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,7 @@ func (q *byIDLookupContext) lookupLeased(
// If we have already read all of the descriptors, use it as a negative
// cache to short-circuit a lookup we know will be doomed to fail.
if q.tc.stored.IsIDKnownToNotExist(id, q.flags.ParentID) {
return nil, catalog.NoValidation, catalog.ErrDescriptorNotFound
return nil, catalog.NoValidation, catalog.NewDescriptorNotFoundError(id)
}
desc, shouldReadFromStore, err := q.tc.leased.getByID(q.ctx, q.tc.deadlineHolder(q.txn), id)
if err != nil || shouldReadFromStore {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/hydrate.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ func HydrateCatalog(ctx context.Context, c nstree.MutableCatalog) error {
defer sp.Finish()

fakeLookupFunc := func(_ context.Context, id descpb.ID, skipHydration bool) (catalog.Descriptor, error) {
return nil, catalog.WrapDescRefErr(id, catalog.ErrDescriptorNotFound)
return nil, catalog.NewDescriptorNotFoundError(id)
}
typeLookupFunc := hydrateddesc.MakeTypeLookupFuncForHydration(c, fakeLookupFunc)
return c.ForEachDescriptorEntry(func(desc catalog.Descriptor) error {
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/catalog/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// ValidateName validates a name.
Expand Down Expand Up @@ -82,10 +83,18 @@ func HasInactiveDescriptorError(err error) bool {
// found with the given id.
var ErrDescriptorNotFound = errors.New("descriptor not found")

func NewDescriptorNotFoundError(id descpb.ID) error {
return errors.Wrapf(ErrDescriptorNotFound, "looking up ID %d", errors.Safe(id))
}

// ErrReferencedDescriptorNotFound is like ErrDescriptorNotFound but for
// descriptors referenced within another descriptor.
var ErrReferencedDescriptorNotFound = errors.New("referenced descriptor not found")

func NewReferencedDescriptorNotFoundError(descType string, id descpb.ID) error {
return errors.Wrapf(ErrReferencedDescriptorNotFound, "referenced %s ID %d", redact.SafeString(descType), errors.Safe(id))
}

// ErrDescriptorWrongType is returned to signal that a descriptor was found but
// that it wasn't of the expected type.
var ErrDescriptorWrongType = errors.New("unexpected descriptor type")
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/internal/catkv/catalog_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,5 +171,5 @@ func requiredError(expectedType catalog.DescriptorType, id descpb.ID) (err error
default:
err = errors.Errorf("failed to find descriptor [%d]", id)
}
return errors.CombineErrors(catalog.ErrDescriptorNotFound, err)
return errors.CombineErrors(catalog.NewDescriptorNotFoundError(id), err)
}
12 changes: 6 additions & 6 deletions pkg/sql/catalog/internal/validate/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -294,7 +294,7 @@ var _ catalog.ValidationDescGetter = (*validationDescGetterImpl)(nil)
func (vdg *validationDescGetterImpl) GetDescriptor(id descpb.ID) (catalog.Descriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("descriptor", id)
}
return desc, nil
}
Expand All @@ -305,7 +305,7 @@ func (vdg *validationDescGetterImpl) GetDatabaseDescriptor(
) (catalog.DatabaseDescriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapDatabaseDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("database", id)
}
return catalog.AsDatabaseDescriptor(desc)
}
Expand All @@ -316,7 +316,7 @@ func (vdg *validationDescGetterImpl) GetSchemaDescriptor(
) (catalog.SchemaDescriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapSchemaDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("schema", id)
}
return catalog.AsSchemaDescriptor(desc)
}
Expand All @@ -327,7 +327,7 @@ func (vdg *validationDescGetterImpl) GetTableDescriptor(
) (catalog.TableDescriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapTableDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("table", id)
}
return catalog.AsTableDescriptor(desc)
}
Expand All @@ -338,7 +338,7 @@ func (vdg *validationDescGetterImpl) GetTypeDescriptor(
) (catalog.TypeDescriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapTypeDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("type", id)
}
descriptor, err := catalog.AsTypeDescriptor(desc)
if err != nil {
Expand All @@ -352,7 +352,7 @@ func (vdg *validationDescGetterImpl) GetFunctionDescriptor(
) (catalog.FunctionDescriptor, error) {
desc, found := vdg.descriptors[id]
if !found || desc == nil {
return nil, catalog.WrapFunctionDescRefErr(id, catalog.ErrReferencedDescriptorNotFound)
return nil, catalog.NewReferencedDescriptorNotFoundError("function", id)
}
return catalog.AsFunctionDescriptor(desc)
}
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/catalog/lease/lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -890,7 +890,7 @@ func (m *Manager) AcquireByName(
// If the name we had doesn't match the newest descriptor in the DB, then
// we're trying to use an old name.
desc.Release(ctx)
return nil, catalog.ErrDescriptorNotFound
return nil, catalog.NewDescriptorNotFoundError(id)
}
}
return validateDescriptorForReturn(desc)
Expand Down Expand Up @@ -926,7 +926,10 @@ func (m *Manager) resolveName(
return id, err
}
if id == descpb.InvalidID {
return id, catalog.ErrDescriptorNotFound
return id, errors.Wrapf(catalog.ErrDescriptorNotFound,
"resolving name %s with parentID %d and parentSchemaID %d",
name, parentID, parentSchemaID,
)
}
return id, nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/nstree/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ func (c Catalog) ValidateNamespaceEntry(key catalog.NameKey) error {
// Compare the namespace entry with the referenced descriptor.
desc := c.LookupDescriptorEntry(ne.GetID())
if desc == nil {
return catalog.ErrDescriptorNotFound
return catalog.NewReferencedDescriptorNotFoundError("schema", ne.GetID())
}
if desc.Dropped() {
return errors.Newf("no matching name info in draining names of dropped %s",
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/crdb_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,9 +494,9 @@ UPDATE system.namespace SET id = %d WHERE id = %d;
{fmt.Sprintf("%d", schemaID), fmt.Sprintf("[%d]", databaseID), "public", "",
fmt.Sprintf(`schema "public" (%d): referenced database ID %d: referenced descriptor not found`, schemaID, databaseID),
},
{fmt.Sprintf("%d", databaseID), "t", "", "", `descriptor not found`},
{fmt.Sprintf("%d", tableFkTblID), "defaultdb", "public", "fktbl", `descriptor not found`},
{fmt.Sprintf("%d", fakeID), fmt.Sprintf("[%d]", databaseID), "public", "test", `descriptor not found`},
{fmt.Sprintf("%d", databaseID), "t", "", "", `referenced schema ID 104: referenced descriptor not found`},
{fmt.Sprintf("%d", tableFkTblID), "defaultdb", "public", "fktbl", `referenced schema ID 107: referenced descriptor not found`},
{fmt.Sprintf("%d", fakeID), fmt.Sprintf("[%d]", databaseID), "public", "test", `referenced schema ID 12345: referenced descriptor not found`},
})
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/doctor/doctor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -372,7 +372,7 @@ func TestExamineDescriptors(t *testing.T) {
{NameInfo: descpb.NameInfo{Name: "causes_error"}, ID: 2},
},
expected: `Examining 0 descriptors and 4 namespace entries...
ParentID 0, ParentSchemaID 0: namespace entry "causes_error" (2): descriptor not found
ParentID 0, ParentSchemaID 0: namespace entry "causes_error" (2): referenced schema ID 2: referenced descriptor not found
`,
},
{ // 14
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -1085,10 +1085,10 @@ query ITTTT colnames
SELECT * FROM "".crdb_internal.invalid_objects
----
id database_name schema_name obj_name error
500 baddb · · descriptor not found
501 system badschema · descriptor not found
502 system public badobj descriptor not found
503 system [404] badobj descriptor not found
500 baddb · · referenced schema ID 500: referenced descriptor not found
501 system badschema · referenced schema ID 501: referenced descriptor not found
502 system public badobj referenced schema ID 502: referenced descriptor not found
503 system [404] badobj referenced schema ID 503: referenced descriptor not found

statement ok
SELECT crdb_internal.unsafe_delete_namespace_entry(0, 0, 'baddb', 500, true);
Expand Down

0 comments on commit 75768e5

Please sign in to comment.