sql: add support for collecting partial statistics automatically#93067
sql: add support for collecting partial statistics automatically#93067craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
88d4c2d to
49d9cba
Compare
79d3738 to
170d5f8
Compare
810eea8 to
a7ad2f3
Compare
19734c7 to
0f38b4f
Compare
0f38b4f to
152a2f6
Compare
michae2
left a comment
There was a problem hiding this comment.
Thanks for dusting this off, @Uzair5162! (And retroactive thanks, @faizaanmadhani! Your work is not forgotten! 🙂) It's looking pretty good. I'm still reading, but have some initial comments.
Reviewed 10 of 31 files at r3, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, and @faizaanmadhani)
pkg/sql/create_stats.go line 222 at r5 (raw file):
} if n.Options.UsingExtremes && !n.p.SessionData().EnableCreateStatsUsingExtremes && n.Name.String() != jobspb.AutoPartialStatsName {
I think the check for n.Options.UsingExtremes here should be enough. We shouldn't need to check the name.
pkg/sql/create_stats.go line 741 at r5 (raw file):
len(details.ColumnStats), i, ); err != nil { retErr = errors.CombineErrors(retErr, err)
Is the idea that we should make a best-effort attempt to collect the rest of the partial stats even if one fails?
I think this might not interact correctly with cancellation of the CREATE STATISTICS: if the error is ContextCanceled, we should fail right away. (That reminds me: it would probably be a good idea to check ctx.Err() at the top of this loop.)
I'm also not sure this is going to interact correctly with the single transaction within which all of these distsql invocations are running. The transaction could be dead, for example, in which case all subsequent distsql invocations will fail, and we'll retry the entire thing anyway.
pkg/sql/catalog/catpb/catalog.go line 73 at r5 (raw file):
// AutoPartialStatsEnabledTableSettingName is the name of the automatic // partial stats collection enabled table setting. AutoPartialStatsEnabledTableSettingName = "sql_partial_stats_automatic_collection_enabled"
To be consistent with the other per-table settings, I think we should make these names the same as the matching cluster setting names but with periods replaced with underscores.
pkg/sql/stats/automatic_stats.go line 144 at r5 (raw file):
catpb.AutoPartialStatsMinStaleSettingName, "target minimum number of stale rows per table that will trigger a partial statistics refresh", 500,
It might be worth making this smaller than 500? 100?
pkg/sql/stats/automatic_stats.go line 392 at r5 (raw file):
if desc == nil { // If the descriptor could not be accessed, defer to the cluster setting. return AutomaticPartialStatisticsClusterMode.Get(&r.st.SV)
I think we should require both sql.stats.automatic_partial_collection.enabled and sql.stats.automatic_collection.enabled to be true in order to automatically collect partial stats, because (a) partial stats don't work without full stats anyway, and (b) that will preserve the behavior of SET CLUSTER SETTING sql.stats.automatic_collection.enabled = off; disabling all automatic stats collection, which is relied upon by many tests (and customers).
And same with table settings. So the precedence hierarchy would be:
- per-table auto full
- per-table auto partial
- cluster auto full
- cluster auto partial
In other words, I think sql.stats.automatic_partial_collection.enabled should only describe what happens when automatic stats are already enabled.
pkg/sql/stats/automatic_stats.go line 986 at r5 (raw file):
if isPartial { stmt = fmt.Sprintf( "CREATE STATISTICS %s FROM [%d] WITH OPTIONS THROTTLING %g AS OF SYSTEM TIME '-%s' USING EXTREMES",
nit: Instead of duplicating the call to Sprintf it might be nice to add %s to the end of the original format string which takes " USING EXTREMES" if isPartial else "".
c4b06f5 to
b3fe7f7
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, and @michae2)
pkg/sql/create_stats.go line 222 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
I think the check for
n.Options.UsingExtremeshere should be enough. We shouldn't need to check the name.
Done. Does this work because we use session settings' default values in job runs?
pkg/sql/create_stats.go line 741 at r5 (raw file):
Is the idea that we should make a best-effort attempt to collect the rest of the partial stats even if one fails?
That was my thinking, yeah. Partial stats will fail if prior full stats don't exist on the column and I didn't want that to prevent stat collections on other columns that have existing stats. Now that I think about it though, this is unlikely to happen for auto partial stats since the default column set for these collections is a subset of the default auto full stats column set, so this is probably unnecessary.
pkg/sql/catalog/catpb/catalog.go line 73 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
To be consistent with the other per-table settings, I think we should make these names the same as the matching cluster setting names but with periods replaced with underscores.
Done.
pkg/sql/stats/automatic_stats.go line 144 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
It might be worth making this smaller than 500? 100?
Done, I agree!
pkg/sql/stats/automatic_stats.go line 392 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
I think we should require both
sql.stats.automatic_partial_collection.enabledandsql.stats.automatic_collection.enabledto be true in order to automatically collect partial stats, because (a) partial stats don't work without full stats anyway, and (b) that will preserve the behavior ofSET CLUSTER SETTING sql.stats.automatic_collection.enabled = off;disabling all automatic stats collection, which is relied upon by many tests (and customers).And same with table settings. So the precedence hierarchy would be:
- per-table auto full
- per-table auto partial
- cluster auto full
- cluster auto partial
In other words, I think
sql.stats.automatic_partial_collection.enabledshould only describe what happens when automatic stats are already enabled.
This was my thinking as well and I think this is the current behavior. We check if auto full stats are enabled in automatic_stats.go#L793 after mutations, and early return if they're not (either (sql_stats_automatic_partial_collection_enabled table setting is unset and the sql.stats.automatic_partial_collection.enabled cluster setting is false) or (sql_stats_automatic_partial_collection_enabled table setting is false)).
We also check if auto full stats are enabled again before calling maybeRefreshStats() (which has the auto partial stat logic) in automatic_stats.go#L592 in case settings change between the mutation notification and the refresher picking it up.
The logic test in distsql_automatic_partial_stats#L5-L26 should confirm this where auto partial stats are enable for the table but auto full stats are disabled for the cluster, so auto partial stats are not collected.
Do you think it would be a good idea to add another check for auto full stats being enabled in autoPartialStatsEnabled() to make this more clear?
pkg/sql/stats/automatic_stats.go line 986 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: Instead of duplicating the call to
Sprintfit might be nice to add %s to the end of the original format string which takes" USING EXTREMES"ifisPartialelse"".
Done.
b3fe7f7 to
e2f9f44
Compare
michae2
left a comment
There was a problem hiding this comment.
Awesome work! This is very close. Main thing I'm worried about is the error handling in that loop in makeJobRecord.
Reviewed 15 of 31 files at r3, 1 of 5 files at r5, 5 of 6 files at r6, 2 of 2 files at r7, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, @faizaanmadhani, and @Uzair5162)
-- commits line 9 at r7:
nit: looks like the per-table setting names need to be updated here
(Also, you might consider moving the details about the settings to the release note.)
-- commits line 39 at r7:
@rytaft do you have any objection to multiple automatic partial stats collections at once? It seems like a good way to start trying out multiple concurrent stats jobs?
I remember we discussed this before in:
- #118983
- some slack conversations I can't seem to find now
pkg/sql/create_stats.go line 222 at r5 (raw file):
Previously, Uzair5162 (Uzair Ahmad) wrote…
Done. Does this work because we use session settings' default values in job runs?
For auto stats, these session variables come from the internal executor when the stats.Refresher runs CREATE STATISTICS. We have a way to set session variables for the internal executor, too, so I think it would be nice if CREATE STATISTICS respected the enable_create_stats_using_extremes session variable regardless of whether it was run manually or automatically.
In other words, I think SET CLUSTER SETTING sql.internal_executor.session_overrides = 'enable_create_stats_using_extremes=off'; should prevent automatic collection of partial stats (even if sql.stats.automatic_partial_collection.enabled is on).
pkg/sql/create_stats.go line 741 at r5 (raw file):
Partial stats will fail if prior full stats don't exist on the column and I didn't want that to prevent stat collections on other columns that have existing stats.
Could we protect against that case by changing distsql_plan_stats.go to skip over columns that don't have a prior full stat if this is an automatic collection?
I think simpler behavior here (failing the entire partial stats collection) will lead to fewer surprises, both here and in the optimizer when using these partial stats.
pkg/sql/catalog/catpb/catalog.go line 73 at r5 (raw file):
Previously, Uzair5162 (Uzair Ahmad) wrote…
Done.
Thanks!
pkg/sql/stats/automatic_stats.go line 392 at r5 (raw file):
Thank you for explaining!
Do you think it would be a good idea to add another check for auto full stats being enabled in autoPartialStatsEnabled() to make this more clear?
No, it's fine the way you have it. I just didn't read all of the code.
pkg/sql/stats/automatic_stats.go line 866 at r7 (raw file):
var rowCount float64 mustRefresh := false mustRefreshPartial := false
bikeshedding: The purpose of this is a little different than mustRefresh, maybe isPartial would be better?
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 686 at r7 (raw file):
return nil } return errors.Newf("unable to set table setting %s", key)
nit: This could probably be errors.AssertionFailedf.
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 712 at r7 (raw file):
return nil } return errors.Newf("unable to set table setting %s", key)
nit: could also be errors.AssertionFailedf
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 740 at r7 (raw file):
return nil } return errors.Newf("unable to set table setting %s", key)
nit: could also be errors.AssertionFailedf
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 771 at r7 (raw file):
return nil } return errors.Newf("unable to reset table setting %s", key)
nit: this could also become errors.AssertionFailedf
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 38 at r7 (raw file):
# not triggered. statement ok SET CLUSTER SETTING sql.stats.automatic_collection.min_stale_rows = 2147483647
nit: might be safer to do this before turning on sql.stats.automatic_collection.enabled
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 90 at r7 (raw file):
ALTER TABLE data SET (sql_stats_automatic_partial_collection_enabled = true) # Insert enough data to guarantee an partial stats collection.
typo: s/an/a/g
e2f9f44 to
2fc4182
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, @faizaanmadhani, and @michae2)
Previously, michae2 (Michael Erickson) wrote…
nit: looks like the per-table setting names need to be updated here
(Also, you might consider moving the details about the settings to the release note.)
Done.
pkg/sql/create_stats.go line 222 at r5 (raw file):
Previously, michae2 (Michael Erickson) wrote…
For auto stats, these session variables come from the internal executor when the stats.Refresher runs CREATE STATISTICS. We have a way to set session variables for the internal executor, too, so I think it would be nice if CREATE STATISTICS respected the
enable_create_stats_using_extremessession variable regardless of whether it was run manually or automatically.In other words, I think
SET CLUSTER SETTING sql.internal_executor.session_overrides = 'enable_create_stats_using_extremes=off';should prevent automatic collection of partial stats (even ifsql.stats.automatic_partial_collection.enabledis on).
Interesting! I've added a test for this here.
pkg/sql/create_stats.go line 741 at r5 (raw file):
I think simpler behavior here (failing the entire partial stats collection) will lead to fewer surprises, both here and in the optimizer when using these partial stats.
I agree, let's keep it as is.
pkg/sql/stats/automatic_stats.go line 866 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
bikeshedding: The purpose of this is a little different than
mustRefresh, maybeisPartialwould be better?
Agree, renamed to isPartial.
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 686 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: This could probably be
errors.AssertionFailedf.
Done.
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 712 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: could also be
errors.AssertionFailedf
Done.
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 740 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: could also be
errors.AssertionFailedf
Done.
pkg/sql/storageparam/tablestorageparam/table_storage_param.go line 771 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: this could also become
errors.AssertionFailedf
Done.
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 38 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
nit: might be safer to do this before turning on
sql.stats.automatic_collection.enabled
Good point, done.
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 90 at r7 (raw file):
Previously, michae2 (Michael Erickson) wrote…
typo: s/an/a/g
Fixed.
michae2
left a comment
There was a problem hiding this comment.
After addressing @rytaft's concerns, this
Nice work bringing this back to life!!
Reviewed 1 of 5 files at r5, 1 of 6 files at r6, 3 of 3 files at r8, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @DrewKimball, @dt, and @faizaanmadhani)
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 89 at r8 (raw file):
# setting instead statement ok SET CLUSTER SETTING sql.internal_executor.session_overrides = 'EnableCreateStatsUsingExtremes=false';
Nice!
pkg/sql/stats/create_stats_job_test.go line 404 at r8 (raw file):
allowRequest = make(chan struct{}) initialFullStatErrCh := make(chan error) go func() {
speculation: You might not need the goroutine here if you close(allowRequest) first? I'm not 100% sure, though.
pkg/sql/stats/create_stats_job_test.go line 451 at r8 (raw file):
} <-runningAutoPartialStatErrCh
Should we also check that this is nil?
pkg/sql/stats/create_stats_job_test.go line 489 at r8 (raw file):
} <-runningAutoPartialStatErrCh
Same question here
2fc4182 to
78a0852
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Thanks for the review @michae2 :)
I've changed it now so that auto partial stats jobs can't run concurrently on the same table, but they can still run at the same time as any other stats job.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball, @dt, @faizaanmadhani, and @michae2)
pkg/sql/stats/create_stats_job_test.go line 404 at r8 (raw file):
Previously, michae2 (Michael Erickson) wrote…
speculation: You might not need the goroutine here if you
close(allowRequest)first? I'm not 100% sure, though.
Correct :)
pkg/sql/stats/create_stats_job_test.go line 451 at r8 (raw file):
Previously, michae2 (Michael Erickson) wrote…
Should we also check that this is nil?
Done.
pkg/sql/stats/create_stats_job_test.go line 489 at r8 (raw file):
Previously, michae2 (Michael Erickson) wrote…
Same question here
Done.
rytaft
left a comment
There was a problem hiding this comment.
I just looked at the last revision, and that , thanks!
Reviewed 3 of 3 files at r9, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @DrewKimball, @dt, @faizaanmadhani, and @michae2)
Previously, michae2 (Michael Erickson) wrote…
@rytaft do you have any objection to multiple automatic partial stats collections at once? It seems like a good way to start trying out multiple concurrent stats jobs?
I remember we discussed this before in:
- #118983
- some slack conversations I can't seem to find now
Yea I think this should be fine now.
pkg/jobs/utils.go line 113 at r9 (raw file):
s.WriteByte(')') typeStrs = s.String() }
Can you pull this code to get typeStrs into a helper function? Looks identical to what's in the function above.
aafd4e5 to
1c025e8
Compare
Uzair5162
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @DrewKimball, @dt, @faizaanmadhani, @michae2, and @rytaft)
pkg/jobs/utils.go line 113 at r9 (raw file):
Previously, rytaft (Rebecca Taft) wrote…
Can you pull this code to get typeStrs into a helper function? Looks identical to what's in the function above.
Done.
rytaft
left a comment
There was a problem hiding this comment.
Reviewed 1 of 1 files at r11, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @DrewKimball, @dt, @faizaanmadhani, and @michae2)
1c025e8 to
b575ecf
Compare
We previously triggered full stat refreshes when approximately 20% of a table's rows were changed. This commit introduces additional partial statistic 'at extremes' refreshes that occur automatically in new `TypeAutoCreatePartialStats` jobs when roughly 5% of rows change. Various new cluster and table settings to configure the frequency of auto partial stats collections have been introduced, which mimic existing settings for automatic full stats collection (see release note for details). Table settings take precedence over their corresponding cluster settings if set. Partial stats will never be collected automatically in cases where automatic full stats wouldn't happen, such as when auto full stat collection settings are disabled or tables on which auto stats collections aren't allowed. Automatic partial stat refreshes follow a probability-based model similar to automatic full stats to approximate the desired refresh frequency. We previously decided to trigger a full stat refresh with a probability proportional to the number of rows affected by mutations on a table. This has been extended to make a second decision on whether to trigger a partial stat refresh with a higher likelihood whenever we decide not to trigger a full refresh. Unlike auto full stats, multiple `TypeAutoCreatePartialStats` jobs can run at once in a cluster, as long as they are on different tables. We don't start these jobs if any manual stat collection job (full or partial), any auto full stat collection job, or an auto partial stat collection job on the same table is running. We also don't stop any new stats jobs from running if there is an auto partial stats job running in the cluster (except for new auto partial stats jobs on the same table as one that is running). Epic: CRDB-19449 Release note (sql change): Partial statistics can now be automatically collected at the extremes of indexes when a certain fraction and minimum number of rows are stale (5% and 100 by default). These can be configured with new table and cluster settings, and the feature is disabled by default. The new cluster and table settings are: - `sql.stats.automatic_partial_collection .enabled`/`sql_stats_automatic_partial_collection_enabled` - Defaults to `false` - `sql.stats.automatic_partial_collection .min_stale_rows`/`sql_stats_automatic_partial_collection_min_stale_rows` - Defaults to `100` - `sql.stats.automatic_partial_collection .fraction_stale_rows `/`sql_stats_automatic_partial_collection_fraction_stale_rows` - Defaults to `0.05` Co-authored-by: Faizaan Madhani <fzmadhani@gmail.com>
b575ecf to
6ac3f39
Compare
mgartner
left a comment
There was a problem hiding this comment.
I'm a bit concerned about the extra pressure this will place on the jobs system because this will fairly significantly increase the number of jobs. If we see problems when enabling this we might have to prioritize addressing #108435.
Reviewed 12 of 31 files at r3, 1 of 5 files at r5, 3 of 6 files at r6, 2 of 3 files at r8, 2 of 3 files at r9, 1 of 1 files at r11, 5 of 5 files at r12, 1 of 1 files at r13, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (and 2 stale) (waiting on @DrewKimball, @dt, @faizaanmadhani, and @michae2)
pkg/sql/logictest/testdata/logic_test/distsql_automatic_partial_stats line 73 at r13 (raw file):
query TTIIIT colnames SELECT DISTINCT ON (statistics_name, column_names) statistics_name, column_names, row_count, distinct_count, null_count, partial_predicate FROM [SHOW STATISTICS FOR TABLE data] ORDER BY statistics_name, column_names, created DESC
This test and the one below might erroneously pass because the queries complete before the partial stats could be collected. I don't see an easy way of making the tests more robust though... I suppose if there is a bug here the test will flake periodically in nightly tests and we'll find it.
|
Thanks for the reviews! bors r+ |
In cockroachdb#93067 we added support for automatic collection of partial stats. This commit enables automatic collection of partial stats by default. Release note (sql change): This commit turns on cluster setting `sql.stats.automatic_partial_collection.enabled` by default, which enables automatic collection of partial table stats. Partial table stats (i.e. those created with `CREATE STATISTICS ... USING EXTREMES`) scan the lower and upper ends of indexes to collect statistics outside the range covered by the previous full statistics collection.
133899: sql/stats: silence noisy autostats log message r=rytaft,dt a=michae2 We currently only allow a single automatic stats collection at a time. If a stats collection is running for a long time, any queued automatic stats collections will log the following message once a minute: ``` automatically executing "CREATE STATISTICS __auto__ FROM [... " ``` This is expected, but we don't need to fill the logs with this message. Let's only log it if verbose logging is enabled. Epic: None Release note: None 133988: sql/stats: enable automatic collection of partial table stats r=rytaft a=michae2 In #93067 we added support for automatic collection of partial stats. This commit enables automatic collection of partial stats by default. Epic: None Release note (sql change): This commit turns on cluster setting `sql.stats.automatic_partial_collection.enabled` by default, which enables automatic collection of partial table stats. Partial table stats (i.e. those created with `CREATE STATISTICS ... USING EXTREMES`) scan the lower and upper ends of indexes to collect statistics outside the range covered by the previous full statistics collection. Co-authored-by: Michael Erickson <michae2@cockroachlabs.com>
We previously triggered full stat refreshes when approximately 20% of a table's rows were changed. This commit introduces additional partial statistic 'at extremes' refreshes that occur automatically in new
TypeAutoCreatePartialStatsjobs when roughly 5% of rows change.Various new cluster and table settings to configure the frequency of auto partial stats collections have been introduced, which mimic existing settings for automatic full stats collection (see release note for details).
Table settings take precedence over their corresponding cluster settings if set. Partial stats will never be collected automatically in cases where automatic full stats wouldn't happen, such as when auto full stat collection settings are disabled or tables on which auto stats collections aren't allowed.
Automatic partial stat refreshes follow a probability-based model similar to automatic full stats to approximate the desired refresh frequency. We previously decided to trigger a full stat refresh with a probability proportional to the number of rows affected by mutations on a table. This has been extended to make a second decision on whether to trigger a partial stat refresh with a higher likelihood whenever we decide not to trigger a full refresh.
Unlike auto full stats, multiple
TypeAutoCreatePartialStatsjobs can run at once in a cluster, as long as they are on different tables. We don't start these jobs if any manual stat collection job (full or partial), any auto full stat collection job, or an auto partial stat collection job on the same table is running. We also don't stop any new stats jobs from running if there is an auto partial stats job running in the cluster (except for new auto partial stats jobs on the same table as one that is running).Epic: CRDB-19449
Release note (sql change): Partial statistics can now be automatically collected at the extremes of indexes when a certain fraction and minimum number of rows are stale (5% and 100 by default). These can be configured with new table and cluster settings, and the feature is disabled by default. The new cluster and table settings are:
sql.stats.automatic_partial_collection.enabled/sql_stats_automatic_partial_collection_enabledfalsesql.stats.automatic_partial_collection.min_stale_rows/sql_stats_automatic_partial_collection_min_stale_rows100sql.stats.automatic_partial_collection.fraction_stale_rows/sql_stats_automatic_partial_collection_fraction_stale_rows0.05