Skip to content

Commit

Permalink
sql: mark index as GCed if table has been GCed in legacy gc path
Browse files Browse the repository at this point in the history
Previously, if a table is GCed before an index is GCed by a TRUNCATE TABLE
gc job, the TRUNCATE TABLE gc job can be stuck in running status because
the table descriptor is missing. This is problematic because these jobs will
never succeed and doing nothing. This commit marks the indexes as GCed if
the descriptor cannot be found assuming that the table has been GCed. Also,
table GC should have GCed all the indexes.

Note that this only affect the legacy GC path.

Epic: None

Release note (bug fix): This commit fixes a bug where TRUNCATE TABLE gc
job can be stuck in running status if the table descriptor has been GCed.
It was because TRUNCATE TABLE actually creates new empty indexes, then
replaces and drops the old indexes. The dropped indexes data are deleted
and GCed within the TRUNCATE TABLE gc job which needed to see the table
descriptor to make progress. But, if the table data has been GCed, the
TRUNCATE TABLE gc job couldn't make progress. This patch makes it able
to handle the missing descriptor edge case and let the TRUNCATE TABLE
gc job succeed.
  • Loading branch information
chengxiong-ruan committed Mar 29, 2023
1 parent 0745cd4 commit a22ef53
Show file tree
Hide file tree
Showing 3 changed files with 84 additions and 0 deletions.
3 changes: 3 additions & 0 deletions pkg/sql/gcjob/refresh_statuses.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,9 @@ func updateStatusForGCElements(
if isMissingDescriptorError(err) {
log.Warningf(ctx, "table %d not found, marking as GC'd", tableID)
markTableGCed(ctx, tableID, progress, jobspb.SchemaChangeGCProgress_CLEARED)
for indexID := range indexDropTimes {
markIndexGCed(ctx, indexID, progress, jobspb.SchemaChangeGCProgress_CLEARED)
}
return false, true, maxDeadline
}
log.Warningf(ctx, "error while calculating GC time for table %d, err: %+v", tableID, err)
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/gcjob_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_test(
args = ["-test.timeout=295s"],
deps = [
"//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/keys",
Expand All @@ -33,6 +34,7 @@ go_test(
"//pkg/sql/gcjob",
"//pkg/sql/isql",
"//pkg/sql/sem/catid",
"//pkg/sql/tests",
"//pkg/storage",
"//pkg/testutils",
"//pkg/testutils/jobutils",
Expand All @@ -44,6 +46,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)
Expand Down
78 changes: 78 additions & 0 deletions pkg/sql/gcjob_test/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -28,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"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/bootstrap"
Expand All @@ -38,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/gcjob"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
Expand All @@ -48,6 +51,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -668,3 +672,77 @@ SELECT descriptor_id, index_id
})
})
}

func TestLegacyIndexGCSucceedsWithMissingDescriptor(t *testing.T) {
defer leaktest.AfterTest(t)()
params, _ := tests.CreateTestServerParams()
// Override binary version to be older.
params.Knobs.Server = &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
// Need to disable MVCC since this test is testing the legacy GC path.
BinaryVersionOverride: clusterversion.ByKey(clusterversion.V23_1_MVCCRangeTombstonesUnconditionallyEnabled - 1),
}
params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()

s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())
tDB := sqlutils.MakeSQLRunner(sqlDB)

tDB.Exec(t, `SET CLUSTER SETTING storage.mvcc.range_tombstones.enabled = false`)
tDB.Exec(t, `CREATE TABLE t(a INT)`)
tDB.Exec(t, `INSERT INTO t VALUES (1), (2)`)
tDB.Exec(t, `TRUNCATE TABLE t`)

var truncateJobID string
testutils.SucceedsSoon(t, func() error {
rslt := tDB.QueryStr(t, `SELECT job_id, status, running_status FROM [SHOW JOBS] WHERE description = 'GC for TRUNCATE TABLE defaultdb.public.t'`)
if len(rslt) != 1 {
t.Fatalf("expect only 1 truncate job, found %d", len(rslt))
}
if rslt[0][1] != "running" {
return errors.New("job not running yet")
}
if rslt[0][2] != "waiting for GC TTL" {
return errors.New("not waiting for gc yet")
}
truncateJobID = rslt[0][0]
return nil
})

tDB.Exec(t, `PAUSE JOB `+truncateJobID)
testutils.SucceedsSoon(t, func() error {
rslt := tDB.QueryStr(t, `SELECT status FROM [SHOW JOBS] WHERE job_id = `+truncateJobID)
if len(rslt) != 1 {
t.Fatalf("expect only 1 truncate job, found %d", len(rslt))
}
if rslt[0][0] != "paused" {
return errors.New("job not paused yet")
}
return nil
})

tDB.Exec(t, `ALTER TABLE t CONFIGURE ZONE USING gc.ttlseconds = 1;`)
tDB.Exec(t, `DROP TABLE t`)
testutils.SucceedsSoon(t, func() error {
rslt := tDB.QueryStr(t, `SELECT status FROM [SHOW JOBS] WHERE description = 'GC for DROP TABLE defaultdb.public.t'`)
if len(rslt) != 1 {
t.Fatalf("expect only 1 truncate job, found %d", len(rslt))
}
if rslt[0][0] != "succeeded" {
return errors.New("job not running yet")
}
return nil
})

tDB.Exec(t, `RESUME JOB `+truncateJobID)
testutils.SucceedsSoon(t, func() error {
rslt := tDB.QueryStr(t, `SELECT status FROM [SHOW JOBS] WHERE job_id = `+truncateJobID)
if len(rslt) != 1 {
t.Fatalf("expect only 1 truncate job, found %d", len(rslt))
}
if rslt[0][0] != "succeeded" {
return errors.New("job not running")
}
return nil
})
}

0 comments on commit a22ef53

Please sign in to comment.