diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index fe073146b8f5..1521578d559a 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -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) @@ -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) @@ -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 { @@ -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. diff --git a/pkg/sql/catalog/catpb/BUILD.bazel b/pkg/sql/catalog/catpb/BUILD.bazel index 295baeeef441..75e84729611d 100644 --- a/pkg/sql/catalog/catpb/BUILD.bazel +++ b/pkg/sql/catalog/catpb/BUILD.bazel @@ -29,6 +29,7 @@ go_proto_library( go_library( name = "catpb", srcs = [ + "catalog.go", "constraint.go", "default_privilege.go", "doc.go", diff --git a/pkg/sql/catalog/catpb/catalog.go b/pkg/sql/catalog/catpb/catalog.go new file mode 100644 index 000000000000..3afdaaee4e03 --- /dev/null +++ b/pkg/sql/catalog/catpb/catalog.go @@ -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 +} diff --git a/pkg/sql/catalog/catpb/catalog.proto b/pkg/sql/catalog/catpb/catalog.proto index ebce7abc9eb1..bb5d9afaf522 100644 --- a/pkg/sql/catalog/catpb/catalog.proto +++ b/pkg/sql/catalog/catpb/catalog.proto @@ -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; } diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index c11099f853ea..d0d7cca93594 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -678,6 +678,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. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 7d0bd5389dd1..c7a71bb38a7e 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -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 } @@ -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 diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index f501ab12cdb1..801a847ed4ee 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -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" @@ -521,6 +522,8 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) { } } + desc.validateAutoStatsSettings(vea) + if desc.IsSequence() { return } @@ -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)) + } + } +} diff --git a/pkg/sql/catalog/tabledesc/validate_test.go b/pkg/sql/catalog/tabledesc/validate_test.go index ce6f98f04ac1..d34d7e121536 100644 --- a/pkg/sql/catalog/tabledesc/validate_test.go +++ b/pkg/sql/catalog/tabledesc/validate_test.go @@ -131,10 +131,8 @@ var validationMap = []struct { "ExcludeDataFromBackup": {status: thisFieldReferencesNoObjects}, "NextConstraintID": {status: iSolemnlySwearThisFieldIsValidated}, "DeclarativeSchemaChangerState": {status: iSolemnlySwearThisFieldIsValidated}, - "AutoStatsSettings": { - status: todoIAmKnowinglyAddingTechDebt, - reason: "initial import: TODO(msirek): add validation"}, - "ForecastStats": {status: thisFieldReferencesNoObjects}, + "AutoStatsSettings": {status: iSolemnlySwearThisFieldIsValidated}, + "ForecastStats": {status: thisFieldReferencesNoObjects}, }, }, { @@ -294,13 +292,11 @@ var validationMap = []struct { }, }, { - // TODO(msirek): These fields can't be set until #78110 merges. That PR - // will add validation. obj: catpb.AutoStatsSettings{}, fieldMap: map[string]validationStatusInfo{ - "Enabled": {status: thisFieldReferencesNoObjects}, - "MinStaleRows": {status: thisFieldReferencesNoObjects}, - "FractionStaleRows": {status: thisFieldReferencesNoObjects}, + "Enabled": {status: iSolemnlySwearThisFieldIsValidated}, + "MinStaleRows": {status: iSolemnlySwearThisFieldIsValidated}, + "FractionStaleRows": {status: iSolemnlySwearThisFieldIsValidated}, }, }, } @@ -346,6 +342,9 @@ func TestValidateTableDesc(t *testing.T) { computedExpr := "1 + 1" generatedAsIdentitySequenceOptionExpr := " START 2 INCREMENT 3 CACHE 10" + boolTrue := true + negativeOne := int64(-1) + negativeOneFloat := float64(-1) testData := []struct { err string @@ -1883,6 +1882,95 @@ func TestValidateTableDesc(t *testing.T) { }, }, }, + {`Setting sql_stats_automatic_collection_enabled may not be set on virtual table`, + descpb.TableDescriptor{ + ID: catconstants.MinVirtualID, + ParentID: 1, + Name: "foo", + FormatVersion: descpb.InterleavedFormatVersion, + Columns: []descpb.ColumnDescriptor{ + {ID: 1, Name: "bar"}, + }, + NextColumnID: 2, + AutoStatsSettings: &catpb.AutoStatsSettings{Enabled: &boolTrue}, + }}, + {`Setting sql_stats_automatic_collection_enabled may not be set on a view or sequence`, + descpb.TableDescriptor{ + Name: "bar", + ID: 52, + ParentID: 1, + FormatVersion: descpb.InterleavedFormatVersion, + UnexposedParentSchemaID: keys.PublicSchemaID, + ViewQuery: "SELECT * FROM foo", + DependsOn: []descpb.ID{51}, + NextColumnID: 2, + Columns: []descpb.ColumnDescriptor{ + {Name: "a", ID: 1, Type: types.Int}, + }, + Privileges: catpb.NewBasePrivilegeDescriptor(security.AdminRoleName()), + AutoStatsSettings: &catpb.AutoStatsSettings{Enabled: &boolTrue}, + }}, + {`Setting sql_stats_automatic_collection_enabled may not be set on a view or sequence`, + descpb.TableDescriptor{ + ID: 51, + ParentID: 1, + Name: "foo", + FormatVersion: descpb.InterleavedFormatVersion, + Columns: []descpb.ColumnDescriptor{ + {ID: 1, Name: "a", Type: types.Int}, + }, + SequenceOpts: &descpb.TableDescriptor_SequenceOpts{ + Increment: 1, + }, + PrimaryIndex: descpb.IndexDescriptor{ + ID: 1, + Name: "primary", + Unique: true, + KeyColumnIDs: []descpb.ColumnID{1}, + KeyColumnNames: []string{"a"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + Version: descpb.PrimaryIndexWithStoredColumnsVersion, + EncodingType: descpb.PrimaryIndexEncoding, + ConstraintID: 1, + }, + Families: []descpb.ColumnFamilyDescriptor{ + {ID: 0, Name: "primary", + ColumnIDs: []descpb.ColumnID{1}, + ColumnNames: []string{"a"}, + }, + }, + NextColumnID: 2, + NextFamilyID: 1, + NextIndexID: 5, + NextConstraintID: 2, + Privileges: catpb.NewBasePrivilegeDescriptor(security.AdminRoleName()), + AutoStatsSettings: &catpb.AutoStatsSettings{Enabled: &boolTrue}, + }, + }, + {`invalid integer value for sql_stats_automatic_collection_min_stale_rows: cannot be set to a negative value: -1`, + descpb.TableDescriptor{ + ID: 2, + ParentID: 1, + Name: "foo", + FormatVersion: descpb.InterleavedFormatVersion, + Columns: []descpb.ColumnDescriptor{ + {ID: 1, Name: "bar"}, + }, + NextColumnID: 2, + AutoStatsSettings: &catpb.AutoStatsSettings{MinStaleRows: &negativeOne}, + }}, + {`invalid float value for sql_stats_automatic_collection_fraction_stale_rows: cannot set to a negative value: -1.000000`, + descpb.TableDescriptor{ + ID: 2, + ParentID: 1, + Name: "foo", + FormatVersion: descpb.InterleavedFormatVersion, + Columns: []descpb.ColumnDescriptor{ + {ID: 1, Name: "bar"}, + }, + NextColumnID: 2, + AutoStatsSettings: &catpb.AutoStatsSettings{FractionStaleRows: &negativeOneFloat}, + }}, } for i, d := range testData { t.Run(d.err, func(t *testing.T) { @@ -2277,6 +2365,7 @@ func TestValidateCrossTableReferences(t *testing.T) { }, // Views. { // 10 + err: ``, desc: descpb.TableDescriptor{ Name: "foo", ID: 51, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index b8a1f7a12ca1..1dcbca846343 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1444,6 +1444,12 @@ func NewTableDesc( primaryIndexColumnSet[string(regionalByRowCol)] = struct{}{} } + if autoStatsSettings := desc.GetAutoStatsSettings(); autoStatsSettings != nil { + if err := checkAutoStatsTableSettingsEnabledForCluster(ctx, st); err != nil { + return nil, err + } + } + // Create the TTL column if one does not already exist. if ttl := desc.GetRowLevelTTL(); ttl != nil { if err := checkTTLEnabledForCluster(ctx, st); err != nil { @@ -2402,6 +2408,16 @@ func checkTTLEnabledForCluster(ctx context.Context, st *cluster.Settings) error return nil } +func checkAutoStatsTableSettingsEnabledForCluster(ctx context.Context, st *cluster.Settings) error { + if !st.Version.IsActive(ctx, clusterversion.AutoStatsTableSettings) { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "auto stats table settings are only available once the cluster is fully upgraded", + ) + } + return nil +} + // CreateRowLevelTTLScheduledJob creates a new row-level TTL schedule. func CreateRowLevelTTLScheduledJob( ctx context.Context, diff --git a/pkg/sql/create_test.go b/pkg/sql/create_test.go index 16875ecfb196..531aeeda7e00 100644 --- a/pkg/sql/create_test.go +++ b/pkg/sql/create_test.go @@ -19,8 +19,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" @@ -411,3 +413,46 @@ func TestSetUserPasswordInsecure(t *testing.T) { }) } } + +func TestAutoStatsTableSettingsDisallowedOnOldCluster(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + serverArgs := base.TestServerArgs{ + Insecure: true, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey(clusterversion.ClusterLocksVirtualTable), + }, + }, + } + + var ( + ctx = context.Background() + s, conn, _ = serverutils.StartServer(t, serverArgs) + sqlDB = sqlutils.MakeSQLRunner(conn) + ) + defer conn.Close() + defer s.Stopper().Stop(ctx) + + sqlDB.Exec(t, + `CREATE DATABASE t;`) + + sqlDB.ExpectErr(t, "pq: auto stats table settings are only available once the cluster is fully upgraded", "CREATE TABLE t1 (a int) WITH (sql_stats_automatic_collection_enabled = true)") + + sqlDB.Exec(t, + `CREATE TABLE t2 (a int)`) + + sqlDB.ExpectErr(t, "pq: auto stats table settings are only available once the cluster is fully upgraded", "ALTER TABLE t2 SET (sql_stats_automatic_collection_enabled = true)") + + // Run the migration. + sqlDB.Exec(t, "SET CLUSTER SETTING version = $1", clusterversion.ByKey(clusterversion.AutoStatsTableSettings).String()) + + sqlDB.Exec(t, + `CREATE TABLE t1 (a int) WITH (sql_stats_automatic_collection_enabled = true)`) + + sqlDB.Exec(t, + `ALTER TABLE t2 SET (sql_stats_automatic_collection_enabled = true)`) + +} diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index 1afeca796780..f4d922aeb442 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -213,6 +213,9 @@ func (dsp *DistSQLPlanner) createStatsPlan( var rowsExpected uint64 if len(tableStats) > 0 { overhead := stats.AutomaticStatisticsFractionStaleRows.Get(&dsp.st.SV) + if autoStatsFractionStaleRowsForTable, ok := desc.AutoStatsFractionStaleRows(); ok { + overhead = autoStatsFractionStaleRowsForTable + } // Convert to a signed integer first to make the linter happy. rowsExpected = uint64(int64( // The total expected number of rows is the same number that was measured diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index ab682db27ef5..30454f4b3c5e 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -2364,3 +2364,114 @@ COMMIT; statement ok ROLLBACK; + +subtest table_settings + +statement ok +CREATE TABLE t5 (a int) + +# Turn on automatic stats collection +statement ok +ALTER TABLE t5 SET (sql_stats_automatic_collection_enabled = true) + +# Verify automatic collection is enabled. +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't5' + AND tbl.drop_time IS NULL + AND crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' + ->> 'enabled' = 'true' +---- +{"enabled": true} + +# Strings in settings should be converted to the proper data type. +statement ok +ALTER TABLE t5 SET (sql_stats_automatic_collection_enabled = 'false') + +# Verify automatic collection is disabled. +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't5' + AND tbl.drop_time IS NULL + AND crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' + ->> 'enabled' = 'false' +---- +{"enabled": false} + +# SHOW CREATE TABLE displays the value properly. +query T +SELECT create_statement FROM [SHOW CREATE TABLE t5] +---- +CREATE TABLE public.t5 ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t5_pkey PRIMARY KEY (rowid ASC) +) WITH (sql_stats_automatic_collection_enabled = false) + +statement error pq: parameter "sql_stats_automatic_collection_enabled" requires a Boolean value +ALTER TABLE t5 SET (sql_stats_automatic_collection_enabled = 123) + +statement ok +ALTER TABLE t5 RESET (sql_stats_automatic_collection_enabled) + +# Verify the automatic collection setting is removed. +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->>'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't5' + AND tbl.drop_time IS NULL +---- +{} + +statement error pq: invalid float value for sql_stats_automatic_collection_fraction_stale_rows: could not parse "hello" as type float: strconv.ParseFloat: parsing "hello": invalid syntax +ALTER TABLE t5 SET (sql_stats_automatic_collection_fraction_stale_rows = 'hello') + +statement error pq: invalid integer value for sql_stats_automatic_collection_min_stale_rows: could not parse "world" as type int: strconv.ParseInt: parsing "world": invalid syntax +ALTER TABLE t5 SET (sql_stats_automatic_collection_min_stale_rows = 'world') + +# Verify strings can be converted to proper setting values. +statement ok +ALTER TABLE t5 SET (sql_stats_automatic_collection_fraction_stale_rows = '0.15', + sql_stats_automatic_collection_min_stale_rows = '1234') + +# Verify settings +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->>'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't5' + AND tbl.drop_time IS NULL +---- +{"fractionStaleRows": 0.15, "minStaleRows": "1234"} + +query T +SELECT create_statement FROM [SHOW CREATE TABLE t5] +---- +CREATE TABLE public.t5 ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t5_pkey PRIMARY KEY (rowid ASC) +) WITH (sql_stats_automatic_collection_min_stale_rows = 1234, sql_stats_automatic_collection_fraction_stale_rows = 0.15) diff --git a/pkg/sql/logictest/testdata/logic_test/create_table b/pkg/sql/logictest/testdata/logic_test/create_table index 039042219dd2..202b59c6ca16 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_table +++ b/pkg/sql/logictest/testdata/logic_test/create_table @@ -872,3 +872,73 @@ CREATE TABLE public.t_good_hash_indexes_2 ( crdb_internal_a_shard_5 INT8 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 5:::INT8)) VIRTUAL, CONSTRAINT t_good_hash_indexes_2_pkey PRIMARY KEY (a ASC) USING HASH WITH (bucket_count=5) ) + +subtest table_settings + +statement ok +CREATE TABLE t1 (a int) WITH (sql_stats_automatic_collection_enabled = true) + +# Verify automatic collection is enabled. +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->>'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't1' + AND tbl.drop_time IS NULL +---- +{"enabled": true} + +statement ok +DROP TABLE t1 + +statement ok +CREATE TABLE t1 (a int) WITH (sql_stats_automatic_collection_fraction_stale_rows = 0.5, + sql_stats_automatic_collection_min_stale_rows = 4000) + +# Verify settings +query T +SELECT + crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->>'autoStatsSettings' +FROM + crdb_internal.tables AS tbl + INNER JOIN system.descriptor AS d ON d.id = tbl.table_id +WHERE + tbl.name = 't1' + AND tbl.drop_time IS NULL +---- +{"fractionStaleRows": 0.5, "minStaleRows": "4000"} + +query T +SELECT create_statement FROM [SHOW CREATE TABLE t1] +---- +CREATE TABLE public.t1 ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t1_pkey PRIMARY KEY (rowid ASC) +) WITH (sql_stats_automatic_collection_min_stale_rows = 4000, sql_stats_automatic_collection_fraction_stale_rows = 0.5) + +statement ok +CREATE TABLE t11 (a int) WITH (sql_stats_automatic_collection_enabled = true, + sql_stats_automatic_collection_fraction_stale_rows = 1.797693134862315708145274237317043567981e+308, + sql_stats_automatic_collection_min_stale_rows = 9223372036854775807) + +# Using max values for auto stats table settings +query T +SELECT create_statement FROM [SHOW CREATE TABLE t11] +---- +CREATE TABLE public.t11 ( + a INT8 NULL, + rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), + CONSTRAINT t11_pkey PRIMARY KEY (rowid ASC) +) WITH (sql_stats_automatic_collection_enabled = true, sql_stats_automatic_collection_min_stale_rows = 9223372036854775807, sql_stats_automatic_collection_fraction_stale_rows = 1.7976931348623157e+308) + +statement error pq: invalid float value for sql_stats_automatic_collection_fraction_stale_rows: cannot set to a negative value: -1.000000 +CREATE TABLE t22 (a int) WITH (sql_stats_automatic_collection_fraction_stale_rows = -1.0) + +statement error pq: invalid integer value for sql_stats_automatic_collection_min_stale_rows: cannot be set to a negative value: -1 +CREATE TABLE t22 (a int) WITH (sql_stats_automatic_collection_min_stale_rows = -1) diff --git a/pkg/sql/paramparse/BUILD.bazel b/pkg/sql/paramparse/BUILD.bazel index 7faf6d4c7020..a55adb92f205 100644 --- a/pkg/sql/paramparse/BUILD.bazel +++ b/pkg/sql/paramparse/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/geo/geoindex", "//pkg/server/telemetry", + "//pkg/settings", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", diff --git a/pkg/sql/paramparse/paramobserver.go b/pkg/sql/paramparse/paramobserver.go index 584d33be5955..99d61e1708bd 100644 --- a/pkg/sql/paramparse/paramobserver.go +++ b/pkg/sql/paramparse/paramobserver.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -139,6 +140,34 @@ func boolFromDatum(evalCtx *tree.EvalContext, key string, datum tree.Datum) (boo return bool(*s), nil } +func intFromDatum(evalCtx *tree.EvalContext, key string, datum tree.Datum) (int64, error) { + intDatum := datum + if stringVal, err := DatumAsString(evalCtx, key, datum); err == nil { + if intDatum, err = tree.ParseDInt(stringVal); err != nil { + return 0, errors.Wrapf(err, "invalid integer value for %s", key) + } + } + s, err := DatumAsInt(evalCtx, key, intDatum) + if err != nil { + return 0, err + } + return s, nil +} + +func floatFromDatum(evalCtx *tree.EvalContext, key string, datum tree.Datum) (float64, error) { + floatDatum := datum + if stringVal, err := DatumAsString(evalCtx, key, datum); err == nil { + if floatDatum, err = tree.ParseDFloat(stringVal); err != nil { + return 0, errors.Wrapf(err, "invalid float value for %s", key) + } + } + s, err := DatumAsFloat(evalCtx, key, floatDatum) + if err != nil { + return 0, err + } + return s, nil +} + type tableParam struct { onSet func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, key string, datum tree.Datum) error onReset func(po *TableStorageParamObserver, evalCtx *tree.EvalContext, key string) error @@ -465,6 +494,18 @@ var tableParams = map[string]tableParam{ return nil }, }, + catpb.AutoStatsEnabledTableSettingName: { + onSet: autoStatsEnabledSettingFunc, + onReset: autoStatsTableSettingResetFunc, + }, + catpb.AutoStatsMinStaleTableSettingName: { + onSet: autoStatsMinStaleRowsSettingFunc(settings.NonNegativeInt), + onReset: autoStatsTableSettingResetFunc, + }, + catpb.AutoStatsFractionStaleTableSettingName: { + onSet: autoStatsFractionStaleRowsSettingFunc(settings.NonNegativeFloat), + onReset: autoStatsTableSettingResetFunc, + }, } func init() { @@ -508,6 +549,88 @@ func init() { } } +func autoStatsEnabledSettingFunc( + ctx context.Context, + po *TableStorageParamObserver, + semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, + key string, + datum tree.Datum, +) error { + boolVal, err := boolFromDatum(evalCtx, key, datum) + if err != nil { + return err + } + if po.tableDesc.AutoStatsSettings == nil { + po.tableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + } + po.tableDesc.AutoStatsSettings.Enabled = &boolVal + return nil +} + +func autoStatsMinStaleRowsSettingFunc( + validateFunc func(v int64) error, +) func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, key string, datum tree.Datum) error { + return func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, key string, datum tree.Datum) error { + intVal, err := intFromDatum(evalCtx, key, datum) + if err != nil { + return err + } + if po.tableDesc.AutoStatsSettings == nil { + po.tableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + } + if err = validateFunc(intVal); err != nil { + return errors.Wrapf(err, "invalid integer value for %s", key) + } + po.tableDesc.AutoStatsSettings.MinStaleRows = &intVal + return nil + } +} + +func autoStatsFractionStaleRowsSettingFunc( + validateFunc func(v float64) error, +) func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, key string, datum tree.Datum) error { + return func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, key string, datum tree.Datum) error { + floatVal, err := floatFromDatum(evalCtx, key, datum) + if err != nil { + return err + } + if po.tableDesc.AutoStatsSettings == nil { + po.tableDesc.AutoStatsSettings = &catpb.AutoStatsSettings{} + } + if err = validateFunc(floatVal); err != nil { + return errors.Wrapf(err, "invalid float value for %s", key) + } + po.tableDesc.AutoStatsSettings.FractionStaleRows = &floatVal + return nil + } +} + +func autoStatsTableSettingResetFunc( + po *TableStorageParamObserver, evalCtx *tree.EvalContext, key string, +) error { + if po.tableDesc.AutoStatsSettings == nil { + return nil + } + autoStatsSettings := po.tableDesc.AutoStatsSettings + switch key { + case catpb.AutoStatsEnabledTableSettingName: + autoStatsSettings.Enabled = nil + return nil + case catpb.AutoStatsMinStaleTableSettingName: + autoStatsSettings.MinStaleRows = nil + return nil + case catpb.AutoStatsFractionStaleTableSettingName: + autoStatsSettings.FractionStaleRows = nil + return nil + } + return errors.Newf("unable to reset table setting %s", key) +} + // onSet implements the StorageParamObserver interface. func (po *TableStorageParamObserver) onSet( ctx context.Context, diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index 0ab76f88400e..35bf813fcb63 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" @@ -37,7 +38,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" - "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -228,7 +228,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error { // Report tracked cluster settings via telemetry. // TODO(justin): implement a more general mechanism for tracking these. switch n.name { - case stats.AutoStatsClusterSettingName: + case catpb.AutoStatsEnabledSettingName: switch expectedEncodedValue { case "true": telemetry.Inc(sqltelemetry.TurnAutoStatsOnUseCounter) diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index 45d47a829f0b..1f830e0f426a 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -26,6 +26,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql/catalog", + "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/systemschema", @@ -85,9 +86,11 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/desctestutils", + "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", "//pkg/sql/execinfra", diff --git a/pkg/sql/stats/automatic_stats.go b/pkg/sql/stats/automatic_stats.go index 36c9093d2210..8f268c4bbb63 100644 --- a/pkg/sql/stats/automatic_stats.go +++ b/pkg/sql/stats/automatic_stats.go @@ -18,31 +18,31 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) -// AutoStatsClusterSettingName is the name of the automatic stats collection -// cluster setting. -const AutoStatsClusterSettingName = "sql.stats.automatic_collection.enabled" - // AutomaticStatisticsClusterMode controls the cluster setting for enabling // automatic table statistics collection. var AutomaticStatisticsClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, - AutoStatsClusterSettingName, + catpb.AutoStatsEnabledSettingName, "automatic statistics collection mode", true, ).WithPublic() @@ -81,7 +81,7 @@ var AutomaticStatisticsMaxIdleTime = settings.RegisterFloatSetting( var AutomaticStatisticsFractionStaleRows = func() *settings.FloatSetting { s := settings.RegisterFloatSetting( settings.TenantWritable, - "sql.stats.automatic_collection.fraction_stale_rows", + catpb.AutoStatsFractionStaleSettingName, "target fraction of stale rows per table that will trigger a statistics refresh", 0.2, settings.NonNegativeFloat, @@ -96,7 +96,7 @@ var AutomaticStatisticsFractionStaleRows = func() *settings.FloatSetting { var AutomaticStatisticsMinStaleRows = func() *settings.IntSetting { s := settings.RegisterIntSetting( settings.TenantWritable, - "sql.stats.automatic_collection.min_stale_rows", + catpb.AutoStatsMinStaleSettingName, "target minimum number of stale rows per table that will trigger a statistics refresh", 500, settings.NonNegativeInt, @@ -199,6 +199,10 @@ type Refresher struct { // metadata about SQL mutations to the background Refresher thread. mutations chan mutation + // settings is the buffered channel used to pass messages containing + // autostats setting override information to the background Refresher thread. + settings chan settingOverride + // asOfTime is a duration which is used to define the AS OF time for // runs of CREATE STATISTICS by the Refresher. asOfTime time.Duration @@ -212,6 +216,13 @@ type Refresher struct { // mutationCounts contains aggregated mutation counts for each table that // have yet to be processed by the refresher. mutationCounts map[descpb.ID]int64 + + // settingOverrides holds any autostats cluster setting overrides for each + // table. + settingOverrides map[descpb.ID]catpb.AutoStatsSettings + + // numTablesEnsured is an internal counter for testing ensureAllTables. + numTablesEnsured int } // mutation contains metadata about a SQL mutation and is the message passed to @@ -219,6 +230,17 @@ type Refresher struct { type mutation struct { tableID descpb.ID rowsAffected int + // removeSettingOverrides, when true, removes any pre-existing auto stats + // cluster setting overrides for the table with the above tableID. + // The default value of false is a no-Op. + removeSettingOverrides bool +} + +// settingOverride specifies the autostats setting override values to use in +// place of the cluster settings. +type settingOverride struct { + tableID descpb.ID + settings catpb.AutoStatsSettings } // MakeRefresher creates a new Refresher. @@ -232,16 +254,96 @@ func MakeRefresher( randSource := rand.NewSource(rand.Int63()) return &Refresher{ - AmbientContext: ambientCtx, - st: st, - ex: ex, - cache: cache, - randGen: makeAutoStatsRand(randSource), - mutations: make(chan mutation, refreshChanBufferLen), - asOfTime: asOfTime, - extraTime: time.Duration(rand.Int63n(int64(time.Hour))), - mutationCounts: make(map[descpb.ID]int64, 16), + AmbientContext: ambientCtx, + st: st, + ex: ex, + cache: cache, + randGen: makeAutoStatsRand(randSource), + mutations: make(chan mutation, refreshChanBufferLen), + settings: make(chan settingOverride, refreshChanBufferLen), + asOfTime: asOfTime, + extraTime: time.Duration(rand.Int63n(int64(time.Hour))), + mutationCounts: make(map[descpb.ID]int64, 16), + settingOverrides: make(map[descpb.ID]catpb.AutoStatsSettings), + } +} + +func (r *Refresher) getNumTablesEnsured() int { + return r.numTablesEnsured +} + +func (r *Refresher) autoStatsEnabled(desc catalog.TableDescriptor) bool { + if desc == nil { + // If the descriptor could not be accessed, defer to the cluster setting. + return AutomaticStatisticsClusterMode.Get(&r.st.SV) + } + enabledForTable := desc.AutoStatsCollectionEnabled() + // The table-level setting of sql_stats_automatic_collection_enabled takes + // precedence over the cluster setting. + if enabledForTable == catpb.AutoStatsCollectionNotSet { + return AutomaticStatisticsClusterMode.Get(&r.st.SV) + } + return enabledForTable == catpb.AutoStatsCollectionEnabled +} + +func (r *Refresher) autoStatsEnabledForTableID( + tableID descpb.ID, settingOverrides map[descpb.ID]catpb.AutoStatsSettings, +) bool { + var setting catpb.AutoStatsSettings + var ok bool + if settingOverrides == nil { + // If the setting overrides map doesn't exist, defer to the cluster setting. + return AutomaticStatisticsClusterMode.Get(&r.st.SV) + } + if setting, ok = settingOverrides[tableID]; !ok { + // If there are no setting overrides, defer to the cluster setting. + return AutomaticStatisticsClusterMode.Get(&r.st.SV) + } + autoStatsSettingValue := setting.AutoStatsCollectionEnabled() + if autoStatsSettingValue == catpb.AutoStatsCollectionNotSet { + return AutomaticStatisticsClusterMode.Get(&r.st.SV) + } + // The table-level setting of sql_stats_automatic_collection_enabled takes + // precedence over the cluster setting. + return autoStatsSettingValue == catpb.AutoStatsCollectionEnabled +} + +func (r *Refresher) autoStatsMinStaleRows(explicitSettings *catpb.AutoStatsSettings) int64 { + if explicitSettings == nil { + return AutomaticStatisticsMinStaleRows.Get(&r.st.SV) + } + if minStaleRows, ok := explicitSettings.AutoStatsMinStaleRows(); ok { + return minStaleRows + } + return AutomaticStatisticsMinStaleRows.Get(&r.st.SV) +} + +func (r *Refresher) autoStatsFractionStaleRows(explicitSettings *catpb.AutoStatsSettings) float64 { + if explicitSettings == nil { + return AutomaticStatisticsFractionStaleRows.Get(&r.st.SV) + } + if fractionStaleRows, ok := explicitSettings.AutoStatsFractionStaleRows(); ok { + return fractionStaleRows + } + return AutomaticStatisticsFractionStaleRows.Get(&r.st.SV) +} + +func (r *Refresher) getTableDescriptor( + ctx context.Context, tableID descpb.ID, +) (desc catalog.TableDescriptor) { + if err := r.cache.collectionFactory.Txn(ctx, r.cache.SQLExecutor, r.cache.ClientDB, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + flags := tree.ObjectLookupFlagsWithRequired() + if desc, err = descriptors.GetImmutableTableByID(ctx, txn, tableID, flags); err != nil { + err = errors.Wrapf(err, + "failed to get table descriptor for automatic stats on table id: %d", tableID) + } + return err + }); err != nil { + log.Errorf(ctx, "%v", err) } + return desc } // Start starts the stats refresher thread, which polls for messages about @@ -274,8 +376,27 @@ func (r *Refresher) Start( case <-timer.C: mutationCounts := r.mutationCounts + + var settingOverrides map[descpb.ID]catpb.AutoStatsSettings + // For each mutation count, look up auto stats setting overrides using + // the associated table ID. r.settingOverrides is never rebuilt. It is + // always added to or deleted from. We could just copy the entire hash + // map here, but maybe it is quicker and causes less memory pressure to + // just create a map with entries for the tables we're processing. + for tableID := range mutationCounts { + if settings, ok := r.settingOverrides[tableID]; ok { + if settingOverrides == nil { + settingOverrides = make(map[descpb.ID]catpb.AutoStatsSettings) + } + settingOverrides[tableID] = settings + } + } + if err := stopper.RunAsyncTask( ctx, "stats.Refresher: maybeRefreshStats", func(ctx context.Context) { + // Record the start time of processing this batch of tables. + start := timeutil.Now() + // Wait so that the latest changes will be reflected according to the // AS OF time. timerAsOf := time.NewTimer(r.asOfTime) @@ -288,13 +409,44 @@ func (r *Refresher) Start( } for tableID, rowsAffected := range mutationCounts { - // Check the cluster setting before each refresh in case it was - // disabled recently. - if !AutomaticStatisticsClusterMode.Get(&r.st.SV) { - break + var desc catalog.TableDescriptor + now := timeutil.Now() + elapsed := now.Sub(start) + // If a long-running stats collection caused a delay in + // processing the current table longer than the refresh + // interval, look up the table descriptor to ensure we don't + // have stale table settings. + if elapsed > DefaultRefreshInterval { + desc = r.getTableDescriptor(ctx, tableID) + if desc != nil { + if !r.autoStatsEnabled(desc) { + continue + } + if settingOverrides == nil { + settingOverrides = make(map[descpb.ID]catpb.AutoStatsSettings) + } + autoStatsSettings := desc.GetAutoStatsSettings() + if autoStatsSettings == nil { + delete(settingOverrides, tableID) + } else { + settingOverrides[tableID] = *autoStatsSettings + } + } } - - r.maybeRefreshStats(ctx, stopper, tableID, rowsAffected, r.asOfTime) + if desc == nil { + // Check the cluster setting and table setting before each + // refresh in case they were disabled recently. + if !r.autoStatsEnabledForTableID(tableID, settingOverrides) { + continue + } + } + var explicitSettings *catpb.AutoStatsSettings + if settingOverrides != nil { + if settings, ok := settingOverrides[tableID]; ok { + explicitSettings = &settings + } + } + r.maybeRefreshStats(ctx, tableID, explicitSettings, rowsAffected, r.asOfTime) select { case <-stopper.ShouldQuiesce(): @@ -308,12 +460,27 @@ func (r *Refresher) Start( }); err != nil { log.Errorf(ctx, "failed to refresh stats: %v", err) } + // This clears out any tables that may have been added to the + // mutationCounts map by ensureAllTables and any mutation counts that + // have been added since the last call to maybeRefreshStats. + // This is by design. We don't want to constantly refresh tables that + // are read-only. r.mutationCounts = make(map[descpb.ID]int64, len(r.mutationCounts)) case mut := <-r.mutations: r.mutationCounts[mut.tableID] += int64(mut.rowsAffected) + // The mutations channel also handles resetting of cluster setting + // overrides when none exist (so that we don't have to pass two messages + // when nothing is overridden). + if mut.removeSettingOverrides { + delete(r.settingOverrides, mut.tableID) + } + + case clusterSettingOverride := <-r.settings: + r.settingOverrides[clusterSettingOverride.tableID] = clusterSettingOverride.settings case <-stopper.ShouldQuiesce(): + log.Info(ctx, "quiescing auto stats refresher") return } } @@ -321,19 +488,8 @@ func (r *Refresher) Start( return nil } -// ensureAllTables ensures that an entry exists in r.mutationCounts for each -// table in the database. -func (r *Refresher) ensureAllTables( - ctx context.Context, settings *settings.Values, initialTableCollectionDelay time.Duration, -) { - if !AutomaticStatisticsClusterMode.Get(settings) { - // Automatic stats are disabled. - return - } - - // Use a historical read so as to disable txn contention resolution. - getAllTablesQuery := fmt.Sprintf( - ` +const ( + getAllTablesTemplateSQL = ` SELECT tbl.table_id FROM @@ -346,16 +502,33 @@ WHERE AND tbl.drop_time IS NULL AND ( crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', d.descriptor, false)->'table'->>'viewQuery' - ) IS NULL;`, - initialTableCollectionDelay, - systemschema.SystemDatabaseName, - ) + ) IS NULL + %s` + + explicitlyEnabledTablesPredicate = `AND + (crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' ->> 'enabled' = 'true' + )` + + autoStatsEnabledOrNotSpecifiedPredicate = `AND + (crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings'->'enabled' IS NULL + OR crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', + d.descriptor, false)->'table'->'autoStatsSettings' ->> 'enabled' = 'true' + )` +) +func (r *Refresher) getApplicableTables( + ctx context.Context, stmt string, opname string, forTesting bool, +) { + if forTesting { + r.numTablesEnsured = 0 + } it, err := r.ex.QueryIterator( ctx, - "get-tables", + opname, nil, /* txn */ - getAllTablesQuery, + stmt, ) if err == nil { var ok bool @@ -366,6 +539,9 @@ WHERE // The query already excludes views and system tables. if !descpb.IsVirtualTable(tableID) { r.mutationCounts[tableID] += 0 + if forTesting { + r.numTablesEnsured++ + } } } } @@ -376,8 +552,41 @@ WHERE // entry is idempotent (i.e. we didn't mess up anything for the next // call to this method). log.Errorf(ctx, "failed to get tables for automatic stats: %v", err) + } +} + +// ensureAllTables ensures that an entry exists in r.mutationCounts for each +// table in the database which has auto stats enabled, either explicitly via +// a table-level setting, or implicitly via the cluster setting. +func (r *Refresher) ensureAllTables( + ctx context.Context, settings *settings.Values, initialTableCollectionDelay time.Duration, +) { + if !AutomaticStatisticsClusterMode.Get(settings) { + // Use a historical read so as to disable txn contention resolution. + // A table-level setting of sql_stats_automatic_collection_enabled=true is + // checked and only those tables are included in this scan. + getTablesWithAutoStatsExplicitlyEnabledQuery := fmt.Sprintf( + getAllTablesTemplateSQL, + initialTableCollectionDelay, + systemschema.SystemDatabaseName, + explicitlyEnabledTablesPredicate, + ) + r.getApplicableTables(ctx, getTablesWithAutoStatsExplicitlyEnabledQuery, + "get-tables-with-autostats-explicitly-enabled", false) return } + + // Use a historical read so as to disable txn contention resolution. + // A table-level setting of sql_stats_automatic_collection_enabled of null, + // meaning not set, or true qualifies rows we're interested in. + getAllTablesQuery := fmt.Sprintf( + getAllTablesTemplateSQL, + initialTableCollectionDelay, + systemschema.SystemDatabaseName, + autoStatsEnabledOrNotSpecifiedPredicate, + ) + r.getApplicableTables(ctx, getAllTablesQuery, + "get-tables", false) } // NotifyMutation is called by SQL mutation operations to signal to the @@ -385,8 +594,7 @@ WHERE // successful insert, update, upsert or delete. rowsAffected refers to the // number of rows written as part of the mutation operation. func (r *Refresher) NotifyMutation(table catalog.TableDescriptor, rowsAffected int) { - if !AutomaticStatisticsClusterMode.Get(&r.st.SV) { - // Automatic stats are disabled. + if !r.autoStatsEnabled(table) { return } if !hasStatistics(table) { @@ -394,10 +602,39 @@ func (r *Refresher) NotifyMutation(table catalog.TableDescriptor, rowsAffected i return } + noSettingOverrides := table.NoAutoStatsSettingsOverrides() + var autoStatsSettings *catpb.AutoStatsSettings + if !noSettingOverrides { + autoStatsSettings = table.GetAutoStatsSettings() + } + + // Send setting override information over first, so it could take effect + // before the mutation is processed. + if autoStatsSettings != nil { + autoStatsOverrides := *protoutil.Clone(autoStatsSettings).(*catpb.AutoStatsSettings) + select { + case r.settings <- settingOverride{ + tableID: table.GetID(), + settings: autoStatsOverrides, + }: + default: + // Don't block if there is no room in the buffered channel. + if bufferedChanFullLogLimiter.ShouldLog() { + log.Warningf(context.TODO(), + "buffered channel is full. Unable to update settings for table %q (%d) during auto stats refreshing", + table.GetName(), table.GetID()) + } + } + } + // Send mutation info to the refresher thread to avoid adding latency to // the calling transaction. select { - case r.mutations <- mutation{tableID: table.GetID(), rowsAffected: rowsAffected}: + case r.mutations <- mutation{ + tableID: table.GetID(), + rowsAffected: rowsAffected, + removeSettingOverrides: noSettingOverrides, + }: default: // Don't block if there is no room in the buffered channel. if bufferedChanFullLogLimiter.ShouldLog() { @@ -410,10 +647,12 @@ func (r *Refresher) NotifyMutation(table catalog.TableDescriptor, rowsAffected i // maybeRefreshStats implements the core logic described in the comment for // Refresher. It is called by the background Refresher thread. +// explicitSettings, if non-nil, holds any autostats cluster setting overrides +// for this table. func (r *Refresher) maybeRefreshStats( ctx context.Context, - stopper *stop.Stopper, tableID descpb.ID, + explicitSettings *catpb.AutoStatsSettings, rowsAffected int64, asOf time.Duration, ) { @@ -453,9 +692,15 @@ func (r *Refresher) maybeRefreshStats( mustRefresh = true } - targetRows := int64(rowCount*AutomaticStatisticsFractionStaleRows.Get(&r.st.SV)) + - AutomaticStatisticsMinStaleRows.Get(&r.st.SV) - if !mustRefresh && rowsAffected < math.MaxInt32 && r.randGen.randInt(targetRows) >= rowsAffected { + statsFractionStaleRows := r.autoStatsFractionStaleRows(explicitSettings) + statsMinStaleRows := r.autoStatsMinStaleRows(explicitSettings) + targetRows := int64(rowCount*statsFractionStaleRows) + statsMinStaleRows + // randInt will panic if we pass it a value of 0. + randomTargetRows := int64(0) + if targetRows > 0 { + randomTargetRows = r.randGen.randInt(targetRows) + } + if !mustRefresh && rowsAffected < math.MaxInt32 && randomTargetRows >= rowsAffected { // No refresh is happening this time. return } diff --git a/pkg/sql/stats/automatic_stats_test.go b/pkg/sql/stats/automatic_stats_test.go index 592c6d7c5cc5..ce1a12256445 100644 --- a/pkg/sql/stats/automatic_stats_test.go +++ b/pkg/sql/stats/automatic_stats_test.go @@ -24,9 +24,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -45,6 +47,7 @@ func TestMaybeRefreshStats(t *testing.T) { ctx := context.Background() s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer sqlDB.Close() defer s.Stopper().Stop(ctx) st := cluster.MakeTestingClusterSettings() @@ -83,7 +86,7 @@ func TestMaybeRefreshStats(t *testing.T) { // There are no stats yet, so this must refresh the statistics on table t // even though rowsAffected=0. refresher.maybeRefreshStats( - ctx, s.Stopper(), descA.GetID(), 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, descA.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ ) if err := checkStatsCount(ctx, cache, descA, 1 /* expected */); err != nil { t.Fatal(err) @@ -92,7 +95,28 @@ func TestMaybeRefreshStats(t *testing.T) { // Try to refresh again. With rowsAffected=0, the probability of a refresh // is 0, so refreshing will not succeed. refresher.maybeRefreshStats( - ctx, s.Stopper(), descA.GetID(), 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, descA.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ) + if err := checkStatsCount(ctx, cache, descA, 1 /* expected */); err != nil { + t.Fatal(err) + } + + // Setting minStaleRows for the table prevents refreshing from occurring. + minStaleRows := int64(100000000) + explicitSettings := catpb.AutoStatsSettings{MinStaleRows: &minStaleRows} + refresher.maybeRefreshStats( + ctx, descA.GetID(), &explicitSettings, 10 /* rowsAffected */, time.Microsecond, /* asOf */ + ) + if err := checkStatsCount(ctx, cache, descA, 1 /* expected */); err != nil { + t.Fatal(err) + } + + // Setting fractionStaleRows for the table can also prevent refreshing from + // occurring, though this is a not a typical value for this setting. + fractionStaleRows := float64(100000000) + explicitSettings = catpb.AutoStatsSettings{FractionStaleRows: &fractionStaleRows} + refresher.maybeRefreshStats( + ctx, descA.GetID(), &explicitSettings, 10 /* rowsAffected */, time.Microsecond, /* asOf */ ) if err := checkStatsCount(ctx, cache, descA, 1 /* expected */); err != nil { t.Fatal(err) @@ -101,7 +125,7 @@ func TestMaybeRefreshStats(t *testing.T) { // With rowsAffected=10, refreshing should work. Since there are more rows // updated than exist in the table, the probability of a refresh is 100%. refresher.maybeRefreshStats( - ctx, s.Stopper(), descA.GetID(), 10 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, descA.GetID(), nil /* explicitSettings */, 10 /* rowsAffected */, time.Microsecond, /* asOf */ ) if err := checkStatsCount(ctx, cache, descA, 2 /* expected */); err != nil { t.Fatal(err) @@ -112,7 +136,7 @@ func TestMaybeRefreshStats(t *testing.T) { // TODO(rytaft): Should not enqueue views to begin with. descVW := desctestutils.TestingGetPublicTableDescriptor(s.DB(), keys.SystemSQLCodec, "t", "vw") refresher.maybeRefreshStats( - ctx, s.Stopper(), descVW.GetID(), 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, descVW.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ ) select { case <-refresher.mutations: @@ -121,12 +145,110 @@ func TestMaybeRefreshStats(t *testing.T) { } } +func TestEnsureAllTablesQueries(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer sqlDB.Close() + defer s.Stopper().Stop(ctx) + + st := cluster.MakeTestingClusterSettings() + + sqlRun := sqlutils.MakeSQLRunner(sqlDB) + sqlRun.Exec(t, + `CREATE DATABASE t; + CREATE TABLE t.a (k INT PRIMARY KEY);`) + + sqlRun.Exec(t, `CREATE TABLE t.b (k INT PRIMARY KEY);`) + + executor := s.InternalExecutor().(sqlutil.InternalExecutor) + cache := NewTableStatisticsCache( + ctx, + 10, /* cacheSize */ + kvDB, + executor, + keys.SystemSQLCodec, + s.ClusterSettings(), + s.RangeFeedFactory().(*rangefeed.Factory), + s.CollectionFactory().(*descs.CollectionFactory), + ) + r := MakeRefresher(s.AmbientCtx(), st, executor, cache, time.Microsecond /* asOfTime */) + + if err := checkAllTablesCount(ctx, 2, r); err != nil { + t.Fatal(err) + } + if err := checkExplicitlyEnabledTablesCount(ctx, 0, r); err != nil { + t.Fatal(err) + } + sqlRun.Exec(t, + `ALTER TABLE t.a SET (sql_stats_automatic_collection_enabled = true)`) + if err := checkAllTablesCount(ctx, 2, r); err != nil { + t.Fatal(err) + } + if err := checkExplicitlyEnabledTablesCount(ctx, 1, r); err != nil { + t.Fatal(err) + } + sqlRun.Exec(t, + `ALTER TABLE t.b SET (sql_stats_automatic_collection_enabled = false)`) + if err := checkAllTablesCount(ctx, 1, r); err != nil { + t.Fatal(err) + } + if err := checkExplicitlyEnabledTablesCount(ctx, 1, r); err != nil { + t.Fatal(err) + } + sqlRun.Exec(t, + `ALTER TABLE t.a SET (sql_stats_automatic_collection_enabled = false)`) + if err := checkAllTablesCount(ctx, 0, r); err != nil { + t.Fatal(err) + } + if err := checkExplicitlyEnabledTablesCount(ctx, 0, r); err != nil { + t.Fatal(err) + } +} + +func checkAllTablesCount(ctx context.Context, expected int, r *Refresher) error { + const collectionDelay = time.Microsecond + getAllTablesQuery := fmt.Sprintf( + getAllTablesTemplateSQL, + collectionDelay, + systemschema.SystemDatabaseName, + autoStatsEnabledOrNotSpecifiedPredicate, + ) + r.getApplicableTables(ctx, getAllTablesQuery, + "get-tables", true) + actual := r.getNumTablesEnsured() + if expected != actual { + return fmt.Errorf("expected %d table(s) but found %d", expected, actual) + } + return nil +} + +func checkExplicitlyEnabledTablesCount(ctx context.Context, expected int, r *Refresher) error { + const collectionDelay = time.Microsecond + getTablesWithAutoStatsExplicitlyEnabledQuery := fmt.Sprintf( + getAllTablesTemplateSQL, + collectionDelay, + systemschema.SystemDatabaseName, + explicitlyEnabledTablesPredicate, + ) + r.getApplicableTables(ctx, getTablesWithAutoStatsExplicitlyEnabledQuery, + "get-tables-with-autostats-explicitly-enabled", true) + actual := r.getNumTablesEnsured() + if expected != actual { + return fmt.Errorf("expected %d table(s) but found %d", expected, actual) + } + return nil +} + func TestAverageRefreshTime(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer sqlDB.Close() defer s.Stopper().Stop(ctx) st := cluster.MakeTestingClusterSettings() @@ -311,7 +433,7 @@ func TestAverageRefreshTime(t *testing.T) { // the statistics on table t. With rowsAffected=0, the probability of refresh // is 0. refresher.maybeRefreshStats( - ctx, s.Stopper(), table.GetID(), 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, table.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ ) if err := checkStatsCount(ctx, cache, table, 20 /* expected */); err != nil { t.Fatal(err) @@ -361,7 +483,7 @@ func TestAverageRefreshTime(t *testing.T) { // remain (5 from column k and 10 from column v), since the old stats on k // were deleted. refresher.maybeRefreshStats( - ctx, s.Stopper(), table.GetID(), 0 /* rowsAffected */, time.Microsecond, /* asOf */ + ctx, table.GetID(), nil /* explicitSettings */, 0 /* rowsAffected */, time.Microsecond, /* asOf */ ) if err := checkStatsCount(ctx, cache, table, 15 /* expected */); err != nil { t.Fatal(err) @@ -374,6 +496,7 @@ func TestAutoStatsReadOnlyTables(t *testing.T) { ctx := context.Background() s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + defer sqlDB.Close() defer s.Stopper().Stop(ctx) st := cluster.MakeTestingClusterSettings() @@ -454,7 +577,8 @@ func TestNoRetryOnFailure(t *testing.T) { // Try to refresh stats on a table that doesn't exist. r.maybeRefreshStats( - ctx, s.Stopper(), 100 /* tableID */, math.MaxInt32, time.Microsecond, /* asOfTime */ + ctx, 100 /* tableID */, nil /* explicitSettings */, math.MaxInt32, + time.Microsecond, /* asOfTime */ ) // Ensure that we will not try to refresh tableID 100 again. @@ -463,7 +587,7 @@ func TestNoRetryOnFailure(t *testing.T) { } } -func TestMutationsChannel(t *testing.T) { +func TestMutationsAndSettingOverrideChannels(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() @@ -475,6 +599,7 @@ func TestMutationsChannel(t *testing.T) { r := Refresher{ st: st, mutations: make(chan mutation, refreshChanBufferLen), + settings: make(chan settingOverride, refreshChanBufferLen), } tbl := descpb.TableDescriptor{ID: 53, ParentID: 52, Name: "foo"} @@ -489,6 +614,22 @@ func TestMutationsChannel(t *testing.T) { if expected, actual := refreshChanBufferLen, len(r.mutations); expected != actual { t.Fatalf("expected channel size %d but found %d", expected, actual) } + + // Test that the settings channel doesn't block even when we add 10 more + // items than can fit in the buffer. + autoStatsSettings := &catpb.AutoStatsSettings{} + tableDesc.TableDesc().AutoStatsSettings = autoStatsSettings + minStaleRows := int64(1) + autoStatsSettings.MinStaleRows = &minStaleRows + for i := 0; i < refreshChanBufferLen+10; i++ { + int64CurrIteration := int64(i) + autoStatsSettings.MinStaleRows = &int64CurrIteration + r.NotifyMutation(tableDesc, 5 /* rowsAffected */) + } + + if expected, actual := refreshChanBufferLen, len(r.settings); expected != actual { + t.Fatalf("expected channel size %d but found %d", expected, actual) + } } func TestDefaultColumns(t *testing.T) { @@ -497,6 +638,7 @@ func TestDefaultColumns(t *testing.T) { ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer sqlDB.Close() defer s.Stopper().Stop(ctx) st := cluster.MakeTestingClusterSettings()