-
Notifications
You must be signed in to change notification settings - Fork 4k
sql: support partial stats at extremes without specifying columns #127836
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
sql: support partial stats at extremes without specifying columns #127836
Conversation
DrewKimball
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 3 of 3 files at r1, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @michae2 and @Uzair5162)
pkg/sql/create_stats.go line 486 at r1 (raw file):
} // Only collect statistics on single columns that are prefixes of forwards
[nit] forwards -> forward
fb1ba9c to
e59b3ab
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball and @michae2)
pkg/sql/create_stats.go line 486 at r1 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
[nit]
forwards->forward
Done.
e59b3ab to
1be4b6a
Compare
michae2
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball and @Uzair5162)
-- commits line 10 at r3:
nit: another forwards -> forward
pkg/sql/create_stats.go line 732 at r3 (raw file):
singleColDetails.ColumnStats = []jobspb.CreateStatsDetails_ColStat{colStat} planCtx := dsp.NewPlanningCtx(ctx, innerEvalCtx, innerP, txn.KV(), FullDistribution) if err = dsp.planAndRunCreateStats(
Using separate distsql invocations for each index makes sense to me, but I'm still thinking through the ramifications. I'm trying to figure out if we would need a separate planner and/or transaction for each invocation.
pkg/sql/logictest/testdata/logic_test/distsql_stats line 3365 at r3 (raw file):
INDEX(b, c, d), INDEX(d), INVERTED INDEX (j)
More kinds of index to test:
- hash-sharded index like
INDEX (e) USING HASH - partial index like
INDEX (f) WHERE c > 5 - expression index like
INDEX ((g + 1))
It would also be nice to add a test with a LOCALITY REGIONAL BY ROW table. I can help with this.
|
Meant to say: awesome work!!!! |
michae2
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 1 of 3 files at r1, 1 of 2 files at r3.
Reviewable status:complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball and @Uzair5162)
pkg/sql/create_stats.go line 492 at r3 (raw file):
} if idx.NumKeyColumns() != 0 { colID := idx.GetKeyColumnID(0)
I think for an implicitly partitioned index (such as the primary index of a LOCALITY REGIONAL BY ROW table, or a hash-sharded index such as INDEX (e) USING HASH) we actually want to do something smarter, and collect stats over the first column after the partition key. But I think this will also require changes to this function to build the correct spans:
| func ConstructUsingExtremesSpans( |
So it's probably better to skip over indexes for which idx.IsSharded() or idx.ImplicitPartitioningColumnCount() > 0 are true.
pkg/sql/create_stats.go line 732 at r3 (raw file):
Previously, michae2 (Michael Erickson) wrote…
Using separate distsql invocations for each index makes sense to me, but I'm still thinking through the ramifications. I'm trying to figure out if we would need a separate planner and/or transaction for each invocation.
After thinking more, dsp.planAndRunCreateStats is synchronous, so these will run serially. Running serially in a single transaction is fine, and these should each be fairly short since they're each performing a constrained scan. So this looks good to me.
pkg/sql/distsql_plan_stats.go line 174 at r3 (raw file):
// The total expected number of rows is the same number that was measured // most recently, plus some overhead for possible insertions. float64(tableStats[0].RowCount) * (1 + overhead),
rowsExpected is used to estimate the overall progress of CREATE STATISTICS, which we report live as the job runs. I think this will be incorrect for CREATE STATISTICS USING EXTREMES in two ways: (a) this calculation assumes a full table scan, rather than a partial constrained scan, and (b) for this variant of USING EXTREMES a single job will consist of multiple invocations of planAndRunCreateStats instead of a single invocation, so this rowsExpected is only a portion of the total.
So for USING EXTREMES I have two ideas:
- we should probably use
RowCount * overheadinstead ofRowCount * (1 + overhead) - and then we should maybe either multiply this
rowsExpectedby the number of indexes we expect to scan, or maybe better: plumb a count of the number of indexes into sample_aggregator so that we can divide the reported progress here:fractionCompleted = float32(float64(rowsProcessed) / float64(s.spec.RowsExpected))
11ef209 to
baf418a
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball and @michae2)
Previously, michae2 (Michael Erickson) wrote…
nit: another forwards -> forward
Done.
pkg/sql/create_stats.go line 492 at r3 (raw file):
Previously, michae2 (Michael Erickson) wrote…
I think for an implicitly partitioned index (such as the primary index of a
LOCALITY REGIONAL BY ROWtable, or a hash-sharded index such asINDEX (e) USING HASH) we actually want to do something smarter, and collect stats over the first column after the partition key. But I think this will also require changes to this function to build the correct spans:
func ConstructUsingExtremesSpans( So it's probably better to skip over indexes for which
idx.IsSharded()oridx.ImplicitPartitioningColumnCount() > 0are true.
Done.
pkg/sql/distsql_plan_stats.go line 174 at r3 (raw file):
Previously, michae2 (Michael Erickson) wrote…
rowsExpectedis used to estimate the overall progress ofCREATE STATISTICS, which we report live as the job runs. I think this will be incorrect forCREATE STATISTICS USING EXTREMESin two ways: (a) this calculation assumes a full table scan, rather than a partial constrained scan, and (b) for this variant ofUSING EXTREMESa single job will consist of multiple invocations ofplanAndRunCreateStatsinstead of a single invocation, so thisrowsExpectedis only a portion of the total.So for
USING EXTREMESI have two ideas:
- we should probably use
RowCount * overheadinstead ofRowCount * (1 + overhead)- and then we should maybe either multiply this
rowsExpectedby the number of indexes we expect to scan, or maybe better: plumb a count of the number of indexes into sample_aggregator so that we can divide the reported progress here:
fractionCompleted = float32(float64(rowsProcessed) / float64(s.spec.RowsExpected))
Good catch! We also use rowsExpected in our sample size calculation so changing it to RowCount * overhead should also give us more appropriate sample sizes for partial stats collections.
pkg/sql/logictest/testdata/logic_test/distsql_stats line 3365 at r3 (raw file):
Previously, michae2 (Michael Erickson) wrote…
More kinds of index to test:
- hash-sharded index like
INDEX (e) USING HASH- partial index like
INDEX (f) WHERE c > 5- expression index like
INDEX ((g + 1))It would also be nice to add a test with a
LOCALITY REGIONAL BY ROWtable. I can help with this.
Done.
baf418a to
f203c5a
Compare
DrewKimball
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @michae2)
pkg/sql/create_stats.go line 492 at r3 (raw file):
Previously, Uzair5162 (Uzair Ahmad) wrote…
Done.
Hypothetically the optimizer should be able to figure out that it needs to scan each partition using the EXTREMES filter. If we don't handle this case here, we should open a tracking issue for it.
michae2
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice work, @Uzair5162! Just one last comment, and then this
Reviewed 7 of 8 files at r4, 1 of 1 files at r5, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball and @Uzair5162)
pkg/sql/create_stats.go line 492 at r3 (raw file):
Previously, DrewKimball (Drew Kimball) wrote…
Hypothetically the optimizer should be able to figure out that it needs to scan each partition using the EXTREMES filter. If we don't handle this case here, we should open a tracking issue for it.
@DrewKimball, these plans don't go through the optimizer; we construct physical plans directly in
cockroach/pkg/sql/distsql_plan_stats.go
Line 269 in 8a21ac3
| func (dsp *DistSQLPlanner) createPartialStatsPlan( |
I agree, we should open a GH issue about USING EXTREMES on hash-sharded indexes and RBR tables.
pkg/sql/rowexec/sample_aggregator.go line 274 at r5 (raw file):
// Adjust the fraction to account for the indexes that have already // been processed. fractionCompleted = (float32(s.spec.CurIndex) + fractionCompleted) / float32(s.spec.NumIndexes)
During an upgrade, if we're using the new binary we might receive a spec with NumIndexes=0 from one of the old binaries. We should handle this case by falling back to the old fractionCompleted logic.
f203c5a to
d5d528e
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @DrewKimball and @michae2)
pkg/sql/create_stats.go line 492 at r3 (raw file):
Previously, michae2 (Michael Erickson) wrote…
@DrewKimball, these plans don't go through the optimizer; we construct physical plans directly in
cockroach/pkg/sql/distsql_plan_stats.go
Line 269 in 8a21ac3
func (dsp *DistSQLPlanner) createPartialStatsPlan( I agree, we should open a GH issue about USING EXTREMES on hash-sharded indexes and RBR tables.
Opened: #128725
pkg/sql/rowexec/sample_aggregator.go line 274 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
During an upgrade, if we're using the new binary we might receive a spec with NumIndexes=0 from one of the old binaries. We should handle this case by falling back to the old fractionCompleted logic.
Done.
d5d528e to
ac26101
Compare
This commit adds support to collect partial statistics at extremes without specifying a column, such as: `CREATE STATISTICS my_stat FROM t USING EXTREMES` This will collect partial stats on a default set of columns, which are all single columns that are prefixes of forward indexes, excluding partial, sharded, and implicitly partitioned indexes. Modifies create stats job progress reporting to work when multiple indexes will be scanned as part of the stat collection. The i'th index scan will only report progress between `(i-1)/numIndexes`% and `i/numIndexes`%. For example, if we expect to scan 4 indexes, each index will report progress between 0%-25%, 25%-50%, 50%-75%, and 75%-100% respectively. Also changes the `rowsExpected` count used by the SampleAggregator and sample size computation to be the estimated number of stale rows computed as `estimated row count * fraction of stale rows` for partial stat collections. Closes: cockroachdb#127832 Release note (sql): Partial stats at extremes can now be collected on all valid columns of a table using the `CREATE STATISTICS <stat_name>` `FROM <table_name> USING EXTREMES` syntax, without an `ON <col_name>` clause. Valid columns are all single column prefixes of a forward index excluding partial, sharded, and implicitly partitioned indexes.
ac26101 to
414f226
Compare
|
bors r+ |
|
Build failed (retrying...): |
This commit reverts a change in cockroachdb#127836 that changed the error returned when multiple stats are requested at once from a pgerror to a failed assertion. This change caused `GenerateAndCheckRedactedExplainsForPII()` to fail when generating SQL commands for partial stats collections with multiple columns, which should be ignored like other pgerrors. Fixes: cockroachdb#128848 Release note: None
This commit reverts a change in cockroachdb#127836 that changed the error returned when multiple stats are requested at once from a pgerror to a failed assertion. This change caused `GenerateAndCheckRedactedExplainsForPII()` to fail when generating SQL commands for certain explains on partial stats collections with multiple columns, which is unsupported and should be ignored like other pgerrors. Closes: cockroachdb#128848 Release note: None
128862: sql: revert multiple partial stats requests error to a pgerror r=Uzair5162 a=Uzair5162 This commit reverts a change in #127836 that changed the error returned when multiple stats are requested at once from a pgerror to a failed assertion. This change caused `GenerateAndCheckRedactedExplainsForPII()` to fail when generating SQL commands for certain explains on partial stats collections with multiple columns, which is unsupported and should be ignored like other pgerrors. Fixes: #128848 Release note: None Co-authored-by: Uzair Ahmad <uzair.ahmad@cockroachlabs.com>
This commit adds support to collect partial statistics at extremes without specifying a column, such as:
CREATE STATISTICS my_stat FROM t USING EXTREMESThis will collect partial stats on a default set of columns, which are all single columns that are prefixes of forward indexes, excluding partial, sharded, and implicitly partitioned indexes.
Modifies create stats job progress reporting to work when multiple indexes will be scanned as part of the stat collection. The i'th index scan will only report progress between
(i-1)/numIndexes% andi/numIndexes%. For example, if we expect to scan 4 indexes, each index will report progress between 0%-25%, 25%-50%, 50%-75%, and 75%-100% respectively. Also changes therowsExpectedcount used by the SampleAggregator and sample size computation to be the estimated number of stale rows computed asestimated row count * fraction of stale rowsfor partial stat collections.Closes: #127832
Release note (sql): Partial stats at extremes can now be collected on all valid columns of a table using the
CREATE STATISTICS <stat_name> FROM <table_name> USING EXTREMESsyntax, without anON <col_name>clause. Valid columns are all single column prefixes of a forward index excluding partial, sharded, and implicitly partitioned indexes.