Skip to content

Commit 77e13d8

Browse files
craig[bot]kev-caofqazirafiss
committed
148967: roachtest: add fingerprint validation to classic restore roachtests r=jeffswenson a=kev-cao With the switch to the new fixtures in #146555, we temporarily removed fingerprint validation. This commit teaches the restore roachtests to fingerprint after restore, validating the results against the fingerprints stored in the fixtures (introduced in #147662. Epic: CRDB-50823 Fixes: #148962 Release note: None 149265: roachtest: follower-reads/mixed-version handle schema_locked r=rafiss a=fqazi Previously, we unconditionally attempted to toggle schema_locked in the follower read mixed versions tests, which would fail since versions before 25.3 do not support the new create_table_withschema_locked setting. This patch updates the test to only toggle this setting if the active version for the cluster is 25.3 or greater. Fixes: #149150 Release note: None 149353: roachtest: mark cluster as expected to die in validate_system_schema_after_version_upgrade r=rafiss a=rafiss Before wiping the cluster, we need to indicate to the test that the cluster is going to go down. This is needed after eae1e78. fixes #148981 fixes #148998 Release note: None Co-authored-by: Kevin Cao <[email protected]> Co-authored-by: Faizan Qazi <[email protected]> Co-authored-by: Rafi Shamim <[email protected]>
4 parents 164a4ba + 57246d2 + 0d75cc2 + d8530cb commit 77e13d8

File tree

6 files changed

+175
-131
lines changed

6 files changed

+175
-131
lines changed

pkg/cmd/roachtest/tests/backup_fixtures.go

Lines changed: 44 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ package tests
77

88
import (
99
"context"
10+
gosql "database/sql"
1011
"fmt"
1112
"net/url"
1213
"path"
@@ -25,6 +26,7 @@ import (
2526
"github.com/cockroachdb/cockroach/pkg/roachprod"
2627
"github.com/cockroachdb/cockroach/pkg/roachprod/blobfixture"
2728
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
29+
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
2830
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
2931
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
3032
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
@@ -412,20 +414,52 @@ func (bd *backupDriver) queryJobStates(
412414
func (bd *backupDriver) fingerprintFixture(ctx context.Context) map[string]string {
413415
conn := bd.c.Conn(ctx, bd.t.L(), 1)
414416
defer conn.Close()
415-
sql := sqlutils.MakeSQLRunner(conn)
416-
aost := bd.getLatestAOST(ctx, sql)
417-
tables := getDatabaseTables(ctx, bd.t, sql, bd.sp.fixture.DatabaseName())
417+
return fingerprintDatabase(
418+
ctx, bd.t, conn,
419+
bd.sp.fixture.DatabaseName(), bd.getLatestAOST(sqlutils.MakeSQLRunner(conn)),
420+
)
421+
}
418422

419-
m := bd.c.NewDeprecatedMonitor(ctx)
423+
// getLatestAOST returns the end time as seen in SHOW BACKUP of the latest
424+
// backup in the fixture.
425+
func (bd *backupDriver) getLatestAOST(sql *sqlutils.SQLRunner) string {
426+
uri := bd.registry.URI(bd.fixture.DataPath)
427+
query := fmt.Sprintf(
428+
`SELECT end_time FROM
429+
[SHOW BACKUP FROM LATEST IN '%s']
430+
ORDER BY end_time DESC
431+
LIMIT 1`,
432+
uri.String(),
433+
)
434+
var endTime string
435+
sql.QueryRow(bd.t, query).Scan(&endTime)
436+
return endTime
437+
}
438+
439+
// fingerprintDatabase fingerprints all of the tables in the provided database
440+
// and returns a map of fully qualified table names to their fingerprints.
441+
// If AOST is not provided, the current time is used as the AOST.
442+
func fingerprintDatabase(
443+
ctx context.Context, t test.Test, conn *gosql.DB, dbName string, aost string,
444+
) map[string]string {
445+
sql := sqlutils.MakeSQLRunner(conn)
446+
tables := getDatabaseTables(ctx, t, sql, dbName)
447+
if len(tables) == 0 {
448+
t.L().Printf("no tables found in database %s", dbName)
449+
return nil
450+
}
451+
t.L().Printf("fingerprinting %d tables in database %s", len(tables), dbName)
420452

421-
bd.t.L().Printf("fingerprinting %d tables in %s", len(tables), bd.sp.fixture.DatabaseName())
422453
fingerprints := make(map[string]string)
423454
var mu syncutil.Mutex
424455
start := timeutil.Now()
456+
group := t.NewErrorGroup()
425457
for _, table := range tables {
426-
m.Go(func(ctx context.Context) error {
458+
group.Go(func(ctx context.Context, log *logger.Logger) error {
427459
fpContents := newFingerprintContents(conn, table)
428-
if err := fpContents.Load(ctx, bd.t.L(), aost, nil /* tableContents */); err != nil {
460+
if err := fpContents.Load(
461+
ctx, log, aost, nil, /* tableContents */
462+
); err != nil {
429463
return err
430464
}
431465
mu.Lock()
@@ -434,31 +468,14 @@ func (bd *backupDriver) fingerprintFixture(ctx context.Context) map[string]strin
434468
return nil
435469
})
436470
}
437-
m.Wait()
438-
bd.t.L().Printf(
471+
require.NoError(t, group.WaitE(), "error fingerprinting tables in database %s", dbName)
472+
t.L().Printf(
439473
"fingerprinted %d tables in %s in %s",
440-
len(tables), bd.sp.fixture.DatabaseName(), timeutil.Since(start),
474+
len(tables), dbName, timeutil.Since(start),
441475
)
442-
443476
return fingerprints
444477
}
445478

446-
// getLatestAOST returns the end time as seen in SHOW BACKUP of the latest
447-
// backup in the fixture.
448-
func (bd *backupDriver) getLatestAOST(ctx context.Context, sql *sqlutils.SQLRunner) string {
449-
uri := bd.registry.URI(bd.fixture.DataPath)
450-
query := fmt.Sprintf(
451-
`SELECT end_time FROM
452-
[SHOW BACKUP FROM LATEST IN '%s']
453-
ORDER BY end_time DESC
454-
LIMIT 1`,
455-
uri.String(),
456-
)
457-
var endTime string
458-
sql.QueryRow(bd.t, query).Scan(&endTime)
459-
return endTime
460-
}
461-
462479
// getDatabaseTables returns the fully qualified name of every table in the
463480
// fixture.
464481
// Note: This assumes there aren't any funky characters in the identifiers, so

pkg/cmd/roachtest/tests/follower_reads.go

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"strings"
1818
"time"
1919

20+
"github.com/cockroachdb/cockroach/pkg/clusterversion"
2021
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
2122
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
2223
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
@@ -25,6 +26,7 @@ import (
2526
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/task"
2627
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2728
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
29+
"github.com/cockroachdb/cockroach/pkg/roachpb"
2830
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
2931
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
3032
"github.com/cockroachdb/cockroach/pkg/roachprod/vm"
@@ -97,7 +99,7 @@ func registerFollowerReads(r registry.Registry) {
9799
}()
98100

99101
rng, _ := randutil.NewPseudoRand()
100-
data := initFollowerReadsDB(ctx, t, t.L(), c, connFunc, connFunc, rng, topology)
102+
data := initFollowerReadsDB(ctx, t, t.L(), c, connFunc, connFunc, rng, topology, clusterversion.Latest.Version())
101103
runFollowerReadsTest(ctx, t, t.L(), c, connFunc, connFunc, rng, topology, rc, data)
102104
},
103105
})
@@ -488,6 +490,7 @@ func initFollowerReadsDB(
488490
connectFunc, systemConnectFunc func(int) *gosql.DB,
489491
rng *rand.Rand,
490492
topology topologySpec,
493+
clusterVersion roachpb.Version,
491494
) (data map[int]int64) {
492495
systemDB := systemConnectFunc(1)
493496
db := connectFunc(1)
@@ -529,10 +532,12 @@ func initFollowerReadsDB(
529532

530533
// Disable schema_locked within this since it will modify locality on
531534
// tables.
532-
_, err = db.ExecContext(ctx, "SET create_table_with_schema_locked=false")
533-
require.NoError(t, err)
534-
_, err = db.ExecContext(ctx, "ALTER ROLE ALL SET create_table_with_schema_locked=false")
535-
require.NoError(t, err)
535+
if clusterVersion.AtLeast(clusterversion.V25_3.Version()) {
536+
_, err = db.ExecContext(ctx, "SET create_table_with_schema_locked=false")
537+
require.NoError(t, err)
538+
_, err = db.ExecContext(ctx, "ALTER ROLE ALL SET create_table_with_schema_locked=false")
539+
require.NoError(t, err)
540+
}
536541

537542
// Create a multi-region database and table.
538543
_, err = db.ExecContext(ctx, `CREATE DATABASE mr_db`)
@@ -1075,7 +1080,9 @@ func runFollowerReadsMixedVersionTest(
10751080
}
10761081
}
10771082

1078-
data = initFollowerReadsDB(ctx, t, l, c, h.Connect, h.System.Connect, r, topology)
1083+
version, err := h.ClusterVersion(r)
1084+
require.NoError(t, err)
1085+
data = initFollowerReadsDB(ctx, t, l, c, h.Connect, h.System.Connect, r, topology, version)
10791086
return nil
10801087
}
10811088

pkg/cmd/roachtest/tests/mixed_version_backup.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -622,8 +622,9 @@ func newFingerprintContents(db *gosql.DB, table string) *fingerprintContents {
622622
return &fingerprintContents{db: db, table: table}
623623
}
624624

625-
// Load computes the fingerprints for the underlying table and stores
626-
// the contents in the `fingeprints` field.
625+
// Load computes the fingerprints for the underlying table and stores the
626+
// contents in the `fingeprints` field. If timestamp is not set, computes
627+
// the fingerprint for the current time.
627628
func (fc *fingerprintContents) Load(
628629
ctx context.Context, l *logger.Logger, timestamp string, _ tableContents,
629630
) error {

pkg/cmd/roachtest/tests/online_restore.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -292,7 +292,7 @@ func registerOnlineRestorePerf(r registry.Registry) {
292292
// Takes 10 minutes on OR tests for some reason.
293293
SkipPostValidations: registry.PostValidationReplicaDivergence,
294294
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
295-
rd := makeRestoreDriver(t, c, sp.restoreSpecs)
295+
rd := makeRestoreDriver(ctx, t, c, sp.restoreSpecs)
296296
rd.prepareCluster(ctx)
297297

298298
restoreStats := runRestore(
@@ -383,7 +383,7 @@ func registerOnlineRestoreCorrectness(r registry.Registry) {
383383
t, sp, defaultSeed, defaultFakeTime, "-online.trace",
384384
)
385385

386-
rd := makeRestoreDriver(t, c, sp.restoreSpecs)
386+
rd := makeRestoreDriver(ctx, t, c, sp.restoreSpecs)
387387
rd.prepareCluster(ctx)
388388

389389
runRestore(ctx, t, c, regRestoreSpecs, rd, false /* runOnline */, true /* runWorkload */)

0 commit comments

Comments
 (0)