Skip to content

Commit 7fff0b2

Browse files
committed
sql: inform vtable generator of limits
In some cases, generators may be able to deal with a LIMIT clause themselves, saving a good deal of wasted work. I haven't plumbed this into virtual index populate because at the moment we only ever use virtual indexes for point lookups. It may be useful to add this into the populate callback for virtual tables, but that produces a lot of code churn without a user yet so I've skipped that as well. Epic: none Release note: None
1 parent c84aa28 commit 7fff0b2

File tree

3 files changed

+24
-17
lines changed

3 files changed

+24
-17
lines changed

pkg/sql/crdb_internal.go

Lines changed: 17 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -708,7 +708,7 @@ CREATE TABLE crdb_internal.pg_catalog_table_is_implemented (
708708
name STRING NOT NULL,
709709
implemented BOOL
710710
)`,
711-
generator: func(ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
711+
generator: func(ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
712712
row := make(tree.Datums, 2)
713713
worker := func(ctx context.Context, pusher rowPusher) error {
714714
addDesc := func(table *virtualDefEntry, dbName tree.Datum, scName string) error {
@@ -3851,7 +3851,7 @@ CREATE TABLE crdb_internal.table_columns (
38513851
hidden BOOL NOT NULL
38523852
)
38533853
`,
3854-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
3854+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
38553855
const numDatums = 8
38563856
row := make(tree.Datums, numDatums)
38573857
worker := func(ctx context.Context, pusher rowPusher) error {
@@ -3920,7 +3920,7 @@ CREATE TABLE crdb_internal.table_indexes (
39203920
create_statement STRING NOT NULL
39213921
)
39223922
`,
3923-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
3923+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
39243924
primary := tree.NewDString("primary")
39253925
secondary := tree.NewDString("secondary")
39263926
const numDatums = 13
@@ -4489,7 +4489,7 @@ CREATE TABLE crdb_internal.ranges_no_leases (
44894489
)
44904490
`,
44914491
resultColumns: colinfo.RangesNoLeases,
4492-
generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
4492+
generator: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, limit int64, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
44934493
hasAdmin, err := p.HasAdminRole(ctx)
44944494
if err != nil {
44954495
return nil, nil, err
@@ -4524,7 +4524,11 @@ CREATE TABLE crdb_internal.ranges_no_leases (
45244524
}
45254525

45264526
execCfg := p.ExecCfg()
4527-
rangeDescIterator, err := execCfg.RangeDescIteratorFactory.NewIterator(ctx, execCfg.Codec.TenantSpan())
4527+
pageSize := int64(128)
4528+
if limit > 0 {
4529+
pageSize = min(limit, pageSize)
4530+
}
4531+
rangeDescIterator, err := execCfg.RangeDescIteratorFactory.NewLazyIterator(ctx, execCfg.Codec.TenantSpan(), int(pageSize))
45284532
if err != nil {
45294533
return nil, nil, err
45304534
}
@@ -5428,7 +5432,7 @@ CREATE TABLE crdb_internal.partitions (
54285432
subzone_id INT -- references a subzone id in the crdb_internal.zones table
54295433
)
54305434
`,
5431-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
5435+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
54325436
dbName := ""
54335437
if dbContext != nil {
54345438
dbName = dbContext.GetName()
@@ -6874,7 +6878,7 @@ CREATE TABLE crdb_internal.index_usage_statistics (
68746878
total_reads INT NOT NULL,
68756879
last_read TIMESTAMPTZ
68766880
);`,
6877-
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
6881+
generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
68786882
// Perform RPC Fanout.
68796883
stats, err :=
68806884
p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{})
@@ -6939,7 +6943,7 @@ CREATE TABLE crdb_internal.cluster_statement_statistics (
69396943
aggregation_interval INTERVAL NOT NULL,
69406944
index_recommendations STRING[] NOT NULL
69416945
);`,
6942-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
6946+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
69436947
// TODO(azhng): we want to eventually implement memory accounting within the
69446948
// RPC handlers. See #69032.
69456949
acc := p.Mon().MakeBoundAccount()
@@ -7370,7 +7374,7 @@ CREATE TABLE crdb_internal.cluster_transaction_statistics (
73707374
statistics JSONB NOT NULL,
73717375
aggregation_interval INTERVAL NOT NULL
73727376
);`,
7373-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7377+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
73747378
// TODO(azhng): we want to eventually implement memory accounting within the
73757379
// RPC handlers. See #69032.
73767380
acc := p.Mon().MakeBoundAccount()
@@ -7691,7 +7695,7 @@ CREATE TABLE crdb_internal.transaction_contention_events (
76917695
index_name STRING,
76927696
contention_type STRING NOT NULL
76937697
);`,
7694-
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7698+
generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
76957699
// Check permission first before making RPC fanout.
76967700
// If a user has VIEWACTIVITYREDACTED role option but the user does not
76977701
// have the ADMIN role option, then the contending key should be redacted.
@@ -7970,8 +7974,8 @@ type clusterLocksFilters struct {
79707974

79717975
func genClusterLocksGenerator(
79727976
filters clusterLocksFilters,
7973-
) func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7974-
return func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7977+
) func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
7978+
return func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, _ int64, _ *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) {
79757979
hasAdmin, err := p.HasAdminRole(ctx)
79767980
if err != nil {
79777981
return nil, nil, err
@@ -8234,7 +8238,7 @@ func populateClusterLocksWithFilter(
82348238
) (matched bool, err error) {
82358239
var rowGenerator virtualTableGenerator
82368240
generator := genClusterLocksGenerator(filters)
8237-
rowGenerator, _, err = generator(ctx, p, db, nil /* stopper */)
8241+
rowGenerator, _, err = generator(ctx, p, db, 0 /* limit */, nil /* stopper */)
82388242
if err != nil {
82398243
return false, err
82408244
}

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)