Skip to content

Commit

Permalink
stats: table setting to turn auto stats collection on/off
Browse files Browse the repository at this point in the history
Fixes #40989

Previously, there was no way to enable or disable automatic statistics
collection at the table level. It could only be turned on or off via the
`sql.stats.automatic_collection.enabled` cluster setting.

This was inadequate because statistics collection can be expensive for
large tables, and it would be desirable to defer collection until after
data is finished loading, or in off hours. Also, small tables which are
frequently updated may trigger statistics collection leading to
unnecessary overhead and/or unpredictable query plan changes.

To address this, this patch adds support for setting of the following
cluster settings at the table level:
```
sql_stats_automatic_collection_enabled
sql_stats_automatic_collection_min_stale_rows
sql_stats_automatic_collection_fraction_stale_rows
```
which correspond to the similarly-named cluster settings:
```
sql.stats.automatic_collection.enabled
sql.stats.automatic_collection.min_stale_rows
sql.stats.automatic_collection.fraction_stale_rows
```
for example:
```
ALTER TABLE t1 SET (sql_stats_automatic_collection_enabled = true);
ALTER TABLE t1
      SET (sql_stats_automatic_collection_fraction_stale_rows = 0.1,
           sql_stats_automatic_collection_min_stale_rows = 2000);
```
The table-level setting takes precedence over the cluster setting.

Release justification: Low risk fix for missing fine-grained control
over automatic statistics collection.

Release note (sql change): Automatic statistics collection can now be
enabled or disabled for individual tables, taking precedence over the
cluster setting, for example:
```
ALTER TABLE t1 SET (sql_stats_automatic_collection_enabled = true);
ALTER TABLE t1 SET (sql_stats_automatic_collection_enabled = false);
ALTER TABLE t1 RESET (sql_stats_automatic_collection_enabled);
```
RESET removes the setting value entirely, in which case the
similarly-name cluster setting,
`sql.stats.automatic_collection.enabled`, is in effect for the table.

Cluster settings `sql.stats.automatic_collection.fraction_stale_rows`
and `sql.stats.automatic_collection.min_stale_rows` now also have table
setting counterparts:
```
sql_stats_automatic_collection_fraction_stale_rows
sql_stats_automatic_collection_min_stale_rows
```
The table settings may be set at table creation time, or later via ALTER
TABLE ... SET, independent of whether auto stats is enabled:
```
ALTER TABLE t1
      SET (sql_stats_automatic_collection_fraction_stale_rows = 0.1,
           sql_stats_automatic_collection_min_stale_rows = 2000);
CREATE TABLE t1 (a INT, b INT)
       WITH (sql_stats_automatic_collection_enabled = true,
             sql_stats_automatic_collection_min_stale_rows = 1000000,
             sql_stats_automatic_collection_fraction_stale_rows= 0.05
             );
```
The current table settings (storage parameters) are shown in the `WITH`
clause output of `SHOW CREATE TABLE`.
Note, any row mutations which have occurred a minute or two before
disabling auto stats collection via `ALTER TABLE ... SET` may trigger
stats collection, though DML submitted after the setting change will
not.
  • Loading branch information
Mark Sirek committed Apr 26, 2022
1 parent a3c0709 commit 7da9360
Show file tree
Hide file tree
Showing 19 changed files with 1,103 additions and 82 deletions.
31 changes: 31 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -716,6 +716,7 @@ func (n *alterTableNode) startExec(params runParams) error {
}

case *tree.AlterTableSetStorageParams:
oldTableHasAutoStatsSettings := n.tableDesc.GetAutoStatsSettings() != nil
var ttlBefore *catpb.RowLevelTTL
if ttl := n.tableDesc.GetRowLevelTTL(); ttl != nil {
ttlBefore = protoutil.Clone(ttl).(*catpb.RowLevelTTL)
Expand All @@ -741,7 +742,15 @@ func (n *alterTableNode) startExec(params runParams) error {
return err
}

newTableHasAutoStatsSettings := n.tableDesc.GetAutoStatsSettings() != nil
if err := checkDisallowedAutoStatsSettingChange(
params, oldTableHasAutoStatsSettings, newTableHasAutoStatsSettings,
); err != nil {
return err
}

case *tree.AlterTableResetStorageParams:
oldTableHasAutoStatsSettings := n.tableDesc.GetAutoStatsSettings() != nil
var ttlBefore *catpb.RowLevelTTL
if ttl := n.tableDesc.GetRowLevelTTL(); ttl != nil {
ttlBefore = protoutil.Clone(ttl).(*catpb.RowLevelTTL)
Expand All @@ -766,6 +775,13 @@ func (n *alterTableNode) startExec(params runParams) error {
return err
}

newTableHasAutoStatsSettings := n.tableDesc.GetAutoStatsSettings() != nil
if err := checkDisallowedAutoStatsSettingChange(
params, oldTableHasAutoStatsSettings, newTableHasAutoStatsSettings,
); err != nil {
return err
}

case *tree.AlterTableRenameColumn:
descChanged, err := params.p.renameColumn(params.ctx, n.tableDesc, t.Column, t.NewName)
if err != nil {
Expand Down Expand Up @@ -1941,6 +1957,21 @@ func handleTTLStorageParamChange(
return nil
}

func checkDisallowedAutoStatsSettingChange(
params runParams, oldTableHasAutoStatsSettings, newTableHasAutoStatsSettings bool,
) error {
if !oldTableHasAutoStatsSettings && !newTableHasAutoStatsSettings {
// Do not have to do anything here.
return nil
}

if err := checkAutoStatsTableSettingsEnabledForCluster(params.ctx, params.p.ExecCfg().Settings); err != nil {
return err
}

return nil
}

// tryRemoveFKBackReferences determines whether the provided unique constraint
// is used on the referencing side of a FK constraint. If so, it tries to remove
// the references or find an alternate unique constraint that will suffice.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/catpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_proto_library(
go_library(
name = "catpb",
srcs = [
"catalog.go",
"constraint.go",
"default_privilege.go",
"doc.go",
Expand Down
92 changes: 92 additions & 0 deletions pkg/sql/catalog/catpb/catalog.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package catpb

// AutoStatsCollectionStatus represents whether the auto stats collections
// enabled table setting is enabled, disabled, or not set.
type AutoStatsCollectionStatus int

// The values for AutoStatsCollectionStatus.
const (
AutoStatsCollectionNotSet AutoStatsCollectionStatus = iota
AutoStatsCollectionEnabled
AutoStatsCollectionDisabled
)

const (
// AutoStatsEnabledSettingName is the name of the automatic stats collection
// enabled cluster setting.
AutoStatsEnabledSettingName = "sql.stats.automatic_collection.enabled"

// AutoStatsEnabledTableSettingName is the name of the automatic stats
// collection enabled table setting.
AutoStatsEnabledTableSettingName = "sql_stats_automatic_collection_enabled"

// AutoStatsMinStaleSettingName is the name of the automatic stats collection
// min stale rows cluster setting.
AutoStatsMinStaleSettingName = "sql.stats.automatic_collection.min_stale_rows"

// AutoStatsMinStaleTableSettingName is the name of the automatic stats collection
// min stale rows table setting.
AutoStatsMinStaleTableSettingName = "sql_stats_automatic_collection_min_stale_rows"

// AutoStatsFractionStaleSettingName is the name of the automatic stats
// collection fraction stale rows cluster setting.
AutoStatsFractionStaleSettingName = "sql.stats.automatic_collection.fraction_stale_rows"

// AutoStatsFractionStaleTableSettingName is the name of the automatic stats
// collection fraction stale rows table setting.
AutoStatsFractionStaleTableSettingName = "sql_stats_automatic_collection_fraction_stale_rows"
)

// AutoStatsCollectionEnabled indicates if automatic statistics collection is
// explicitly enabled or disabled.
func (as *AutoStatsSettings) AutoStatsCollectionEnabled() AutoStatsCollectionStatus {
if as.Enabled == nil {
return AutoStatsCollectionNotSet
}
if *as.Enabled {
return AutoStatsCollectionEnabled
}
return AutoStatsCollectionDisabled
}

// AutoStatsMinStaleRows indicates the setting of
// sql_stats_automatic_collection_min_stale_rows in AutoStatsSettings. If ok is
// true, then the minStaleRows value is valid, otherwise this has not been set.
func (as *AutoStatsSettings) AutoStatsMinStaleRows() (minStaleRows int64, ok bool) {
if as.MinStaleRows == nil {
return 0, false
}
return *as.MinStaleRows, true
}

// AutoStatsFractionStaleRows indicates the setting of
// sql_stats_automatic_collection_fraction_stale_rows in AutoStatsSettings. If
// ok is true, then the fractionStaleRows value is valid, otherwise this has not
// been set.
func (as *AutoStatsSettings) AutoStatsFractionStaleRows() (fractionStaleRows float64, ok bool) {
if as.FractionStaleRows == nil {
return 0, false
}
return *as.FractionStaleRows, true
}

// NoAutoStatsSettingsOverrides is true if no auto stats related table
// settings are present in these AutoStatsSettings.
func (as *AutoStatsSettings) NoAutoStatsSettingsOverrides() bool {
if as.Enabled != nil ||
as.MinStaleRows != nil ||
as.FractionStaleRows != nil {
return false
}
return true
}
6 changes: 3 additions & 3 deletions pkg/sql/catalog/catpb/catalog.proto
Original file line number Diff line number Diff line change
Expand Up @@ -225,10 +225,10 @@ message RowLevelTTL {
// only list values which have been set. Protobuf type double is float64 in Go.
message AutoStatsSettings {
option (gogoproto.equal) = true;
// sql.stats.automatic_collection.enabled
// Enabled is table setting sql_stats_automatic_collection_enabled.
optional bool enabled = 1;
// sql.stats.automatic_collection.min_stale_rows
// MinStaleRows is table setting sql_stats_automatic_collection_min_stale_rows.
optional int64 min_stale_rows = 2;
// sql.stats.automatic_collection.fraction_stale_rows
// FractionStaleRows is table setting sql_stats_automatic_collection_fraction_stale_rows.
optional double fraction_stale_rows = 3;
}
19 changes: 19 additions & 0 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -684,6 +684,25 @@ type TableDescriptor interface {
GetExcludeDataFromBackup() bool
// GetStorageParams returns a list of storage parameters for the table.
GetStorageParams(spaceBetweenEqual bool) []string
// NoAutoStatsSettingsOverrides is true if no auto stats related settings are
// set at the table level for the given table.
NoAutoStatsSettingsOverrides() bool
// AutoStatsCollectionEnabled indicates if automatic statistics collection is
// explicitly enabled or disabled for this table.
AutoStatsCollectionEnabled() catpb.AutoStatsCollectionStatus
// AutoStatsMinStaleRows indicates the setting of
// sql_stats_automatic_collection_min_stale_rows for this table.
// If ok is true, then the minStaleRows value is valid, otherwise this has not
// been set at the table level.
AutoStatsMinStaleRows() (minStaleRows int64, ok bool)
// AutoStatsFractionStaleRows indicates the setting of
// sql_stats_automatic_collection_fraction_stale_rows for this table. If ok is
// true, then the fractionStaleRows value is valid, otherwise this has not
// been set at the table level.
AutoStatsFractionStaleRows() (fractionStaleRows float64, ok bool)
// GetAutoStatsSettings returns the table settings related to automatic
// statistics collection. May return nil if none are set.
GetAutoStatsSettings() *catpb.AutoStatsSettings
}

// TypeDescriptor will eventually be called typedesc.Descriptor.
Expand Down
54 changes: 54 additions & 0 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -2584,6 +2584,23 @@ func (desc *wrapper) GetStorageParams(spaceBetweenEqual bool) []string {
if exclude := desc.GetExcludeDataFromBackup(); exclude {
appendStorageParam(`exclude_data_from_backup`, `true`)
}
if settings := desc.AutoStatsSettings; settings != nil {
if settings.Enabled != nil {
value := *settings.Enabled
appendStorageParam(catpb.AutoStatsEnabledTableSettingName,
fmt.Sprintf("%v", value))
}
if settings.MinStaleRows != nil {
value := *settings.MinStaleRows
appendStorageParam(catpb.AutoStatsMinStaleTableSettingName,
fmt.Sprintf("%d", value))
}
if settings.FractionStaleRows != nil {
value := *settings.FractionStaleRows
appendStorageParam(catpb.AutoStatsFractionStaleTableSettingName,
fmt.Sprintf("%g", value))
}
}
return storageParams
}

Expand All @@ -2602,6 +2619,43 @@ func (desc *wrapper) GetMultiRegionEnumDependencyIfExists() bool {
return false
}

// NoAutoStatsSettingsOverrides implements the TableDescriptor interface.
func (desc *wrapper) NoAutoStatsSettingsOverrides() bool {
if desc.AutoStatsSettings == nil {
return true
}
return desc.AutoStatsSettings.NoAutoStatsSettingsOverrides()
}

// AutoStatsCollectionEnabled implements the TableDescriptor interface.
func (desc *wrapper) AutoStatsCollectionEnabled() catpb.AutoStatsCollectionStatus {
if desc.AutoStatsSettings == nil {
return catpb.AutoStatsCollectionNotSet
}
return desc.AutoStatsSettings.AutoStatsCollectionEnabled()
}

// AutoStatsMinStaleRows implements the TableDescriptor interface.
func (desc *wrapper) AutoStatsMinStaleRows() (minStaleRows int64, ok bool) {
if desc.AutoStatsSettings == nil {
return 0, false
}
return desc.AutoStatsSettings.AutoStatsMinStaleRows()
}

// AutoStatsFractionStaleRows implements the TableDescriptor interface.
func (desc *wrapper) AutoStatsFractionStaleRows() (fractionStaleRows float64, ok bool) {
if desc.AutoStatsSettings == nil {
return 0, false
}
return desc.AutoStatsSettings.AutoStatsFractionStaleRows()
}

// GetAutoStatsSettings implements the TableDescriptor interface.
func (desc *wrapper) GetAutoStatsSettings() *catpb.AutoStatsSettings {
return desc.AutoStatsSettings
}

// SetTableLocalityRegionalByTable sets the descriptor's locality config to
// regional at the table level in the supplied region. An empty region name
// (or its alias PrimaryRegionNotSpecifiedName) denotes that the table is homed in
Expand Down
53 changes: 53 additions & 0 deletions pkg/sql/catalog/tabledesc/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege"
Expand Down Expand Up @@ -521,6 +522,8 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) {
}
}

desc.validateAutoStatsSettings(vea)

if desc.IsSequence() {
return
}
Expand Down Expand Up @@ -1519,3 +1522,53 @@ func (desc *wrapper) validatePartitioning() error {
)
})
}

// validateAutoStatsSettings validates that any new settings in
// catpb.AutoStatsSettings hold a valid value.
func (desc *wrapper) validateAutoStatsSettings(vea catalog.ValidationErrorAccumulator) {
if desc.AutoStatsSettings == nil {
return
}
desc.validateAutoStatsEnabled(vea, desc.AutoStatsSettings.Enabled)
desc.validateMinStaleRows(vea, desc.AutoStatsSettings.MinStaleRows)
desc.validateFractionStaleRows(vea, desc.AutoStatsSettings.FractionStaleRows)
}

func (desc *wrapper) verifyProperTableForStatsSetting(
vea catalog.ValidationErrorAccumulator, settingName string,
) {
if desc.IsVirtualTable() {
vea.Report(errors.Newf("Setting %s may not be set on virtual table", settingName))
}
if !desc.IsTable() {
vea.Report(errors.Newf("Setting %s may not be set on a view or sequence", settingName))
}
}

func (desc *wrapper) validateAutoStatsEnabled(vea catalog.ValidationErrorAccumulator, value *bool) {
if value != nil {
desc.verifyProperTableForStatsSetting(vea, catpb.AutoStatsEnabledTableSettingName)
}
}

func (desc *wrapper) validateMinStaleRows(vea catalog.ValidationErrorAccumulator, value *int64) {
if value != nil {
settingName := catpb.AutoStatsMinStaleTableSettingName
desc.verifyProperTableForStatsSetting(vea, settingName)
if err := settings.NonNegativeInt(*value); err != nil {
vea.Report(errors.Wrapf(err, "invalid integer value for %s", settingName))
}
}
}

func (desc *wrapper) validateFractionStaleRows(
vea catalog.ValidationErrorAccumulator, value *float64,
) {
if value != nil {
settingName := catpb.AutoStatsFractionStaleTableSettingName
desc.verifyProperTableForStatsSetting(vea, settingName)
if err := settings.NonNegativeFloat(*value); err != nil {
vea.Report(errors.Wrapf(err, "invalid float value for %s", settingName))
}
}
}
Loading

0 comments on commit 7da9360

Please sign in to comment.