Skip to content

Commit 088887e

Browse files
craig[bot]yuzefovich
andcommitted
Merge #155825
155825: sql/stats: enable a couple of tests under test tenants r=yuzefovich a=yuzefovich **sql/stats: prevent deadlocks in a couple of tests when failing** The deferred functions that close the channel (which can block the server) need to be executed _before_ `Stopper.Stop` of that server. **sql/stats: enable a couple of tests under test tenants** `TestAtMostOneRunningCreateStats` was subtly broken. Namely, we ignored the error at the end of the test on the "background" full stats job. This was added in 7de0ae1 when we introduced the logic to treat the paused auto stats job as blocking any other concurrent auto stats jobs. This commit improves the test by pausing the job in 50% cases (in other 50% we simply block the background stats job, similar to how things were when the test was first added in 048bd3d) and then adjusting the expectations around the error sent on the channel. In particular, if the job is paused, then we expect to see an error even though the table stats are collected. `TestStatsAreDeletedForDroppedTables` and `TestBackgroundAutoPartialStats` simply worked. Fixes: #109379. Fixes: #109380. Epic: CRDB-48945 Release note: None Co-authored-by: Yahor Yuzefovich <[email protected]>
2 parents 448eea6 + cd1ff14 commit 088887e

File tree

2 files changed

+81
-65
lines changed

2 files changed

+81
-65
lines changed

pkg/sql/stats/create_stats_job_test.go

Lines changed: 81 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ import (
3232
"github.com/cockroachdb/cockroach/pkg/util/encoding"
3333
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
3434
"github.com/cockroachdb/cockroach/pkg/util/log"
35+
"github.com/cockroachdb/cockroach/pkg/util/randutil"
3536
"github.com/cockroachdb/cockroach/pkg/util/retry"
3637
"github.com/cockroachdb/errors"
3738
"github.com/stretchr/testify/require"
@@ -147,8 +148,7 @@ func runCreateStatsJob(
147148
case err := <-errCh:
148149
return 0, errors.Wrapf(err, "query returned before expected: %s", query)
149150
}
150-
var jobID jobspb.JobID
151-
db.QueryRow(t, `SELECT id FROM system.jobs WHERE job_type = 'CREATE STATS' ORDER BY created DESC LIMIT 1`).Scan(&jobID)
151+
jobID := getLastRunningCreateStatsJobID(t, db)
152152
db.Exec(t, fmt.Sprintf("%s JOB %d", op, jobID))
153153
*allowProgressIota <- struct{}{}
154154
close(*allowProgressIota)
@@ -234,24 +234,26 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
234234

235235
var allowRequest chan struct{}
236236
var allowRequestOpen bool
237-
defer func() {
238-
if allowRequestOpen {
239-
close(allowRequest)
240-
}
241-
}()
242237

243238
filter, setTableID := createStatsRequestFilter(&allowRequest)
244239
var params base.TestClusterArgs
245240
params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
246241
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
247242
TestingRequestFilter: filter,
248243
}
249-
params.ServerArgs.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109379)
250244

245+
rng, _ := randutil.NewTestRand()
251246
ctx := context.Background()
252247
const nodes = 1
253248
tc := testcluster.StartTestCluster(t, nodes, params)
254249
defer tc.Stopper().Stop(ctx)
250+
251+
defer func() {
252+
if allowRequestOpen {
253+
close(allowRequest)
254+
}
255+
}()
256+
255257
conn := tc.ApplicationLayer(0).SQLConn(t)
256258
sqlDB := sqlutils.MakeSQLRunner(conn)
257259

@@ -301,28 +303,30 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
301303
runAutoStatsJob(t, sqlDB, "d.t", false /* partial */, errorOnConcurrentCreateStats)
302304
runAutoStatsJob(t, sqlDB, "d.t", true /* partial */, errorOnConcurrentCreateStats)
303305

304-
// PAUSE JOB does not block until the job is paused but only requests it.
305-
// Wait until the job is set to paused.
306-
var jobID jobspb.JobID
307-
sqlDB.QueryRow(t, `SELECT id FROM system.jobs ORDER BY created DESC LIMIT 1`).Scan(&jobID)
308-
opts := retry.Options{
309-
InitialBackoff: 1 * time.Millisecond,
310-
MaxBackoff: time.Second,
311-
Multiplier: 2,
312-
}
313-
if err := retry.WithMaxAttempts(context.Background(), opts, 10, func() error {
314-
_, err := sqlDB.DB.ExecContext(context.Background(), `PAUSE JOB $1`, jobID)
315-
if err != nil {
316-
t.Fatal(err)
306+
jobID := getLastRunningCreateStatsJobID(t, sqlDB)
307+
pauseJob := rng.Float64() < 0.5
308+
if pauseJob {
309+
// PAUSE JOB does not block until the job is paused but only requests it.
310+
// Wait until the job is set to paused.
311+
opts := retry.Options{
312+
InitialBackoff: 1 * time.Millisecond,
313+
MaxBackoff: time.Second,
314+
Multiplier: 2,
317315
}
318-
var status string
319-
sqlDB.QueryRow(t, `SELECT status FROM system.jobs WHERE id = $1 LIMIT 1`, jobID).Scan(&status)
320-
if status != "paused" {
321-
return errors.New("could not pause job")
316+
if err := retry.WithMaxAttempts(context.Background(), opts, 10, func() error {
317+
_, err := sqlDB.DB.ExecContext(context.Background(), `PAUSE JOB $1`, jobID)
318+
if err != nil {
319+
t.Fatal(err)
320+
}
321+
var status string
322+
sqlDB.QueryRow(t, `SELECT status FROM system.jobs WHERE id = $1 LIMIT 1`, jobID).Scan(&status)
323+
if status != "paused" {
324+
return errors.New("could not pause job")
325+
}
326+
return err
327+
}); err != nil {
328+
t.Fatal(err)
322329
}
323-
return err
324-
}); err != nil {
325-
t.Fatal(err)
326330
}
327331

328332
// Starting automatic full and partial stats run should still fail.
@@ -341,19 +345,6 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
341345
manualPartialStatErrCh <- err
342346
}()
343347

344-
select {
345-
case allowRequest <- struct{}{}:
346-
case err := <-runningManualFullStatErrCh:
347-
t.Fatal(err)
348-
case err := <-manualFullStatErrCh:
349-
t.Fatal(err)
350-
case err := <-manualPartialStatErrCh:
351-
t.Fatal(err)
352-
}
353-
354-
close(allowRequest)
355-
allowRequestOpen = false
356-
357348
// Verify that the manual full and partial stat jobs completed successfully.
358349
if err := <-manualFullStatErrCh; err != nil {
359350
t.Fatalf("create stats job should have completed: %s", err)
@@ -362,10 +353,30 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
362353
t.Fatalf("create partial stats job should have completed: %s", err)
363354
}
364355

365-
// Verify that the running full stat job completed successfully.
356+
beforeCount := getNumberOfTableStats(t, sqlDB, "d.t", "s1")
366357
sqlDB.Exec(t, fmt.Sprintf("RESUME JOB %d", jobID))
358+
close(allowRequest)
359+
allowRequestOpen = false
360+
361+
// Verify that the running full stat job completed successfully.
367362
jobutils.WaitForJobToSucceed(t, sqlDB, jobID)
368-
<-runningManualFullStatErrCh
363+
if pauseJob {
364+
// If the job was paused, then we expect an error to be returned to us
365+
// even though the stats were collected.
366+
if err := <-runningManualFullStatErrCh; !testutils.IsError(err, "node liveness error: restarting in background") {
367+
t.Fatalf("expected 'node liveness error: restarting in background' error, found %v", err)
368+
}
369+
} else {
370+
// If the job wasn't paused, then we expect no error.
371+
if err := <-runningManualFullStatErrCh; err != nil {
372+
t.Fatalf("expected no error, found %v", err)
373+
}
374+
}
375+
// Now ensure that the new statistic is present.
376+
afterCount := getNumberOfTableStats(t, sqlDB, "d.t", "s1")
377+
if beforeCount == afterCount {
378+
t.Fatal("expected new statistic to have been collected")
379+
}
369380
}
370381

371382
// TestBackgroundAutoPartialStats tests that a running auto partial stats job
@@ -385,24 +396,25 @@ func testBackgroundAutoPartialStatsImpl(t *testing.T, errorOnConcurrentCreateSta
385396

386397
var allowRequest chan struct{}
387398
var allowRequestOpen bool
388-
defer func() {
389-
if allowRequestOpen {
390-
close(allowRequest)
391-
}
392-
}()
393399

394400
filter, setTableID := createStatsRequestFilter(&allowRequest)
395401
var params base.TestClusterArgs
396402
params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
397403
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
398404
TestingRequestFilter: filter,
399405
}
400-
params.ServerArgs.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109379)
401406

402407
ctx := context.Background()
403408
const nodes = 1
404409
tc := testcluster.StartTestCluster(t, nodes, params)
405410
defer tc.Stopper().Stop(ctx)
411+
412+
defer func() {
413+
if allowRequestOpen {
414+
close(allowRequest)
415+
}
416+
}()
417+
406418
conn := tc.ApplicationLayer(0).SQLConn(t)
407419
sqlDB := sqlutils.MakeSQLRunner(conn)
408420

@@ -641,7 +653,7 @@ func TestCreateStatsProgress(t *testing.T) {
641653
}
642654

643655
// Fetch the new job ID since we know it's running now.
644-
jobID := getLastCreateStatsJobID(t, sqlDB)
656+
jobID := getLastRunningCreateStatsJobID(t, sqlDB)
645657

646658
// Ensure that 0 progress has been recorded since there are no existing
647659
// stats available to estimate progress.
@@ -695,7 +707,7 @@ func TestCreateStatsProgress(t *testing.T) {
695707
}
696708

697709
// Fetch the new job ID since we know it's running now.
698-
jobID = getLastCreateStatsJobID(t, sqlDB)
710+
jobID = getLastRunningCreateStatsJobID(t, sqlDB)
699711

700712
// Ensure that partial progress has been recorded since there are existing
701713
// stats available.
@@ -823,7 +835,7 @@ func TestCreateStatsUsingExtremesProgress(t *testing.T) {
823835
}
824836

825837
// Fetch the new job ID since we know it's running now.
826-
jobID := getLastCreateStatsJobID(t, sqlDB)
838+
jobID := getLastRunningCreateStatsJobID(t, sqlDB)
827839

828840
var fractionCompleted float32
829841
prevFractionCompleted := getFractionCompleted(t, sqlDB, jobID)
@@ -930,7 +942,7 @@ func createStatsRequestFilter(
930942
}, func(id descpb.ID) { tableToBlock.Store(id) }
931943
}
932944

933-
func getLastCreateStatsJobID(t testing.TB, db *sqlutils.SQLRunner) jobspb.JobID {
945+
func getLastRunningCreateStatsJobID(t testing.TB, db *sqlutils.SQLRunner) jobspb.JobID {
934946
var jobID jobspb.JobID
935947
db.QueryRow(t, "SELECT id FROM system.jobs WHERE status = 'running' AND "+
936948
"job_type = 'CREATE STATS' ORDER BY created DESC LIMIT 1").Scan(&jobID)
@@ -949,12 +961,20 @@ func getFractionCompleted(t testing.TB, sqlDB *sqlutils.SQLRunner, jobID jobspb.
949961
return progress.Progress.(*jobspb.Progress_FractionCompleted).FractionCompleted
950962
}
951963

952-
func runAutoStatsJob(
953-
t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, partial bool, shouldFail bool,
954-
) {
964+
func getNumberOfTableStats(
965+
t *testing.T, sqlDB *sqlutils.SQLRunner, tableName, statsName string,
966+
) int {
955967
var tableID descpb.ID
956968
sqlDB.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID)
957969

970+
var count int
971+
sqlDB.QueryRow(t, `SELECT count(*) FROM system.table_statistics WHERE "tableID" = $1 AND name = $2`, tableID, statsName).Scan(&count)
972+
return count
973+
}
974+
975+
func runAutoStatsJob(
976+
t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, partial bool, shouldFail bool,
977+
) {
958978
var statsName string
959979
if partial {
960980
statsName = "__auto_partial__"
@@ -966,19 +986,16 @@ func runAutoStatsJob(
966986
queryPostfix = " USING EXTREMES"
967987
}
968988

969-
var beforeCount int
970-
sqlDB.QueryRow(t, `SELECT count(*) FROM system.table_statistics WHERE "tableID" = $1 AND name = $2`, tableID, statsName).Scan(&beforeCount)
989+
query := fmt.Sprintf("CREATE STATISTICS %s FROM %s%s", statsName, tableName, queryPostfix)
990+
beforeCount := getNumberOfTableStats(t, sqlDB, tableName, statsName)
971991

972992
if shouldFail {
973-
sqlDB.ExpectErr(t, "another CREATE STATISTICS job is already running", fmt.Sprintf("CREATE STATISTICS %s FROM %s%s", statsName, tableName, queryPostfix))
993+
sqlDB.ExpectErr(t, "another CREATE STATISTICS job is already running", query)
974994
return
975995
}
976996

977-
sqlDB.Exec(t, fmt.Sprintf("CREATE STATISTICS %s FROM %s%s", statsName, tableName, queryPostfix))
978-
979-
var afterCount int
980-
sqlDB.QueryRow(t, `SELECT count(*) FROM system.table_statistics WHERE "tableID" = $1 AND name = $2`, tableID, statsName).Scan(&afterCount)
981-
997+
sqlDB.Exec(t, query)
998+
afterCount := getNumberOfTableStats(t, sqlDB, tableName, statsName)
982999
if beforeCount != afterCount {
9831000
t.Fatalf("auto stats job should have failed, but it didn't (beforeCount: %d, afterCount: %d)", beforeCount, afterCount)
9841001
}

pkg/sql/stats/delete_stats_test.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -682,7 +682,6 @@ func TestStatsAreDeletedForDroppedTables(t *testing.T) {
682682

683683
var params base.TestServerArgs
684684
params.ScanMaxIdleTime = time.Millisecond // speed up MVCC GC queue scans
685-
params.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109380)
686685
s, sqlDB, _ := serverutils.StartServer(t, params)
687686
defer s.Stopper().Stop(context.Background())
688687
runner := sqlutils.MakeSQLRunner(sqlDB)

0 commit comments

Comments
 (0)