Skip to content

Commit

Permalink
roachtest: change TPCC functions to take a logger instance
Browse files Browse the repository at this point in the history
This makes it easier to organize logs for a complex test by allowing
the caller to inject the logger instance that should be used in a
particular call to `runTPCC`. More immediately, we make use of this
change in the recently introduced `multi-region/mixed-version` test,
and pass the step's logger to those functions.

Epic: none

Release note: None
  • Loading branch information
renatolabs committed Feb 6, 2024
1 parent 1ab858b commit 1fa1df0
Show file tree
Hide file tree
Showing 47 changed files with 99 additions and 95 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func registerElasticControlForBackups(r registry.Registry) {
t.Status(fmt.Sprintf("initializing + running tpcc for %s (<%s)", workloadDuration, estimatedSetupTime))
}

runTPCC(ctx, t, c, tpccOptions{
runTPCC(ctx, t, t.L(), c, tpccOptions{
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/admission_control_elastic_cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ func registerElasticControlForCDC(r registry.Registry) {
t.Fatal(err)
}

runTPCC(ctx, t, c, tpccOptions{
runTPCC(ctx, t, t.L(), c, tpccOptions{
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ func runAdmissionControlFollowerOverload(
nodes := c.Range(1, 3)
c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(), nodes)
db := c.Conn(ctx, t.L(), 1)
require.NoError(t, WaitFor3XReplication(ctx, t, db))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), db))

{
_, err := c.Conn(ctx, t.L(), 1).ExecContext(ctx, `SET CLUSTER SETTING admission.kv.pause_replication_io_threshold = 0.8`)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/admission_control_row_level_ttl.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func makeElasticControlRowLevelTTL(spec spec.ClusterSpec, expiredRows bool) regi
t.Status(fmt.Sprintf("initializing + running tpcc for %s (<%s)", workloadDuration, estimatedSetupTime))
}

runTPCC(ctx, t, c, tpccOptions{
runTPCC(ctx, t, t.L(), c, tpccOptions{
Warehouses: numWarehouses,
Duration: workloadDuration,
SetupType: usingImport,
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/admission_control_tpcc_overload.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ type tpccOLAPSpec struct {

func (s tpccOLAPSpec) run(ctx context.Context, t test.Test, c cluster.Cluster) {
crdbNodes, workloadNode := setupTPCC(
ctx, t, c, tpccOptions{
ctx, t, t.L(), c, tpccOptions{
Warehouses: s.Warehouses, SetupType: usingImport,
})
// We make use of querybench below, only available through the `workload`
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ func registerAllocator(r registry.Registry) {
startTime := timeutil.Now()
m = c.NewMonitor(ctx, clusNodes)
m.Go(func(ctx context.Context) error {
err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
replicateTime = timeutil.Now()
return err
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -645,7 +645,7 @@ func runBackupMVCCRangeTombstones(
_, err = conn.Exec(`SET CLUSTER SETTING server.debug.default_vmodule = 'txn=2,sst_batcher=4,revert=2'`)
require.NoError(t, err)
// Wait for ranges to upreplicate.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the orders table. It's about 16 GB across 8 files.
t.Status("creating table")
Expand Down
8 changes: 4 additions & 4 deletions pkg/cmd/roachtest/tests/cdc_bench.go
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ func runCDCBenchScan(
}

// Wait for system ranges to upreplicate.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create and split the workload table. We don't import data here, because it
// imports before splitting, which takes a very long time.
Expand All @@ -259,7 +259,7 @@ func runCDCBenchScan(
t.L().Printf("creating table with %s ranges", humanize.Comma(numRanges))
c.Run(ctx, option.WithNodes(nCoord), fmt.Sprintf(
`./cockroach workload init kv --splits %d {pgurl:%d}`, numRanges, nData[0]))
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

cursor := timeutil.Now() // before data is ingested

Expand Down Expand Up @@ -391,7 +391,7 @@ func runCDCBenchWorkload(
}

// Wait for system ranges to upreplicate.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create and split the workload table.
//
Expand All @@ -400,7 +400,7 @@ func runCDCBenchWorkload(
t.L().Printf("creating table with %s ranges", humanize.Comma(numRanges))
c.Run(ctx, option.WithNodes(nWorkload), fmt.Sprintf(
`./cockroach workload init kv --splits %d {pgurl:%d}`, numRanges, nData[0]))
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// For read-only workloads, ingest some data. init --insert-count does not use
// the standard key generator that the read workload uses, so we have to write
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ func runCLINodeStatus(ctx context.Context, t test.Test, c cluster.Cluster) {
db := c.Conn(ctx, t.L(), 1)
defer db.Close()

err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)

lastWords := func(s string) []string {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cluster_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ func runClusterInit(ctx context.Context, t test.Test, c cluster.Cluster) {
fmt.Sprintf(`./cockroach init --insecure --port={pgport:%d}`, initNode))

// This will only succeed if 3 nodes joined the cluster.
err = WaitFor3XReplication(ctx, t, dbs[0])
err = WaitFor3XReplication(ctx, t, t.L(), dbs[0])
require.NoError(t, err)

execCLI := func(runNode int, extraArgs ...string) (string, error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cluster_to_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -879,7 +879,7 @@ func (rd *replicationDriver) main(ctx context.Context) {
// the probability that the producer returns a topology with more than one node in it,
// else the node shutdown tests can flake.
if rd.rs.srcNodes >= 3 {
require.NoError(rd.t, WaitFor3XReplication(ctx, rd.t, rd.setup.src.db))
require.NoError(rd.t, WaitFor3XReplication(ctx, rd.t, rd.t.L(), rd.setup.src.db))
}

rd.t.L().Printf("begin workload on src cluster")
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/roachtest/tests/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func runDrainAndDecommission(
run(`SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`)

// Wait for initial up-replication.
err := WaitForReplication(ctx, t, db, defaultReplicationFactor, atLeastReplicationFactor)
err := WaitForReplication(ctx, t, t.L(), db, defaultReplicationFactor, atLeastReplicationFactor)
require.NoError(t, err)
}

Expand Down Expand Up @@ -1030,7 +1030,7 @@ func runDecommissionDrains(ctx context.Context, t test.Test, c cluster.Cluster)
require.NoError(t, err)

// Wait for initial up-replication.
err = WaitFor3XReplication(ctx, t, db)
err = WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)
}

Expand Down Expand Up @@ -1121,7 +1121,7 @@ func runDecommissionSlow(ctx context.Context, t test.Test, c cluster.Cluster) {
run(db, `SET CLUSTER SETTING kv.snapshot_rebalance.max_rate='2GiB'`)

// Wait for initial up-replication.
err := WaitForReplication(ctx, t, db, replicationFactor, atLeastReplicationFactor)
err := WaitForReplication(ctx, t, t.L(), db, replicationFactor, atLeastReplicationFactor)
require.NoError(t, err)
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/decommissionbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,7 +395,7 @@ func setupDecommissionBench(
// import can saturate snapshots and leave underreplicated system ranges
// struggling.
// See GH issue #101532 for longer term solution.
if err := WaitForReplication(ctx, t, db, 3, atLeastReplicationFactor); err != nil {
if err := WaitForReplication(ctx, t, t.L(), db, 3, atLeastReplicationFactor); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -437,7 +437,7 @@ func setupDecommissionBench(
}

// Wait for initial up-replication.
err = WaitFor3XReplication(ctx, t, db)
err = WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/disk_full.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ func registerDiskFull(r registry.Registry) {
// a range on node 1, but node 1 will not restart until the query
// completes.
db := c.Conn(ctx, t.L(), 1)
err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)
_ = db.Close()

Expand Down
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 @@ -120,7 +120,7 @@ func runDiskStalledDetection(
require.NoError(t, n1Conn.PingContext(ctx))

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, n2conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), n2conn))

c.Run(ctx, option.WithNodes(c.Node(4)), `./cockroach workload init kv --splits 1000 {pgurl:1}`)

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/drain.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ func runClusterNotAtQuorum(ctx context.Context, t test.Test, c cluster.Cluster)
db := c.Conn(ctx, t.L(), 1)
defer func() { _ = db.Close() }()

err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)

stopOpts := option.DefaultStopOpts()
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func runEventLog(ctx context.Context, t test.Test, c cluster.Cluster) {
// a node starts and contacts the cluster.
db := c.Conn(ctx, t.L(), 1)
defer db.Close()
err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)

err = retry.ForDuration(10*time.Second, func() error {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/export_parquet.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ func registerExportParquet(r registry.Registry) {
ExtraSetupArgs: "--checks=false",
DisableDefaultScheduledBackup: true,
}
setupTPCC(ctx, t, c, tpccOpts)
setupTPCC(ctx, t, t.L(), c, tpccOpts)
t.Status("finished initializing tpcc database")

// Add padding to let the cluster metrics settle after initializing tpcc.
Expand Down Expand Up @@ -144,7 +144,7 @@ func registerExportParquet(r registry.Registry) {
ExtraSetupArgs: "--checks=false",
DisableDefaultScheduledBackup: true,
}
setupTPCC(ctx, t, c, tpccOpts)
setupTPCC(ctx, t, t.L(), c, tpccOpts)
t.Status("finished initializing tpcc database")

// Add padding to let the cluster metrics settle after initializing tpcc.
Expand Down
16 changes: 8 additions & 8 deletions pkg/cmd/roachtest/tests/failover.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,7 +234,7 @@ func runFailoverChaos(ctx context.Context, t test.Test, c cluster.Cluster, readO

// Wait for upreplication.
require.NoError(
t, WaitForReplication(ctx, t, conn, 5 /* replicationFactor */, atLeastReplicationFactor),
t, WaitForReplication(ctx, t, t.L(), conn, 5 /* replicationFactor */, atLeastReplicationFactor),
)

// Create the kv database. If this is a read-only workload, populate it with
Expand All @@ -257,7 +257,7 @@ func runFailoverChaos(ctx context.Context, t test.Test, c cluster.Cluster, readO

// Wait for upreplication of the new ranges.
require.NoError(
t, WaitForReplication(ctx, t, conn, 5 /* replicationFactor */, atLeastReplicationFactor),
t, WaitForReplication(ctx, t, t.L(), conn, 5 /* replicationFactor */, atLeastReplicationFactor),
)

// Run workload on n10 via n1-n2 gateways until test ends (context cancels).
Expand Down Expand Up @@ -406,7 +406,7 @@ func runFailoverPartialLeaseGateway(ctx context.Context, t test.Test, c cluster.
configureAllZones(t, ctx, conn, zoneConfig{replicas: 3, onlyNodes: []int{1, 2, 3}})

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the kv database with 5 replicas on n2-n6, and leases on n4.
t.L().Printf("creating workload database")
Expand Down Expand Up @@ -543,7 +543,7 @@ func runFailoverPartialLeaseLeader(ctx context.Context, t test.Test, c cluster.C

// NB: We want to ensure the system ranges are all down-replicated from their
// initial RF of 5, so pass in exactlyReplicationFactor below.
require.NoError(t, WaitForReplication(ctx, t, conn, 3, exactlyReplicationFactor))
require.NoError(t, WaitForReplication(ctx, t, t.L(), conn, 3, exactlyReplicationFactor))

// Now that system ranges are properly placed on n1-n3, start n4-n6.
c.Start(ctx, t.L(), opts, settings, c.Range(4, 6))
Expand Down Expand Up @@ -675,7 +675,7 @@ func runFailoverPartialLeaseLiveness(ctx context.Context, t test.Test, c cluster
replicas: 4, onlyNodes: []int{1, 2, 3, 4}, leasePreference: "[+node4]"})

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the kv database on n5-n7.
t.L().Printf("creating workload database")
Expand Down Expand Up @@ -791,7 +791,7 @@ func runFailoverNonSystem(
configureAllZones(t, ctx, conn, zoneConfig{replicas: 3, onlyNodes: []int{1, 2, 3}})

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the kv database, constrained to n4-n6. Despite the zone config, the
// ranges will initially be distributed across all cluster nodes.
Expand Down Expand Up @@ -903,7 +903,7 @@ func runFailoverLiveness(
configureZone(t, ctx, conn, `RANGE liveness`, zoneConfig{replicas: 4, leasePreference: "[+node4]"})

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the kv database, constrained to n1-n3. Despite the zone config, the
// ranges will initially be distributed across all cluster nodes.
Expand Down Expand Up @@ -1017,7 +1017,7 @@ func runFailoverSystemNonLiveness(
configureZone(t, ctx, conn, `RANGE liveness`, zoneConfig{replicas: 3, onlyNodes: []int{1, 2, 3}})

// Wait for upreplication.
require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))

// Create the kv database, constrained to n1-n3. Despite the zone config, the
// ranges will initially be distributed across all cluster nodes.
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func registerGossip(r registry.Registry) {
startOpts := option.DefaultStartOpts()
startOpts.RoachprodOpts.ExtraArgs = append(startOpts.RoachprodOpts.ExtraArgs, "--vmodule=*=1")
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.All())
err := WaitFor3XReplication(ctx, t, c.Conn(ctx, t.L(), 1))
err := WaitFor3XReplication(ctx, t, t.L(), c.Conn(ctx, t.L(), 1))
require.NoError(t, err)

gossipNetworkAccordingTo := func(node int) (nodes []int) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/inconsistency.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func runInconsistency(ctx context.Context, t test.Test, c cluster.Cluster) {
// to expect it.
_, err := db.ExecContext(ctx, `SET CLUSTER SETTING server.consistency_check.interval = '0'`)
require.NoError(t, err)
require.NoError(t, WaitFor3XReplication(ctx, t, db))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), db))
require.NoError(t, db.Close())
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/jobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func executeNodeShutdown(
// is in a healthy state before we start bringing any
// nodes down.
t.Status("waiting for cluster to be 3x replicated")
err := WaitFor3XReplication(ctx, t, watcherDB)
err := WaitFor3XReplication(ctx, t, t.L(), watcherDB)
if err != nil {
return err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/cmd/roachtest/tests/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -451,7 +451,7 @@ func registerKVQuiescenceDead(r registry.Registry) {
db := c.Conn(ctx, t.L(), 1)
defer db.Close()

err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)

qps := func(f func()) float64 {
Expand Down Expand Up @@ -533,7 +533,7 @@ func registerKVGracefulDraining(r registry.Registry) {
db2 := c.Conn(ctx, t.L(), 2)
defer db2.Close()

err := WaitFor3XReplication(ctx, t, db1)
err := WaitFor3XReplication(ctx, t, t.L(), db1)
require.NoError(t, err)

t.Status("initializing workload")
Expand Down Expand Up @@ -793,7 +793,7 @@ func registerKVRangeLookups(r registry.Registry) {
conns[i].Close()
}
}()
err := WaitFor3XReplication(ctx, t, conns[0])
err := WaitFor3XReplication(ctx, t, t.L(), conns[0])
require.NoError(t, err)

m := c.NewMonitor(ctx, c.Range(1, nodes))
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/tests/lease_preferences.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,13 +242,13 @@ func runLeasePreferences(
// Wait for the existing ranges (not kv) to be up-replicated. That way,
// creating the splits and waiting for up-replication on kv will be much
// quicker.
require.NoError(t, WaitForReplication(ctx, t, conn, spec.replFactor, atLeastReplicationFactor))
require.NoError(t, WaitForReplication(ctx, t, t.L(), conn, spec.replFactor, atLeastReplicationFactor))
c.Run(ctx, option.WithNodes(c.Node(numNodes)), fmt.Sprintf(
`./cockroach workload init kv --scatter --splits %d {pgurl:%d}`,
spec.ranges, numNodes))
// Wait for under-replicated ranges before checking lease preference
// enforcement.
require.NoError(t, WaitForReplication(ctx, t, conn, spec.replFactor, atLeastReplicationFactor))
require.NoError(t, WaitForReplication(ctx, t, t.L(), conn, spec.replFactor, atLeastReplicationFactor))

// Set a lease preference for the liveness range, to be on n5. This test
// would occasionally fail due to the liveness heartbeat failures, when the
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/limit_capacity.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func runLimitCapacity(ctx context.Context, t test.Test, c cluster.Cluster, cfg l
conn := c.Conn(ctx, t.L(), 1)
defer conn.Close()

require.NoError(t, WaitFor3XReplication(ctx, t, conn))
require.NoError(t, WaitFor3XReplication(ctx, t, t.L(), conn))
var cancels []func()

c.Run(ctx, option.WithNodes(appNode), "./cockroach workload init kv --splits=1000 {pgurl:1}")
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/many_splits.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func runManySplits(ctx context.Context, t test.Test, c cluster.Cluster) {
defer db.Close()

// Wait for up-replication then create many ranges.
err := WaitFor3XReplication(ctx, t, db)
err := WaitFor3XReplication(ctx, t, t.L(), db)
require.NoError(t, err)

m := c.NewMonitor(ctx, c.All())
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/mismatched_locality.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func runMismatchedLocalityTest(ctx context.Context, t test.Test, c cluster.Clust
// Verify that we can add and drop regions for the database. There's no longer
// any node with the old localities, but that's fine.
db = c.Conn(ctx, t.L(), 3)
if err := WaitFor3XReplication(ctx, t, db); err != nil {
if err := WaitFor3XReplication(ctx, t, t.L(), db); err != nil {
t.Fatal(err)
}
if _, err := db.Exec(`ALTER DATABASE defaultdb ADD REGION "venus";`); err != nil {
Expand Down
Loading

0 comments on commit 1fa1df0

Please sign in to comment.