From ed2e0904af26fca429e8245b8cd4778c90d45654 Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Tue, 23 Aug 2022 15:37:04 -0400 Subject: [PATCH] sql/gcjob: make index GC robust to descriptors being deleted If the descriptor was deleted, the GC job should exit gracefully. Fixes #86340 Release justification: bug fix for backport Release note (bug fix): In some scenarios, when a DROP INDEX was run around the same time as a DROP TABLE or DROP DATABASE covering the same data, the `DROP INDEX` gc job could get caught retrying indefinitely. This has been fixed. --- pkg/sql/gcjob/BUILD.bazel | 1 + pkg/sql/gcjob/index_garbage_collection.go | 38 ++++++- pkg/sql/gcjob_test/BUILD.bazel | 2 + pkg/sql/gcjob_test/gc_job_test.go | 118 ++++++++++++++++++++++ 4 files changed, 156 insertions(+), 3 deletions(-) diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index fc05b14b936e..431d2b43177b 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -38,6 +38,7 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/tree", + "//pkg/sql/sqlerrors", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", "//pkg/util/log", diff --git a/pkg/sql/gcjob/index_garbage_collection.go b/pkg/sql/gcjob/index_garbage_collection.go index 8e63f08d11cd..22b3261543aa 100644 --- a/pkg/sql/gcjob/index_garbage_collection.go +++ b/pkg/sql/gcjob/index_garbage_collection.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -44,6 +45,10 @@ 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) { + handleTableDescriptorDeleted(ctx, parentID, progress) + return nil + } if err != nil { return err } @@ -89,6 +94,10 @@ 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) { + handleTableDescriptorDeleted(ctx, parentID, progress) + return nil + } if err != nil { return err } @@ -129,10 +138,15 @@ func gcIndexes( ctx, txn, execCfg, descriptors, freshParentTableDesc, []uint32{uint32(index.IndexID)}, ) } - if err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs); err != nil { + err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs) + if errors.Is(err, catalog.ErrDescriptorNotFound) || + sqlerrors.IsUndefinedRelationError(err) { + handleTableDescriptorDeleted(ctx, parentID, progress) + return nil + } + if err != nil { return errors.Wrapf(err, "removing index %d zone configs", index.IndexID) } - markIndexGCed( ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED, ) @@ -214,7 +228,8 @@ func deleteIndexZoneConfigsAfterGC( } err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs) switch { - case errors.Is(err, catalog.ErrDescriptorNotFound): + case errors.Is(err, catalog.ErrDescriptorNotFound), + sqlerrors.IsUndefinedRelationError(err): log.Infof(ctx, "removing index %d zone config from table %d failed: %v", index.IndexID, parentID, err) case err != nil: @@ -226,3 +241,20 @@ func deleteIndexZoneConfigsAfterGC( } return nil } + +// handleTableDescriptorDeleted should be called when logic detects that +// a table descriptor has been deleted while attempting to GC an index. +// The function marks in progress that all indexes have been cleared. +func handleTableDescriptorDeleted( + ctx context.Context, parentID descpb.ID, progress *jobspb.SchemaChangeGCProgress, +) { + droppedIndexes := progress.Indexes + // If the descriptor has been removed, then we need to assume that the relevant + // zone configs and data have been cleaned up by another process. + log.Infof(ctx, "descriptor %d dropped, assuming another process has handled GC", parentID) + for _, index := range droppedIndexes { + markIndexGCed( + ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED, + ) + } +} diff --git a/pkg/sql/gcjob_test/BUILD.bazel b/pkg/sql/gcjob_test/BUILD.bazel index 8ceb0892c437..4d0b4a1db8ce 100644 --- a/pkg/sql/gcjob_test/BUILD.bazel +++ b/pkg/sql/gcjob_test/BUILD.bazel @@ -33,6 +33,8 @@ go_test( "//pkg/sql/catalog/tabledesc", "//pkg/sql/gcjob", "//pkg/sql/gcjob/gcjobnotifier", + "//pkg/sql/sem/catid", + "//pkg/sql/sqlutil", "//pkg/testutils", "//pkg/testutils/jobutils", "//pkg/testutils/serverutils", diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index d26bdbc932d0..709e61d5726f 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -39,6 +39,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -506,6 +508,122 @@ func TestGCTenant(t *testing.T) { }) } +// This test exercises code whereby an index 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) { + 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, + ) { + ctx, cancel := context.WithCancel(context.Background()) + gcJobID := make(chan jobspb.JobID) + knobs := base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + GCJob: &sql.GCJobTestingKnobs{ + RunBeforeResume: func(jobID jobspb.JobID) error { + select { + case <-ctx.Done(): + return ctx.Err() + case gcJobID <- jobID: + return nil + } + }, + SkipWaitingForMVCCGC: true, + }, + } + delRangeChan := make(chan chan struct{}) + var tablePrefix atomic.Value + tablePrefix.Store(roachpb.Key{}) + // If not running beforeDelRange, we want to delete the descriptor during + // the DeleteRange operation. To do this, we install the below testing knob. + if !beforeDelRange { + knobs.Store = &kvserver.StoreTestingKnobs{ + TestingRequestFilter: func( + ctx context.Context, request roachpb.BatchRequest, + ) *roachpb.Error { + req, ok := request.GetArg(roachpb.DeleteRange) + if !ok { + return nil + } + dr := req.(*roachpb.DeleteRangeRequest) + if !dr.UseRangeTombstone { + return nil + } + k := tablePrefix.Load().(roachpb.Key) + if len(k) == 0 { + return nil + } + ch := make(chan struct{}) + select { + case delRangeChan <- ch: + case <-ctx.Done(): + } + select { + case <-ch: + case <-ctx.Done(): + } + return nil + }, + } + } + s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: knobs, + }) + defer s.Stopper().Stop(ctx) + defer cancel() + tdb := sqlutils.MakeSQLRunner(sqlDB) + + // Create the table and index to be dropped. + tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY, j INT, INDEX(j, i))") + // Store the relevant IDs to make it easy to intercept the DelRange. + var tableID catid.DescID + var indexID catid.IndexID + tdb.QueryRow(t, ` +SELECT descriptor_id, index_id + FROM crdb_internal.table_indexes + 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") + codec := s.ExecutorConfig().(sql.ExecutorConfig).Codec + tablePrefix.Store(codec.TablePrefix(uint32(tableID))) + + deleteDescriptor := func(t *testing.T) { + t.Helper() + k := catalogkeys.MakeDescMetadataKey(codec, tableID) + _, err := kvDB.Del(ctx, k) + require.NoError(t, err) + } + + // Delete the descriptor either before the initial job run, or after + // the job has started, but during the sending of DeleteRange requests. + var jobID jobspb.JobID + if beforeDelRange { + deleteDescriptor(t) + jobID = <-gcJobID + } else { + jobID = <-gcJobID + ch := <-delRangeChan + deleteDescriptor(t) + close(ch) + } + // Ensure that the job completes successfully in either case. + require.NoError(t, s.JobRegistry().(*jobs.Registry).WaitForJobs( + ctx, s.InternalExecutor().(sqlutil.InternalExecutor), []jobspb.JobID{jobID}, + )) + }) +} + // TestGCJobNoSystemConfig tests that the GC job is robust to running with // no system config provided by the SystemConfigProvider. It is a regression // test for a panic which could occur due to a slow systemconfigwatcher