Skip to content

Commit 36e539b

Browse files
craig[bot]stevendannamsbutler
committed
153807: sql: inform vtable generator of limits r=stevendanna a=stevendanna In some cases, generators may be able to deal with a LIMIT clause themselves, saving a good deal of wasted work. Epic: none Release note: None 154154: roachtest: ensure backup-restore workloads get positive seed r=darrylwong a=msbutler As of #153615, the seed passed to the workloads could be a negative integer, but the workload command cannot take a negative seed, causing the workload and the test to fail. This patch ensures negative seeds aren't passed to workloads. Fixes #152498 Release note: none Co-authored-by: Steven Danna <[email protected]> Co-authored-by: Michael Butler <[email protected]>
3 parents 541b9d3 + 7fff0b2 + bd31724 commit 36e539b

File tree

6 files changed

+38
-25
lines changed

6 files changed

+38
-25
lines changed

pkg/cmd/roachtest/tests/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -241,7 +241,6 @@ go_library(
241241
"//pkg/cmd/roachtest/grafana",
242242
"//pkg/cmd/roachtest/option",
243243
"//pkg/cmd/roachtest/registry",
244-
"//pkg/cmd/roachtest/roachtestflags",
245244
"//pkg/cmd/roachtest/roachtestutil",
246245
"//pkg/cmd/roachtest/roachtestutil/clusterupgrade",
247246
"//pkg/cmd/roachtest/roachtestutil/mixedversion",

pkg/cmd/roachtest/tests/backup_restore_roundtrip.go

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -134,7 +134,11 @@ func backupRestoreRoundTrip(
134134
) {
135135
pauseProbability := 0.2
136136
testRNG, seed := randutil.NewLockedPseudoRand()
137-
t.L().Printf("random seed: %d", seed)
137+
138+
// Workload can only take a positive int as a seed, but seed could be a
139+
// negative int. Ensure the seed passed to workload is an int.
140+
workloadSeed := testRNG.Int63()
141+
t.L().Printf("random seed: %d; workload seed: %d", seed, workloadSeed)
138142

139143
envOption := install.EnvOption([]string{
140144
"COCKROACH_MIN_RANGE_MAX_BYTES=1",
@@ -157,7 +161,7 @@ func backupRestoreRoundTrip(
157161

158162
dbs := []string{"bank", "tpcc", schemaChangeDB}
159163
d, runBackgroundWorkload, _, err := createDriversForBackupRestore(
160-
ctx, t, c, m, testRNG, seed, testUtils, dbs,
164+
ctx, t, c, m, testRNG, workloadSeed, testUtils, dbs,
161165
)
162166
if err != nil {
163167
return err
@@ -421,12 +425,12 @@ func createDriversForBackupRestore(
421425
c cluster.Cluster,
422426
m cluster.Monitor,
423427
rng *rand.Rand,
424-
seed int64,
428+
workloadSeed int64,
425429
testUtils *CommonTestUtils,
426430
dbs []string,
427431
) (*BackupRestoreTestDriver, func() (func(), error), [][]string, error) {
428432
runBackgroundWorkload, err := startBackgroundWorkloads(
429-
ctx, t.L(), c, m, rng, seed, c.CRDBNodes(), c.WorkloadNode(), testUtils, dbs,
433+
ctx, t.L(), c, m, rng, workloadSeed, c.CRDBNodes(), c.WorkloadNode(), testUtils, dbs,
430434
)
431435
if err != nil {
432436
return nil, nil, nil, err

pkg/cmd/roachtest/tests/mixed_version_backup.go

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@ import (
3030
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
3131
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
3232
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
33-
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestflags"
3433
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
3534
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/clusterupgrade"
3635
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/mixedversion"
@@ -2934,6 +2933,10 @@ func registerBackupMixedVersion(r registry.Registry) {
29342933
)
29352934
testRNG := mvt.RNG()
29362935

2936+
// Workload can only take a positive int as a seed, but seed could be a
2937+
// negative int. Ensure the seed passed to workload is an int.
2938+
workloadSeed := testRNG.Int63()
2939+
29372940
dbs := []string{"bank", "tpcc"}
29382941
backupTest, err := newMixedVersionBackup(t, c, c.CRDBNodes(), dbs)
29392942
if err != nil {
@@ -2951,8 +2954,8 @@ func registerBackupMixedVersion(r registry.Registry) {
29512954
// for the cluster used in this test without overloading it,
29522955
// which can make the backups take much longer to finish.
29532956
const numWarehouses = 100
2954-
bankInit, bankRun := bankWorkloadCmd(t.L(), testRNG, roachtestflags.GlobalSeed, c.CRDBNodes(), false)
2955-
tpccInit, tpccRun := tpccWorkloadCmd(t.L(), testRNG, roachtestflags.GlobalSeed, numWarehouses, c.CRDBNodes())
2957+
bankInit, bankRun := bankWorkloadCmd(t.L(), testRNG, workloadSeed, c.CRDBNodes(), false)
2958+
tpccInit, tpccRun := tpccWorkloadCmd(t.L(), testRNG, workloadSeed, numWarehouses, c.CRDBNodes())
29562959

29572960
mvt.OnStartup("set short job interval", backupTest.setShortJobIntervals)
29582961
mvt.OnStartup("take backup in previous version", backupTest.maybeTakePreviousVersionBackup)

pkg/sql/crdb_internal.go

Lines changed: 17 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -709,7 +709,7 @@ CREATE TABLE crdb_internal.pg_catalog_table_is_implemented (
709709
name STRING NOT NULL,
710710
implemented BOOL
711711
)`,
712-
generator: func(ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
712+
generator: func(ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
713713
row := make(tree.Datums, 2)
714714
worker := func(ctx context.Context, pusher rowPusher) error {
715715
addDesc := func(table *virtualDefEntry, dbName tree.Datum, scName string) error {
@@ -3852,7 +3852,7 @@ CREATE TABLE crdb_internal.table_columns (
38523852
hidden BOOL NOT NULL
38533853
)
38543854
`,
3855-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
3855+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
38563856
const numDatums = 8
38573857
row := make(tree.Datums, numDatums)
38583858
worker := func(ctx context.Context, pusher rowPusher) error {
@@ -3921,7 +3921,7 @@ CREATE TABLE crdb_internal.table_indexes (
39213921
create_statement STRING NOT NULL
39223922
)
39233923
`,
3924-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
3924+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
39253925
primary := tree.NewDString("primary")
39263926
secondary := tree.NewDString("secondary")
39273927
const numDatums = 13
@@ -4490,7 +4490,7 @@ CREATE TABLE crdb_internal.ranges_no_leases (
44904490
)
44914491
`,
44924492
resultColumns: colinfo.RangesNoLeases,
4493-
generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
4493+
generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, limit int64, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
44944494
hasAdmin, err := p.HasAdminRole(ctx)
44954495
if err != nil {
44964496
return nil, nil, err
@@ -4525,7 +4525,11 @@ CREATE TABLE crdb_internal.ranges_no_leases (
45254525
}
45264526

45274527
execCfg := p.ExecCfg()
4528-
rangeDescIterator, err := execCfg.RangeDescIteratorFactory.NewIterator(ctx, execCfg.Codec.TenantSpan())
4528+
pageSize := int64(128)
4529+
if limit > 0 {
4530+
pageSize = min(limit, pageSize)
4531+
}
4532+
rangeDescIterator, err := execCfg.RangeDescIteratorFactory.NewLazyIterator(ctx, execCfg.Codec.TenantSpan(), int(pageSize))
45294533
if err != nil {
45304534
return nil, nil, err
45314535
}
@@ -5429,7 +5433,7 @@ CREATE TABLE crdb_internal.partitions (
54295433
subzone_id INT -- references a subzone id in the crdb_internal.zones table
54305434
)
54315435
`,
5432-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
5436+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
54335437
dbName := ""
54345438
if dbContext != nil {
54355439
dbName = dbContext.GetName()
@@ -6875,7 +6879,7 @@ CREATE TABLE crdb_internal.index_usage_statistics (
68756879
total_reads INT NOT NULL,
68766880
last_read TIMESTAMPTZ
68776881
);`,
6878-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
6882+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
68796883
// Perform RPC Fanout.
68806884
stats, err :=
68816885
p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{})
@@ -6940,7 +6944,7 @@ CREATE TABLE crdb_internal.cluster_statement_statistics (
69406944
aggregation_interval INTERVAL NOT NULL,
69416945
index_recommendations STRING[] NOT NULL
69426946
);`,
6943-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
6947+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
69446948
// TODO(azhng): we want to eventually implement memory accounting within the
69456949
// RPC handlers. See #69032.
69466950
acc := p.Mon().MakeBoundAccount()
@@ -7371,7 +7375,7 @@ CREATE TABLE crdb_internal.cluster_transaction_statistics (
73717375
statistics JSONB NOT NULL,
73727376
aggregation_interval INTERVAL NOT NULL
73737377
);`,
7374-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7378+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
73757379
// TODO(azhng): we want to eventually implement memory accounting within the
73767380
// RPC handlers. See #69032.
73777381
acc := p.Mon().MakeBoundAccount()
@@ -7692,7 +7696,7 @@ CREATE TABLE crdb_internal.transaction_contention_events (
76927696
index_name STRING,
76937697
contention_type STRING NOT NULL
76947698
);`,
7695-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7699+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
76967700
// Check permission first before making RPC fanout.
76977701
// If a user has VIEWACTIVITYREDACTED role option but the user does not
76987702
// have the ADMIN role option, then the contending key should be redacted.
@@ -7971,8 +7975,8 @@ type clusterLocksFilters struct {
79717975

79727976
func genClusterLocksGenerator(
79737977
filters clusterLocksFilters,
7974-
) func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7975-
return func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7978+
) func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7979+
return func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ int64, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
79767980
hasAdmin, err := p.HasAdminRole(ctx)
79777981
if err != nil {
79787982
return nil, nil, err
@@ -8235,7 +8239,7 @@ func populateClusterLocksWithFilter(
82358239
) (matched bool, err error) {
82368240
var rowGenerator virtualTableGenerator
82378241
generator := genClusterLocksGenerator(filters)
8238-
rowGenerator, _, err = generator(ctx, p, db, nil /* stopper */)
8242+
rowGenerator, _, err = generator(ctx, p, db, 0 /* limit */, nil /* stopper */)
82398243
if err != nil {
82408244
return false, err
82418245
}

pkg/sql/exec_factory_util.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -268,7 +268,7 @@ func constructVirtualScan(
268268
}
269269
idx := index.(*optVirtualIndex).idx
270270
columns, constructor := virtual.getPlanInfo(
271-
table.(*optVirtualTable).desc, idx, params.IndexConstraint, p.execCfg.Stopper,
271+
table.(*optVirtualTable).desc, idx, params, p.execCfg.Stopper,
272272
)
273273

274274
n, err := delayedNodeCallback(&delayedNode{

pkg/sql/virtual_schema.go

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,9 @@ type virtualSchemaTable struct {
129129
// generator, if non-nil, is a function that is used when creating a
130130
// virtualTableNode. This function returns a virtualTableGenerator function
131131
// which generates the next row of the virtual table when called.
132-
generator func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error)
132+
//
133+
// Limit is the hard limit of the scan, a limit of 0 means no limit.
134+
generator func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, limit int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error)
133135

134136
// unimplemented indicates that we do not yet implement the contents of this
135137
// table. If the stub_catalog_tables session variable is enabled, the table
@@ -634,10 +636,11 @@ func (e *virtualDefEntry) validateRow(datums tree.Datums, columns colinfo.Result
634636
func (e *virtualDefEntry) getPlanInfo(
635637
table catalog.TableDescriptor,
636638
index catalog.Index,
637-
idxConstraint *constraint.Constraint,
639+
scanParams exec.ScanParams,
638640
stopper *stop.Stopper,
639641
) (colinfo.ResultColumns, virtualTableConstructor) {
640642
var columns colinfo.ResultColumns
643+
idxConstraint := scanParams.IndexConstraint
641644
for _, col := range e.desc.PublicColumns() {
642645
columns = append(columns, colinfo.ResultColumn{
643646
Name: col.GetName(),
@@ -668,7 +671,7 @@ func (e *virtualDefEntry) getPlanInfo(
668671
}
669672

670673
if def.generator != nil && !def.preferIndexOverGenerator(ctx, p, index, idxConstraint) {
671-
next, cleanup, err := def.generator(ctx, p, dbDesc, stopper)
674+
next, cleanup, err := def.generator(ctx, p, dbDesc, scanParams.HardLimit, stopper)
672675
if err != nil {
673676
return nil, err
674677
}

0 commit comments

Comments
 (0)