Skip to content

Commit cd1ff14

Browse files
committed
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. Release note: None
1 parent 2f82071 commit cd1ff14

File tree

2 files changed

+67
-55
lines changed

2 files changed

+67
-55
lines changed

pkg/sql/stats/create_stats_job_test.go

Lines changed: 67 additions & 54 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)
@@ -241,8 +241,8 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
241241
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
242242
TestingRequestFilter: filter,
243243
}
244-
params.ServerArgs.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109379)
245244

245+
rng, _ := randutil.NewTestRand()
246246
ctx := context.Background()
247247
const nodes = 1
248248
tc := testcluster.StartTestCluster(t, nodes, params)
@@ -303,28 +303,30 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
303303
runAutoStatsJob(t, sqlDB, "d.t", false /* partial */, errorOnConcurrentCreateStats)
304304
runAutoStatsJob(t, sqlDB, "d.t", true /* partial */, errorOnConcurrentCreateStats)
305305

306-
// PAUSE JOB does not block until the job is paused but only requests it.
307-
// Wait until the job is set to paused.
308-
var jobID jobspb.JobID
309-
sqlDB.QueryRow(t, `SELECT id FROM system.jobs ORDER BY created DESC LIMIT 1`).Scan(&jobID)
310-
opts := retry.Options{
311-
InitialBackoff: 1 * time.Millisecond,
312-
MaxBackoff: time.Second,
313-
Multiplier: 2,
314-
}
315-
if err := retry.WithMaxAttempts(context.Background(), opts, 10, func() error {
316-
_, err := sqlDB.DB.ExecContext(context.Background(), `PAUSE JOB $1`, jobID)
317-
if err != nil {
318-
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,
319315
}
320-
var status string
321-
sqlDB.QueryRow(t, `SELECT status FROM system.jobs WHERE id = $1 LIMIT 1`, jobID).Scan(&status)
322-
if status != "paused" {
323-
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)
324329
}
325-
return err
326-
}); err != nil {
327-
t.Fatal(err)
328330
}
329331

330332
// Starting automatic full and partial stats run should still fail.
@@ -343,19 +345,6 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
343345
manualPartialStatErrCh <- err
344346
}()
345347

346-
select {
347-
case allowRequest <- struct{}{}:
348-
case err := <-runningManualFullStatErrCh:
349-
t.Fatal(err)
350-
case err := <-manualFullStatErrCh:
351-
t.Fatal(err)
352-
case err := <-manualPartialStatErrCh:
353-
t.Fatal(err)
354-
}
355-
356-
close(allowRequest)
357-
allowRequestOpen = false
358-
359348
// Verify that the manual full and partial stat jobs completed successfully.
360349
if err := <-manualFullStatErrCh; err != nil {
361350
t.Fatalf("create stats job should have completed: %s", err)
@@ -364,10 +353,30 @@ func testAtMostOneRunningCreateStatsImpl(t *testing.T, errorOnConcurrentCreateSt
364353
t.Fatalf("create partial stats job should have completed: %s", err)
365354
}
366355

367-
// Verify that the running full stat job completed successfully.
356+
beforeCount := getNumberOfTableStats(t, sqlDB, "d.t", "s1")
368357
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.
369362
jobutils.WaitForJobToSucceed(t, sqlDB, jobID)
370-
<-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+
}
371380
}
372381

373382
// TestBackgroundAutoPartialStats tests that a running auto partial stats job
@@ -394,7 +403,6 @@ func testBackgroundAutoPartialStatsImpl(t *testing.T, errorOnConcurrentCreateSta
394403
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
395404
TestingRequestFilter: filter,
396405
}
397-
params.ServerArgs.DefaultTestTenant = base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(109379)
398406

399407
ctx := context.Background()
400408
const nodes = 1
@@ -645,7 +653,7 @@ func TestCreateStatsProgress(t *testing.T) {
645653
}
646654

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

650658
// Ensure that 0 progress has been recorded since there are no existing
651659
// stats available to estimate progress.
@@ -699,7 +707,7 @@ func TestCreateStatsProgress(t *testing.T) {
699707
}
700708

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

704712
// Ensure that partial progress has been recorded since there are existing
705713
// stats available.
@@ -827,7 +835,7 @@ func TestCreateStatsUsingExtremesProgress(t *testing.T) {
827835
}
828836

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

832840
var fractionCompleted float32
833841
prevFractionCompleted := getFractionCompleted(t, sqlDB, jobID)
@@ -934,7 +942,7 @@ func createStatsRequestFilter(
934942
}, func(id descpb.ID) { tableToBlock.Store(id) }
935943
}
936944

937-
func getLastCreateStatsJobID(t testing.TB, db *sqlutils.SQLRunner) jobspb.JobID {
945+
func getLastRunningCreateStatsJobID(t testing.TB, db *sqlutils.SQLRunner) jobspb.JobID {
938946
var jobID jobspb.JobID
939947
db.QueryRow(t, "SELECT id FROM system.jobs WHERE status = 'running' AND "+
940948
"job_type = 'CREATE STATS' ORDER BY created DESC LIMIT 1").Scan(&jobID)
@@ -953,12 +961,20 @@ func getFractionCompleted(t testing.TB, sqlDB *sqlutils.SQLRunner, jobID jobspb.
953961
return progress.Progress.(*jobspb.Progress_FractionCompleted).FractionCompleted
954962
}
955963

956-
func runAutoStatsJob(
957-
t *testing.T, sqlDB *sqlutils.SQLRunner, tableName string, partial bool, shouldFail bool,
958-
) {
964+
func getNumberOfTableStats(
965+
t *testing.T, sqlDB *sqlutils.SQLRunner, tableName, statsName string,
966+
) int {
959967
var tableID descpb.ID
960968
sqlDB.QueryRow(t, `SELECT $1::regclass::int`, tableName).Scan(&tableID)
961969

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+
) {
962978
var statsName string
963979
if partial {
964980
statsName = "__auto_partial__"
@@ -970,19 +986,16 @@ func runAutoStatsJob(
970986
queryPostfix = " USING EXTREMES"
971987
}
972988

973-
var beforeCount int
974-
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)
975991

976992
if shouldFail {
977-
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)
978994
return
979995
}
980996

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

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)