Skip to content

Commit

Permalink
Merge #99871 #99888
Browse files Browse the repository at this point in the history
99871: roachtest: add back backups to disk-stalled tests r=jbowens a=nicktrav

Backups were inadvertently disabled in #99747 (to simplify testing). As there is no good reason to run without them, re-enable the backups during the test.

Release note: None.

Epic: CRDB-20293

99888: sql: mark index as GCed if table has been GCed in legacy gc path r=chengxiong-ruan a=chengxiong-ruan

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 (sql change): This commit fixes a bug where TRUNCATE TABLE gc job can be stuck in running status if table descriptor has been GCed before the truncated indexes are GCed. The bug was only a problem before `DelRange` is not available.

Co-authored-by: Nick Travers <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
  • Loading branch information
3 people committed Mar 30, 2023
3 parents 47bd780 + a04827c + a22ef53 commit 73f1291
Show file tree
Hide file tree
Showing 4 changed files with 85 additions and 1 deletion.
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ func registerDiskStalledDetection(r registry.Registry) {
func runDiskStalledDetection(
ctx context.Context, t test.Test, c cluster.Cluster, s diskStaller, doStall bool,
) {
startOpts := option.DefaultStartOptsNoBackups()
startOpts := option.DefaultStartOpts()
startOpts.RoachprodOpts.ExtraArgs = []string{
"--store", s.DataDir(),
"--log", fmt.Sprintf(`{sinks: {stderr: {filter: INFO}}, file-defaults: {dir: "%s"}}`, s.LogDir()),
Expand Down
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 73f1291

Please sign in to comment.