From cf8b698b2290acf82e70d398c67c07cf403cd6a4 Mon Sep 17 00:00:00 2001 From: ajwerner Date: Mon, 27 Mar 2023 11:12:57 -0400 Subject: [PATCH 1/3] sql/catalog: add String and SafeFormat methods to DescriptorIDSet Epic: none Release note: None --- pkg/sql/catalog/BUILD.bazel | 1 + pkg/sql/catalog/descriptor_id_set.go | 14 ++++++++++++++ 2 files changed, 15 insertions(+) diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 257de924276d..0e8c681059c8 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/util/iterutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", + "@com_github_cockroachdb_redact//interfaces", ], ) diff --git a/pkg/sql/catalog/descriptor_id_set.go b/pkg/sql/catalog/descriptor_id_set.go index 07eb0cac6198..ed56cc96fa48 100644 --- a/pkg/sql/catalog/descriptor_id_set.go +++ b/pkg/sql/catalog/descriptor_id_set.go @@ -13,6 +13,8 @@ package catalog import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/intsets" + "github.com/cockroachdb/redact" + "github.com/cockroachdb/redact/interfaces" ) // DescriptorIDSet efficiently stores an unordered set of descriptor ids. @@ -20,6 +22,16 @@ type DescriptorIDSet struct { set intsets.Fast } +// SafeFormat implements SafeFormatter for DescriptorIDSet. +func (d *DescriptorIDSet) SafeFormat(s interfaces.SafePrinter, verb rune) { + s.SafeString(redact.SafeString(d.String())) +} + +// String implement fmt.Stringer for DescriptorIDSet. +func (d *DescriptorIDSet) String() string { + return d.set.String() +} + // MakeDescriptorIDSet returns a set initialized with the given values. func MakeDescriptorIDSet(ids ...descpb.ID) DescriptorIDSet { s := DescriptorIDSet{} @@ -32,6 +44,8 @@ func MakeDescriptorIDSet(ids ...descpb.ID) DescriptorIDSet { // Suppress the linter. var _ = MakeDescriptorIDSet +var _ redact.SafeFormatter = (*DescriptorIDSet)(nil) + // Add adds an id to the set. No-op if the id is already in the set. func (d *DescriptorIDSet) Add(id descpb.ID) { d.set.Add(int(id)) From 184666c84443145db35f6520f5df57896204e161 Mon Sep 17 00:00:00 2001 From: ajwerner Date: Mon, 27 Mar 2023 11:14:15 -0400 Subject: [PATCH 2/3] sql: do not drop table descriptor independently if we're in drop schema If we have dropped schema IDs, we know that this is not an individual drop table schema change. We only have more than one dropped table when we drop a database or a schema. Before this change, we'd drop the table on its own, and then create another GC job to drop all the tables. This is not actually a bug because we should be robust to this, but it's also bad. Epic: none Release note (bug fix): DROP SCHEMA ... CASCADE could create multiple GC jobs: one for every table and one for the cascaded drop itself. This has been fixed. --- pkg/sql/schema_changer.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 8d0e3114248d..14899c474b56 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -544,6 +544,8 @@ func (sc *SchemaChanger) execLogTags() *logtags.Buffer { } if sc.droppedDatabaseID != descpb.InvalidID { buf = buf.Add("db", sc.droppedDatabaseID) + } else if !sc.droppedSchemaIDs.Empty() { + buf = buf.Add("schema", sc.droppedSchemaIDs) } return buf } @@ -720,7 +722,7 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { } // Otherwise, continue with the rest of the schema change state machine. - if tableDesc.Dropped() && sc.droppedDatabaseID == descpb.InvalidID { + if tableDesc.Dropped() && sc.droppedDatabaseID == descpb.InvalidID && sc.droppedSchemaIDs.Empty() { if tableDesc.IsPhysicalTable() { // We've dropped this physical table, let's kick off a GC job. dropTime := timeutil.Now().UnixNano() From 667c45a2e60287932b5716075ab3afecb7da8d54 Mon Sep 17 00:00:00 2001 From: ajwerner Date: Mon, 27 Mar 2023 11:25:02 -0400 Subject: [PATCH 3/3] sql/gc_job,sqlerrors: make GC job robust to missing descriptors The check used for missing descriptors became incorrect in the course of https://github.com/cockroachdb/cockroach/pull/94695. That change updated the underlying error code used in getters by the GC job. The GC job would subsequently retry forever when the descriptor was missing. This bug has not been shipped yet, so not writing a release note. Fixes: #99590 Release note: None --- pkg/sql/gcjob/gc_job.go | 14 +++++++- pkg/sql/gcjob/index_garbage_collection.go | 11 +++---- pkg/sql/gcjob/refresh_statuses.go | 2 +- pkg/sql/gcjob/table_garbage_collection.go | 4 +-- pkg/sql/gcjob_test/gc_job_test.go | 40 +++++++++++++++-------- pkg/sql/sem/builtins/BUILD.bazel | 1 + pkg/sql/sem/builtins/pg_builtins.go | 9 +++-- pkg/sql/sqlerrors/errors.go | 15 +++++++++ 8 files changed, 66 insertions(+), 30 deletions(-) diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index 8e5fdee0c36b..f6ae37d73646 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -137,7 +138,7 @@ func deleteTableData( table, err = col.ByID(txn.KV()).Get().Table(ctx, droppedTable.ID) return err }); err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { // This can happen if another GC job created for the same table got to // the table first. See #50344. log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) @@ -566,6 +567,17 @@ func waitForWork( return ctx.Err() } +// isMissingDescriptorError checks whether the error has a code corresponding +// to a missing descriptor or if there is a lower-level catalog error with +// the same meaning. +// +// TODO(ajwerner,postamar): Nail down when we expect the lower-level error +// and tighten up the collection. +func isMissingDescriptorError(err error) bool { + return errors.Is(err, catalog.ErrDescriptorNotFound) || + sqlerrors.IsMissingDescriptorError(err) +} + // OnFailOrCancel is part of the jobs.Resumer interface. func (r schemaChangeGCResumer) OnFailOrCancel(context.Context, interface{}, error) error { return nil diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index cb64c80b5aa8..7c195bcc58cf 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/isql" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -45,7 +44,7 @@ func deleteIndexData( // are no longer in use. This is necessary in the case of truncate, where we // schedule a GC Job in the transaction that commits the truncation. parentDesc, err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, parentID) - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { handleTableDescriptorDeleted(ctx, parentID, progress) return nil } @@ -94,7 +93,7 @@ func gcIndexes( // are no longer in use. This is necessary in the case of truncate, where we // schedule a GC Job in the transaction that commits the truncation. parentDesc, err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, parentID) - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { handleTableDescriptorDeleted(ctx, parentID, progress) return nil } @@ -131,8 +130,7 @@ func gcIndexes( ) } err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs) - if errors.Is(err, catalog.ErrDescriptorNotFound) || - sqlerrors.IsUndefinedRelationError(err) { + if isMissingDescriptorError(err) { handleTableDescriptorDeleted(ctx, parentID, progress) return nil } @@ -213,8 +211,7 @@ func deleteIndexZoneConfigsAfterGC( } err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs) switch { - case errors.Is(err, catalog.ErrDescriptorNotFound), - sqlerrors.IsUndefinedRelationError(err): + case isMissingDescriptorError(err): log.Infof(ctx, "removing index %d zone config from table %d failed: %v", index.IndexID, parentID, err) case err != nil: diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index 2b9549a54481..350003be1c05 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -138,7 +138,7 @@ func updateStatusForGCElements( return nil }); err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { log.Warningf(ctx, "table %d not found, marking as GC'd", tableID) markTableGCed(ctx, tableID, progress, jobspb.SchemaChangeGCProgress_CLEARED) return false, true, maxDeadline diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 2edbf2bdb627..b9edd5fed38f 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -54,7 +54,7 @@ func gcTables( table, err = col.ByID(txn.KV()).Get().Table(ctx, droppedTable.ID) return err }); err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { // This can happen if another GC job created for the same table got to // the table first. See #50344. log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) @@ -293,7 +293,7 @@ func deleteTableDescriptorsAfterGC( table, err = col.ByID(txn.KV()).Get().Table(ctx, droppedTable.ID) return err }); err != nil { - if errors.Is(err, catalog.ErrDescriptorNotFound) { + if isMissingDescriptorError(err) { // This can happen if another GC job created for the same table got to // the table first. See #50344. log.Warningf(ctx, "table descriptor %d not found while attempting to GC, skipping", droppedTable.ID) diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 643128618946..768bac9fcec5 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -542,23 +542,15 @@ func TestGCTenant(t *testing.T) { }) } -// This test exercises code whereby an index GC job is running, and, in the +// This test exercises code whereby an GC job is running, and, in the // meantime, the descriptor is removed. We want to ensure that the GC job -// finishes without an error. -func TestDropIndexWithDroppedDescriptor(t *testing.T) { +// finishes without an error. We want to test this both for index drops +// and for table drops. +func TestDropWithDeletedDescriptor(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // The way the GC job works is that it initially clears the index - // data, then it waits for the background MVCC GC to run and remove - // the underlying tombstone, and then finally it removes any relevant - // zone configurations for the index from system.zones. In the first - // and final phases, the job resolves the descriptor. This test ensures - // that the code is robust to the descriptor being removed both before - // the initial DelRange, and after, when going to remove the zone config. - testutils.RunTrueAndFalse(t, "before DelRange", func( - t *testing.T, beforeDelRange bool, - ) { + runTest := func(t *testing.T, dropIndex bool, beforeDelRange bool) { ctx, cancel := context.WithCancel(context.Background()) gcJobID := make(chan jobspb.JobID) knobs := base.TestingKnobs{ @@ -628,7 +620,12 @@ SELECT descriptor_id, index_id WHERE descriptor_name = 'foo' AND index_name = 'foo_j_i_idx';`).Scan(&tableID, &indexID) // Drop the index. - tdb.Exec(t, "DROP INDEX foo@foo_j_i_idx") + if dropIndex { + tdb.Exec(t, "DROP INDEX foo@foo_j_i_idx") + } else { + tdb.Exec(t, "DROP TABLE foo") + } + codec := s.ExecutorConfig().(sql.ExecutorConfig).Codec tablePrefix.Store(codec.TablePrefix(uint32(tableID))) @@ -654,5 +651,20 @@ SELECT descriptor_id, index_id // Ensure that the job completes successfully in either case. jr := s.JobRegistry().(*jobs.Registry) require.NoError(t, jr.WaitForJobs(ctx, []jobspb.JobID{jobID})) + } + + // The way the GC job works is that it initially clears the index + // data, then it waits for the background MVCC GC to run and remove + // the underlying tombstone, and then finally it removes any relevant + // zone configurations for the index from system.zones. In the first + // and final phases, the job resolves the descriptor. This test ensures + // that the code is robust to the descriptor being removed both before + // the initial DelRange, and after, when going to remove the zone config. + testutils.RunTrueAndFalse(t, "before DelRange", func( + t *testing.T, beforeDelRange bool, + ) { + testutils.RunTrueAndFalse(t, "drop index", func(t *testing.T, dropIndex bool) { + runTest(t, dropIndex, beforeDelRange) + }) }) } diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel index 638635796fd1..8516d347a76e 100644 --- a/pkg/sql/sem/builtins/BUILD.bazel +++ b/pkg/sql/sem/builtins/BUILD.bazel @@ -90,6 +90,7 @@ go_library( "//pkg/sql/sem/volatility", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", + "//pkg/sql/sqlerrors", "//pkg/sql/sqlliveness", "//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil", "//pkg/sql/sqltelemetry", diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 09a724d7b7b7..621d1c3658a4 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/errors" @@ -988,12 +989,10 @@ var pgBuiltins = map[string]builtinDefinition{ typ, err = evalCtx.Planner.ResolveTypeByOID(ctx, oid) if err != nil { // If the error is a descriptor does not exist error, then swallow it. - unknown := tree.NewDString(fmt.Sprintf("unknown (OID=%s)", oidArg)) switch { - case errors.Is(err, catalog.ErrDescriptorNotFound): - return unknown, nil - case pgerror.GetPGCode(err) == pgcode.UndefinedObject: - return unknown, nil + case sqlerrors.IsMissingDescriptorError(err), + errors.Is(err, catalog.ErrDescriptorNotFound): + return tree.NewDString(fmt.Sprintf("unknown (OID=%s)", oidArg)), nil default: return nil, err } diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index c3f8d04f379c..3369aae2b99f 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -378,6 +378,21 @@ func IsUndefinedSchemaError(err error) bool { return errHasCode(err, pgcode.UndefinedSchema) } +// IsMissingDescriptorError checks whether the error has any indication +// that it corresponds to a missing descriptor of any kind. +// +// Note that this does not deal with the lower-level +// catalog.ErrDescriptorNotFound error. That error should be transformed +// by this package for all uses in the SQL layer and coming out of +// descs.Collection functions. +func IsMissingDescriptorError(err error) bool { + return IsUndefinedRelationError(err) || + IsUndefinedSchemaError(err) || + IsUndefinedDatabaseError(err) || + errHasCode(err, pgcode.UndefinedObject) || + errHasCode(err, pgcode.UndefinedFunction) +} + func errHasCode(err error, code ...pgcode.Code) bool { pgCode := pgerror.GetPGCode(err) for _, c := range code {