Skip to content

Commit

Permalink
sql/gcjob: make index GC robust to descriptors being deleted
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
ajwerner committed Aug 23, 2022
1 parent 660a17a commit f181f82
Show file tree
Hide file tree
Showing 3 changed files with 92 additions and 2 deletions.
42 changes: 40 additions & 2 deletions pkg/sql/gcjob/index_garbage_collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,10 +40,28 @@ func deleteIndexData(
log.Infof(ctx, "GC is being considered on table %d for indexes indexes: %+v", parentID, droppedIndexes)
}

maybeHandleDeletedDescriptor := func(err error) (done bool) {
// 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.
if !errors.Is(err, catalog.ErrDescriptorNotFound) {
return false
}
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,
)
}
return true
}

// Before deleting any indexes, ensure that old versions of the table descriptor
// 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 maybeHandleDeletedDescriptor(err) {
return nil
}
if err != nil {
return err
}
Expand Down Expand Up @@ -84,11 +102,28 @@ func gcIndexes(
if log.V(2) {
log.Infof(ctx, "GC is being considered on table %d for indexes indexes: %+v", parentID, droppedIndexes)
}
maybeHandleDeletedDescriptor := func(err error) (done bool) {
// 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.
if !errors.Is(err, catalog.ErrDescriptorNotFound) {
return false
}
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,
)
}
return true
}

// Before deleting any indexes, ensure that old versions of the table descriptor
// 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 maybeHandleDeletedDescriptor(err) {
return nil
}
if err != nil {
return err
}
Expand Down Expand Up @@ -129,10 +164,13 @@ 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 maybeHandleDeletedDescriptor(err) {
return nil
}
if err != nil {
return errors.Wrapf(err, "removing index %d zone configs", index.IndexID)
}

markIndexGCed(
ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED,
)
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/gcjob_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,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",
Expand Down
50 changes: 50 additions & 0 deletions pkg/sql/gcjob_test/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -506,6 +508,54 @@ 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)

ctx, cancel := context.WithCancel(context.Background())
gcJobID := make(chan jobspb.JobID)
s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{
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
}
}},
},
})
defer s.Stopper().Stop(ctx)
defer cancel()
tdb := sqlutils.MakeSQLRunner(sqlDB)
tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY, j INT, INDEX(j, i))")
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)
tdb.Exec(t, "DROP INDEX foo@foo_j_i_idx")
codec := s.ExecutorConfig().(sql.ExecutorConfig).Codec
errCh := make(chan error, 1)
// Allow the job to proceed in parallel to deleting the descriptor.
jobID := <-gcJobID
go func() {
k := catalogkeys.MakeDescMetadataKey(codec, tableID)
_, err := kvDB.Del(ctx, k)
errCh <- err
}()
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
Expand Down

0 comments on commit f181f82

Please sign in to comment.