From dd64e290f7c31c045e18c9b34f646dd55dc36617 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Fri, 18 Aug 2023 13:46:02 +0200 Subject: [PATCH] settings,*: use the option pattern with the setting register functions TLDR: this patch introduces the "functional option style" in the settings API. For example, this call: ```go var s = func() *settings.BoolSetting { s := settings.RegisterBoolSetting(...).WithPublic() s.SetReportable(true) }() ``` can now be written as: ```go var s = settings.RegisterBoolSetting(..., settings.WithPublic, settings.WithReportable(true), ) ``` Internally, the register function now take a last argument of type `...SettingOption`. The following options are defined: ```go var Retired SettingOption var WithPublic SettingOption func WithReportable(reportable bool) SettingOption func WithVisibility(v Visibility) SettingOption func WithValidateDuration(fn func(time.Duration) error) SettingOption // NEW var NonNegativeDuration SettingOption var PositiveDuration SettingOption func NonNegativeDurationWithMaximum(maxValue time.Duration) SettingOption func NonNegativeDurationWithMinimum(minValue time.Duration) SettingOption func NonNegativeDurationWithMinimumOrZeroDisable(minValue time.Duration) SettingOption // NEW func DurationInRange(minVal, maxVal int64) SettingOption // NEW func WithValidateFloat(fn func(float64) error) SettingOption // NEW var NonNegativeFloat SettingOption var PositiveFloat SettingOption func NonNegativeFloatWithMaximum(maxValue float64) SettingOption func NonNegativeFloatWithMinimum(minValue float64) SettingOption // NEW func NonNegativeFloatWithMinimumOrZeroDisable(minValue float64) SettingOption // NEW var NonZeroFloat SettingOption // NEW var Fraction SettingOption // NEW var FractionUpperExclusive SettingOption // NEW func FloatInRange(minVal, maxVal float64) SettingOption // NEW func FloatInRangeUpperExclusive(minVal, maxVal float64) SettingOption // NEW func WithValidateInt(fn func(int64) error) SettingOption // NEW var NonNegativeInt SettingOption var PositiveInt SettingOption func NonNegativeIntWithMaximum(maxValue int64) SettingOption func NonNegativeIntWithMinimum(minValue int64) SettingOption // NEW func IntInRange(minVal, maxVal int64) SettingOption // NEW func IntInRangeOrZeroDisable(minVal, maxVal int64) SettingOption // NEW func WithValidateProto(fn func(*Values, protoutil.Message) error) SettingOption // NEW func WithValidateString(fn func(*Values, string) error) SettingOption // NEW func ByteSizeWithMinimum(minVal int64) SettingOption // NEW ``` Release note: None --- pkg/ccl/auditloggingccl/audit_log_config.go | 10 +- pkg/ccl/backupccl/backup_planning.go | 2 +- pkg/ccl/backupccl/backup_processor.go | 6 +- .../backupdest/backup_destination.go | 2 +- pkg/ccl/backupccl/create_scheduled_backup.go | 2 +- pkg/ccl/backupccl/restore_planning.go | 2 +- pkg/ccl/changefeedccl/changefeed_dist.go | 2 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 2 +- .../changefeedccl/changefeedbase/settings.go | 51 ++- pkg/ccl/changefeedccl/compression.go | 2 +- .../changefeedccl/schemafeed/schema_feed.go | 2 +- pkg/ccl/jwtauthccl/settings.go | 87 +++-- .../tenantcostclient/tenant_side.go | 23 +- pkg/ccl/oidcccl/settings.go | 300 ++++++++---------- .../stream_ingestion_processor.go | 4 +- pkg/ccl/utilccl/license_check.go | 21 +- pkg/cloud/azure/azure_storage.go | 2 +- pkg/cloud/cloud_io.go | 4 +- pkg/clusterversion/setting.go | 62 ++-- pkg/jobs/config.go | 2 +- pkg/keyvisualizer/keyvissettings/settings.go | 8 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 10 +- .../kvclient/kvcoord/dist_sender_rangefeed.go | 2 +- .../kvcoord/txn_interceptor_pipeliner.go | 4 +- .../kvcoord/txn_interceptor_span_refresher.go | 2 +- pkg/kv/kvprober/settings.go | 69 ++-- pkg/kv/kvserver/allocator/BUILD.bazel | 1 - .../allocator/allocatorimpl/allocator.go | 4 +- .../allocatorimpl/allocator_scorer.go | 49 +-- pkg/kv/kvserver/allocator/base.go | 87 ++--- pkg/kv/kvserver/batcheval/cmd_export.go | 4 +- .../concurrency/concurrency_manager.go | 4 +- pkg/kv/kvserver/consistency_queue.go | 2 +- pkg/kv/kvserver/gc/gc.go | 7 +- pkg/kv/kvserver/kvadmission/kvadmission.go | 28 +- .../kvflowcontroller/kvflowcontroller.go | 4 +- pkg/kv/kvserver/kvserverbase/BUILD.bazel | 1 - pkg/kv/kvserver/kvserverbase/base.go | 17 +- pkg/kv/kvserver/liveness/liveness.go | 54 ++-- .../protectedts/ptreconcile/reconciler.go | 2 +- pkg/kv/kvserver/protectedts/settings.go | 7 +- pkg/kv/kvserver/queue.go | 5 +- pkg/kv/kvserver/raft_log_queue.go | 16 +- pkg/kv/kvserver/rebalance_objective.go | 2 +- pkg/kv/kvserver/replica_backpressure.go | 7 +- pkg/kv/kvserver/replica_circuit_breaker.go | 20 +- pkg/kv/kvserver/replica_follower_read.go | 2 +- pkg/kv/kvserver/replica_raft_overload.go | 12 +- pkg/kv/kvserver/replica_rangefeed.go | 2 +- pkg/kv/kvserver/replica_split_load.go | 28 +- pkg/kv/kvserver/replicastats/replica_stats.go | 2 +- pkg/kv/kvserver/reports/reporter.go | 2 +- pkg/kv/kvserver/store.go | 51 ++- pkg/kv/kvserver/store_rebalancer.go | 2 +- pkg/kv/kvserver/store_snapshot.go | 31 +- pkg/kv/kvserver/tenantrate/settings.go | 8 +- pkg/multitenant/tenant_config.go | 2 +- pkg/multitenant/tenantcostmodel/settings.go | 19 +- pkg/roachpb/span_stats.go | 8 +- pkg/security/cert_expiry_cache.go | 2 +- pkg/security/password.go | 22 +- pkg/security/tls_settings.go | 5 +- pkg/server/authserver/authentication.go | 2 +- pkg/server/clock_monotonicity.go | 4 +- pkg/server/cluster_settings.go | 4 +- pkg/server/debug/server.go | 13 +- pkg/server/diagnostics/reporter.go | 2 +- pkg/server/drain.go | 8 +- pkg/server/hot_ranges.go | 2 +- pkg/server/node.go | 6 +- pkg/server/profiler/cluster_settings.go | 2 +- pkg/server/profiler/profilestore.go | 10 +- pkg/server/server_http.go | 2 +- pkg/server/server_systemlog_gc.go | 10 +- pkg/server/settings_test.go | 18 +- pkg/server/status/recorder.go | 3 +- pkg/settings/BUILD.bazel | 2 + pkg/settings/bool.go | 9 +- pkg/settings/byte_size.go | 42 ++- pkg/settings/common.go | 23 +- pkg/settings/duration.go | 126 +++++--- pkg/settings/duration_test.go | 27 ++ pkg/settings/enum.go | 14 +- pkg/settings/float.go | 115 +++++-- pkg/settings/int.go | 89 ++++-- pkg/settings/options.go | 88 +++++ pkg/settings/protobuf.go | 41 +-- pkg/settings/settings_test.go | 24 +- pkg/settings/string.go | 40 +-- pkg/settings/version.go | 5 +- pkg/spanconfig/spanconfigreporter/reporter.go | 15 +- pkg/spanconfig/spanconfigstore/span_store.go | 4 +- pkg/spanconfig/spanconfigstore/store.go | 2 +- pkg/sql/alter_database.go | 2 +- pkg/sql/alter_role.go | 2 +- pkg/sql/authorization.go | 2 +- pkg/sql/catalog/lease/lease.go | 9 +- .../schematelemetrycontroller/controller.go | 9 +- pkg/sql/catalog/tabledesc/table.go | 2 +- pkg/sql/catalog/tabledesc/validate.go | 9 +- pkg/sql/closed_session_cache.go | 4 +- pkg/sql/conn_executor.go | 2 +- pkg/sql/contention/cluster_settings.go | 4 +- .../contention/txnidcache/cluster_settings.go | 2 +- pkg/sql/create_stats.go | 4 +- pkg/sql/deprecatedshowranges/condition.go | 2 +- pkg/sql/descriptor.go | 4 +- pkg/sql/distsql_plan_stats.go | 2 +- pkg/sql/distsql_running.go | 10 +- pkg/sql/event_log.go | 2 +- pkg/sql/exec_log.go | 11 +- pkg/sql/exec_util.go | 163 +++++----- pkg/sql/export.go | 2 +- pkg/sql/idxusage/cluster_settings.go | 2 +- pkg/sql/importer/export_base.go | 8 +- pkg/sql/importer/import_planning.go | 2 +- pkg/sql/instrumentation.go | 7 +- pkg/sql/notice.go | 2 +- .../opt/optbuilder/mutation_builder_unique.go | 2 +- pkg/sql/opt/optbuilder/util.go | 2 +- pkg/sql/pgwire/auth_methods.go | 2 +- pkg/sql/pgwire/hba_conf.go | 19 +- pkg/sql/pgwire/ident_map_conf.go | 26 +- pkg/sql/pgwire/pgwirebase/encoding.go | 7 +- pkg/sql/pgwire/server.go | 8 +- pkg/sql/row/helper.go | 37 +-- pkg/sql/schema_change_cluster_setting.go | 2 +- .../internal/scbuildstmt/statement_control.go | 46 ++- pkg/sql/sem/eval/settings.go | 2 +- pkg/sql/sessioninit/cache.go | 2 +- pkg/sql/sql_activity_update_job.go | 2 +- .../instancestorage/instancestorage.go | 14 +- pkg/sql/sqlliveness/slstorage/slstorage.go | 7 +- pkg/sql/sqlstats/cluster_settings.go | 28 +- pkg/sql/sqlstats/insights/insights.go | 12 +- .../persistedsqlstats/cluster_settings.go | 22 +- pkg/sql/stats/BUILD.bazel | 2 - pkg/sql/stats/automatic_stats.go | 56 ++-- pkg/sql/stats/delete_stats.go | 2 +- pkg/sql/stats/forecast.go | 2 +- pkg/sql/stats/histogram.go | 4 +- .../stmtdiagnostics/statement_diagnostics.go | 7 +- .../tablestorageparam/BUILD.bazel | 1 - .../tablestorageparam/table_storage_param.go | 31 +- pkg/sql/telemetry_logging.go | 2 +- pkg/sql/temporary_schema.go | 4 +- pkg/sql/tenant_accessors.go | 21 +- pkg/sql/ttl/ttljob/ttljob.go | 8 +- pkg/sql/txn_fingerprint_id_cache.go | 2 +- pkg/sql/user.go | 2 +- pkg/storage/pebble.go | 7 +- pkg/ts/db.go | 6 +- pkg/ui/ui.go | 2 +- pkg/util/admission/io_load_listener.go | 3 +- .../admission/scheduler_latency_listener.go | 36 +-- pkg/util/admission/work_queue.go | 30 +- pkg/util/log/logcrash/crash_reporting.go | 2 +- pkg/util/schedulerlatency/sampler.go | 8 +- pkg/util/tracing/tracer.go | 33 +- 159 files changed, 1253 insertions(+), 1480 deletions(-) create mode 100644 pkg/settings/duration_test.go create mode 100644 pkg/settings/options.go diff --git a/pkg/ccl/auditloggingccl/audit_log_config.go b/pkg/ccl/auditloggingccl/audit_log_config.go index 630f6a281a70..732e6e6a271b 100644 --- a/pkg/ccl/auditloggingccl/audit_log_config.go +++ b/pkg/ccl/auditloggingccl/audit_log_config.go @@ -23,13 +23,14 @@ import ( const auditConfigDefaultValue = "" // UserAuditLogConfig is a cluster setting that takes a user/role-based audit configuration. -var UserAuditLogConfig = settings.RegisterValidatedStringSetting( +var UserAuditLogConfig = settings.RegisterStringSetting( settings.TenantWritable, "sql.log.user_audit", "user/role-based audit logging configuration. An enterprise license is required for this cluster setting to take effect.", auditConfigDefaultValue, - validateAuditLogConfig, -).WithPublic() + settings.WithValidateString(validateAuditLogConfig), + settings.WithPublic, +) // UserAuditEnableReducedConfig is a cluster setting that enables/disables a computed // reduced configuration. This allows us to compute the audit configuration once at @@ -46,7 +47,8 @@ var UserAuditEnableReducedConfig = settings.RegisterBoolSetting( "(user role memberships/cluster setting) are not reflected within session. "+ "Users will need to start a new session to see these changes in their auditing behaviour.", false, -).WithPublic() + settings.WithPublic, +) func validateAuditLogConfig(_ *settings.Values, input string) error { if input == auditConfigDefaultValue { diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 89ced6b9325a..fceecddad189 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -87,7 +87,7 @@ var featureBackupEnabled = settings.RegisterBoolSetting( "feature.backup.enabled", "set to true to enable backups, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) // includeTableSpans returns true if the backup should include spans for the // given table descriptor. diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index 07e815a945d5..46e5d8d3103a 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -56,7 +56,7 @@ var ( "amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads", time.Minute, settings.NonNegativeDuration, - ).WithPublic() + settings.WithPublic) delayPerAttempt = settings.RegisterDurationSetting( settings.TenantWritable, "bulkio.backup.read_retry_delay", @@ -70,13 +70,13 @@ var ( "amount of time after which a read attempt is considered timed out, which causes the backup to fail", time.Minute*5, settings.NonNegativeDuration, - ).WithPublic() + settings.WithPublic) targetFileSize = settings.RegisterByteSizeSetting( settings.TenantWritable, "bulkio.backup.file_size", "target size for individual data files produced during BACKUP", 128<<20, - ).WithPublic() + settings.WithPublic) splitKeysOnTimestamps = settings.RegisterBoolSetting( settings.TenantWritable, diff --git a/pkg/ccl/backupccl/backupdest/backup_destination.go b/pkg/ccl/backupccl/backupdest/backup_destination.go index d55d1525fa25..707610869a7f 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination.go @@ -64,7 +64,7 @@ var featureFullBackupUserSubdir = settings.RegisterBoolSetting( "when true, a backup command with a user specified subdirectory will create a full backup at"+ " the subdirectory if no backup already exists at that subdirectory", false, -).WithPublic() + settings.WithPublic) // TODO(adityamaru): Move this to the soon to be `backupinfo` package. func containsManifest(ctx context.Context, exportStore cloud.ExternalStorage) (bool, error) { diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 78d40a601d4f..09c8e6529c48 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -60,7 +60,7 @@ var scheduledBackupGCProtectionEnabled = settings.RegisterBoolSetting( "schedules.backup.gc_protection.enabled", "enable chaining of GC protection across backups run as part of a schedule", true, /* defaultValue */ -).WithPublic() + settings.WithPublic) // scheduledBackupSpec is a representation of tree.ScheduledBackup, prepared // for evaluation diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index adac62311190..1e1719ff1666 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -96,7 +96,7 @@ var featureRestoreEnabled = settings.RegisterBoolSetting( "feature.restore.enabled", "set to true to enable restore, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) // maybeFilterMissingViews filters the set of tables to restore to exclude views // whose dependencies are either missing or are themselves unrestorable due to diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index 68939f5ca1a3..03b3423b5c1b 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -321,7 +321,7 @@ var enableBalancedRangeDistribution = settings.RegisterBoolSetting( "if enabled, the ranges are balanced equally among all nodes", util.ConstantWithMetamorphicTestBool( "changefeed.balance_range_distribution.enable", false), -).WithPublic() + settings.WithPublic) func makePlan( execCtx sql.JobExecContext, diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index be05cc727c26..2482ae271f0d 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -67,7 +67,7 @@ var featureChangefeedEnabled = settings.RegisterBoolSetting( "feature.changefeed.enabled", "set to true to enable changefeeds, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) func init() { sql.AddPlanHook("changefeed", changefeedPlanHook, changefeedTypeCheck) diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index 1458bcd9b148..cc85d5cdd993 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -14,7 +14,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/errors" ) // TableDescriptorPollInterval controls how fast table descriptors are polled. A @@ -49,7 +48,7 @@ var PerChangefeedMemLimit = settings.RegisterByteSizeSetting( "changefeed.memory.per_changefeed_limit", "controls amount of data that can be buffered per changefeed", 1<<29, // 512MiB -).WithPublic() + settings.WithPublic) // SlowSpanLogThreshold controls when we will log slow spans. var SlowSpanLogThreshold = settings.RegisterDurationSetting( @@ -88,7 +87,7 @@ var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( "controls the maximum the high-water mark is allowed to lag behind the leading spans of the frontier before per-span checkpointing is enabled; if 0, checkpointing due to high-water lag is disabled", 10*time.Minute, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // FrontierCheckpointMaxBytes controls the maximum number of key bytes that will be added // to the checkpoint record. @@ -119,7 +118,7 @@ var ScanRequestLimit = settings.RegisterIntSetting( "changefeed.backfill.concurrent_scan_requests", "number of concurrent scan requests per node issued during a backfill", 0, -).WithPublic() + settings.WithPublic) // ScanRequestSize is the target size of the scan request response. // @@ -130,7 +129,7 @@ var ScanRequestSize = settings.RegisterIntSetting( "changefeed.backfill.scan_request_size", "the maximum number of bytes returned by each scan request", 1<<19, // 1/2 MiB -).WithPublic() + settings.WithPublic) // SinkThrottleConfig describes throttling configuration for the sink. // 0 values for any of the settings disable that setting. @@ -150,18 +149,15 @@ type SinkThrottleConfig struct { } // NodeSinkThrottleConfig is the node wide throttling configuration for changefeeds. -var NodeSinkThrottleConfig = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - "changefeed.node_throttle_config", - "specifies node level throttling configuration for all changefeeeds", - "", - validateSinkThrottleConfig, - ) - s.SetVisibility(settings.Public) - s.SetReportable(true) - return s -}() +var NodeSinkThrottleConfig = settings.RegisterStringSetting( + settings.TenantWritable, + "changefeed.node_throttle_config", + "specifies node level throttling configuration for all changefeeeds", + "", + settings.WithValidateString(validateSinkThrottleConfig), + settings.WithPublic, + settings.WithReportable(true), +) func validateSinkThrottleConfig(values *settings.Values, configStr string) error { if configStr == "" { @@ -181,7 +177,7 @@ var MinHighWaterMarkCheckpointAdvance = settings.RegisterDurationSetting( "advances, as long as the rate of checkpointing keeps up with the rate of frontier changes", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // EventMemoryMultiplier is the multiplier for the amount of memory needed to process an event. // @@ -194,12 +190,7 @@ var EventMemoryMultiplier = settings.RegisterFloatSetting( "changefeed.event_memory_multiplier", "the amount of memory required to process an event is multiplied by this factor", 3, - func(v float64) error { - if v < 1 { - return errors.New("changefeed.event_memory_multiplier must be at least 1") - } - return nil - }, + settings.NonNegativeFloatWithMinimum(1), ) // ProtectTimestampInterval controls the frequency of protected timestamp record updates @@ -209,7 +200,7 @@ var ProtectTimestampInterval = settings.RegisterDurationSetting( "controls how often the changefeed forwards its protected timestamp to the resolved timestamp", 10*time.Minute, settings.PositiveDuration, -).WithPublic() + settings.WithPublic) // MaxProtectedTimestampAge controls the frequency of protected timestamp record updates var MaxProtectedTimestampAge = settings.RegisterDurationSetting( @@ -218,7 +209,7 @@ var MaxProtectedTimestampAge = settings.RegisterDurationSetting( "fail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expiration", 4*24*time.Hour, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // BatchReductionRetryEnabled enables the temporary reduction of batch sizes upon kafka message too large errors var BatchReductionRetryEnabled = settings.RegisterBoolSetting( @@ -226,7 +217,7 @@ var BatchReductionRetryEnabled = settings.RegisterBoolSetting( "changefeed.batch_reduction_retry_enabled", "if true, kafka changefeeds upon erroring on an oversized batch will attempt to resend the messages with progressively lower batch sizes", false, -).WithPublic() + settings.WithPublic) // UseMuxRangeFeed enables the use of MuxRangeFeed RPC. var UseMuxRangeFeed = settings.RegisterBoolSetting( @@ -245,7 +236,7 @@ var EventConsumerWorkers = settings.RegisterIntSetting( "0 assigns a reasonable default, >0 assigns the setting value. for experimental/core "+ "changefeeds and changefeeds using parquet format, this is disabled", 0, -).WithPublic() + settings.WithPublic) // EventConsumerWorkerQueueSize specifies the maximum number of events a worker buffer. var EventConsumerWorkerQueueSize = settings.RegisterIntSetting( @@ -255,7 +246,7 @@ var EventConsumerWorkerQueueSize = settings.RegisterIntSetting( "which a worker can buffer", int64(util.ConstantWithMetamorphicTestRange("changefeed.event_consumer_worker_queue_size", 16, 0, 16)), settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // EventConsumerPacerRequestSize specifies how often (measured in CPU time) // that event consumer workers request CPU time from admission control. @@ -299,7 +290,7 @@ var SinkIOWorkers = settings.RegisterIntSetting( "the number of workers used by changefeeds when sending requests to the sink "+ "(currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value", 0, -).WithPublic() + settings.WithPublic) // SinkPacerRequestSize specifies how often (measured in CPU time) // that the Sink batching worker request CPU time from admission control. For diff --git a/pkg/ccl/changefeedccl/compression.go b/pkg/ccl/changefeedccl/compression.go index c4d744a1445b..a2be6f197948 100644 --- a/pkg/ccl/changefeedccl/compression.go +++ b/pkg/ccl/changefeedccl/compression.go @@ -27,7 +27,7 @@ var useFastGzip = settings.RegisterBoolSetting( util.ConstantWithMetamorphicTestBool( "changefeed.fast_gzip.enabled", true, ), -).WithPublic() + settings.WithPublic) type compressionAlgo string diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index f586c883bc06..74c3cd992824 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -728,7 +728,7 @@ var highPriorityAfter = settings.RegisterDurationSetting( "changefeed.schema_feed.read_with_priority_after", "retry with high priority if we were not able to read descriptors for too long; 0 disables", time.Minute, -).WithPublic() + settings.WithPublic) // sendExportRequestWithPriorityOverride uses KV API Export() to dump all kv pairs // whose key falls under `span` and whose mvcc timestamp falls within [startTs, endTS]. diff --git a/pkg/ccl/jwtauthccl/settings.go b/pkg/ccl/jwtauthccl/settings.go index 633dbfdafb0c..fa90a50c6436 100644 --- a/pkg/ccl/jwtauthccl/settings.go +++ b/pkg/ccl/jwtauthccl/settings.go @@ -28,65 +28,50 @@ const ( ) // JWTAuthClaim sets the JWT claim that is parsed to get the username. -var JWTAuthClaim = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - JWTAuthClaimSettingName, - "sets the JWT claim that is parsed to get the username", - "", - ) - s.SetReportable(true) - return s -}() +var JWTAuthClaim = settings.RegisterStringSetting( + settings.TenantWritable, + JWTAuthClaimSettingName, + "sets the JWT claim that is parsed to get the username", + "", + settings.WithReportable(true), +) // JWTAuthAudience sets accepted audience values for JWT logins over the SQL interface. -var JWTAuthAudience = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - JWTAuthAudienceSettingName, - "sets accepted audience values for JWT logins over the SQL interface", - "", - validateJWTAuthAudiences, - ) - return s -}() +var JWTAuthAudience = settings.RegisterStringSetting( + settings.TenantWritable, + JWTAuthAudienceSettingName, + "sets accepted audience values for JWT logins over the SQL interface", + "", + settings.WithValidateString(validateJWTAuthAudiences), +) // JWTAuthEnabled enables or disabled JWT login over the SQL interface. -var JWTAuthEnabled = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.TenantWritable, - JWTAuthEnabledSettingName, - "enables or disabled JWT login for the SQL interface", - false, - ) - s.SetReportable(true) - return s -}() +var JWTAuthEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + JWTAuthEnabledSettingName, + "enables or disabled JWT login for the SQL interface", + false, + settings.WithReportable(true), +) // JWTAuthJWKS is the public key set for JWT logins over the SQL interface. -var JWTAuthJWKS = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - JWTAuthJWKSSettingName, - "sets the public key set for JWT logins over the SQL interface (JWKS format)", - "{\"keys\":[]}", - validateJWTAuthJWKS, - ) - return s -}() +var JWTAuthJWKS = settings.RegisterStringSetting( + settings.TenantWritable, + JWTAuthJWKSSettingName, + "sets the public key set for JWT logins over the SQL interface (JWKS format)", + "{\"keys\":[]}", + settings.WithValidateString(validateJWTAuthJWKS), +) // JWTAuthIssuers is the list of "issuer" values that are accepted for JWT logins over the SQL interface. -var JWTAuthIssuers = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - JWTAuthIssuersSettingName, - "sets accepted issuer values for JWT logins over the SQL interface either as a string or as a JSON "+ - "string with an array of issuer strings in it", - "", - validateJWTAuthIssuers, - ) - return s -}() +var JWTAuthIssuers = settings.RegisterStringSetting( + settings.TenantWritable, + JWTAuthIssuersSettingName, + "sets accepted issuer values for JWT logins over the SQL interface either as a string or as a JSON "+ + "string with an array of issuer strings in it", + "", + settings.WithValidateString(validateJWTAuthIssuers), +) func validateJWTAuthIssuers(values *settings.Values, s string) error { var issuers []string diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go index 2a19528d6764..ba59326fe415 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go @@ -39,7 +39,7 @@ var TargetPeriodSetting = settings.RegisterDurationSetting( "tenant_cost_control_period", "target duration between token bucket requests from tenants (requires restart)", 10*time.Second, - checkDurationInRange(5*time.Second, 120*time.Second), + settings.DurationInRange(5*time.Second, 120*time.Second), ) // CPUUsageAllowance is exported for testing purposes. @@ -50,26 +50,26 @@ var CPUUsageAllowance = settings.RegisterDurationSetting( "doesn't contribute to consumption; for example, if it is set to 10ms, "+ "that corresponds to 1% of a CPU", 10*time.Millisecond, - checkDurationInRange(0, 1000*time.Millisecond), + settings.DurationInRange(0, 1000*time.Millisecond), ) // ExternalIORUAccountingMode controls whether external ingress and // egress bytes are included in RU calculations. -var ExternalIORUAccountingMode = *settings.RegisterValidatedStringSetting( +var ExternalIORUAccountingMode = *settings.RegisterStringSetting( settings.TenantReadOnly, "tenant_external_io_ru_accounting_mode", "controls how external IO RU accounting behaves; allowed values are 'on' (external IO RUs are accounted for and callers wait for RUs), "+ "'nowait' (external IO RUs are accounted for but callers do not wait for RUs), "+ "and 'off' (no external IO RU accounting)", "on", - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { switch s { case "on", "off", "nowait": return nil default: return errors.Errorf("invalid value %q, expected 'on', 'off', or 'nowait'", s) } - }, + }), ) type externalIORUAccountingMode int64 @@ -100,19 +100,6 @@ func externalIORUAccountingModeFromString(s string) externalIORUAccountingMode { } } -// checkDurationInRange returns a function used to validate duration cluster -// settings. Because these values are currently settable by the tenant, we need -// to restrict the allowed values to avoid possible sabotage of the cost control -// mechanisms. -func checkDurationInRange(min, max time.Duration) func(v time.Duration) error { - return func(v time.Duration) error { - if v < min || v > max { - return errors.Errorf("value %s out of range (%s, %s)", v, min, max) - } - return nil - } -} - // Initial settings for the local token bucket. They are used only until the // first TokenBucket request returns. We allow immediate use of the initial RUs // (we essentially borrow them and pay them back in the first TokenBucket diff --git a/pkg/ccl/oidcccl/settings.go b/pkg/ccl/oidcccl/settings.go index 463bf4b2ebb8..1f606c29188e 100644 --- a/pkg/ccl/oidcccl/settings.go +++ b/pkg/ccl/oidcccl/settings.go @@ -41,40 +41,34 @@ const ( ) // OIDCEnabled enables or disabled OIDC login for the DB Console. -var OIDCEnabled = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.TenantWritable, - OIDCEnabledSettingName, - "enables or disabled OIDC login for the DB Console", - false, - ).WithPublic() - s.SetReportable(true) - return s -}() +var OIDCEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + OIDCEnabledSettingName, + "enables or disabled OIDC login for the DB Console", + false, + settings.WithPublic, + settings.WithReportable(true), +) // OIDCClientID is the OIDC client id. -var OIDCClientID = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - OIDCClientIDSettingName, - "sets OIDC client id", - "", - ).WithPublic() - s.SetReportable(true) - return s -}() +var OIDCClientID = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCClientIDSettingName, + "sets OIDC client id", + "", + settings.WithPublic, + settings.WithReportable(true), +) // OIDCClientSecret is the OIDC client secret. -var OIDCClientSecret = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - OIDCClientSecretSettingName, - "sets OIDC client secret", - "", - ).WithPublic() - s.SetReportable(false) - return s -}() +var OIDCClientSecret = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCClientSecretSettingName, + "sets OIDC client secret", + "", + settings.WithPublic, + settings.WithReportable(false), +) type redirectURLConf struct { mrru *multiRegionRedirectURLs @@ -176,131 +170,110 @@ func validateOIDCRedirectURL(values *settings.Values, s string) error { // use the same callback URL on all auth requests. In a multi-region setting // where the cluster's region is not listed in the `redirect_urls` object, we // will use the required `default_url` callback URL. -var OIDCRedirectURL = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - OIDCRedirectURLSettingName, - "sets OIDC redirect URL via a URL string or a JSON string containing a required "+ - "`redirect_urls` key with an object that maps from region keys to URL strings "+ - "(URLs should point to your load balancer and must route to the path /oidc/v1/callback)", - "https://localhost:8080/oidc/v1/callback", - validateOIDCRedirectURL, - ) - s.SetReportable(true) - s.SetVisibility(settings.Public) - return s -}() +var OIDCRedirectURL = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCRedirectURLSettingName, + "sets OIDC redirect URL via a URL string or a JSON string containing a required "+ + "`redirect_urls` key with an object that maps from region keys to URL strings "+ + "(URLs should point to your load balancer and must route to the path /oidc/v1/callback)", + "https://localhost:8080/oidc/v1/callback", + settings.WithValidateString(validateOIDCRedirectURL), + settings.WithReportable(true), + settings.WithPublic, +) // OIDCProviderURL is the location of the OIDC discovery document for the auth // provider. -var OIDCProviderURL = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - OIDCProviderURLSettingName, - "sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)", - "", - func(values *settings.Values, s string) error { - if _, err := url.Parse(s); err != nil { - return err - } - return nil - }, - ) - s.SetReportable(true) - s.SetVisibility(settings.Public) - return s -}() +var OIDCProviderURL = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCProviderURLSettingName, + "sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)", + "", + settings.WithValidateString(func(values *settings.Values, s string) error { + if _, err := url.Parse(s); err != nil { + return err + } + return nil + }), + settings.WithReportable(true), + settings.WithPublic, +) // OIDCScopes contains the list of scopes to request from the auth provider. -var OIDCScopes = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - OIDCScopesSettingName, - "sets OIDC scopes to include with authentication request "+ - "(space delimited list of strings, required to start with `openid`)", - "openid", - func(values *settings.Values, s string) error { - if s != oidc.ScopeOpenID && !strings.HasPrefix(s, oidc.ScopeOpenID+" ") { - return errors.New("Missing `openid` scope which is required for OIDC") - } - return nil - }, - ) - s.SetReportable(true) - s.SetVisibility(settings.Public) - return s -}() +var OIDCScopes = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCScopesSettingName, + "sets OIDC scopes to include with authentication request "+ + "(space delimited list of strings, required to start with `openid`)", + "openid", + settings.WithValidateString(func(values *settings.Values, s string) error { + if s != oidc.ScopeOpenID && !strings.HasPrefix(s, oidc.ScopeOpenID+" ") { + return errors.New("Missing `openid` scope which is required for OIDC") + } + return nil + }), + settings.WithReportable(true), + settings.WithPublic, +) // OIDCClaimJSONKey is the key of the claim to extract from the OIDC id_token. -var OIDCClaimJSONKey = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - OIDCClaimJSONKeySettingName, - "sets JSON key of principal to extract from payload after OIDC authentication completes "+ - "(usually email or sid)", - "", - ).WithPublic() - return s -}() +var OIDCClaimJSONKey = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCClaimJSONKeySettingName, + "sets JSON key of principal to extract from payload after OIDC authentication completes "+ + "(usually email or sid)", + "", + settings.WithPublic, +) // OIDCPrincipalRegex is a regular expression to apply to the OIDC id_token // claim value to conver it to a DB principal. -var OIDCPrincipalRegex = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - OIDCPrincipalRegexSettingName, - "regular expression to apply to extracted principal (see claim_json_key setting) to "+ - "translate to SQL user (golang regex format, must include 1 grouping to extract)", - "(.+)", - func(values *settings.Values, s string) error { - if _, err := regexp.Compile(s); err != nil { - return errors.Wrapf(err, "unable to initialize %s setting, regex does not compile", - OIDCPrincipalRegexSettingName) - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() +var OIDCPrincipalRegex = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCPrincipalRegexSettingName, + "regular expression to apply to extracted principal (see claim_json_key setting) to "+ + "translate to SQL user (golang regex format, must include 1 grouping to extract)", + "(.+)", + settings.WithValidateString(func(values *settings.Values, s string) error { + if _, err := regexp.Compile(s); err != nil { + return errors.Wrapf(err, "unable to initialize %s setting, regex does not compile", + OIDCPrincipalRegexSettingName) + } + return nil + }), + settings.WithPublic, +) // OIDCButtonText is a string to display on the button in the DB Console to // login with OIDC. -var OIDCButtonText = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - OIDCButtonTextSettingName, - "text to show on button on DB Console login page to login with your OIDC provider "+ - "(only shown if OIDC is enabled)", - "Log in with your OIDC provider", - ).WithPublic() - return s -}() +var OIDCButtonText = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCButtonTextSettingName, + "text to show on button on DB Console login page to login with your OIDC provider "+ + "(only shown if OIDC is enabled)", + "Log in with your OIDC provider", + settings.WithPublic, +) // OIDCAutoLogin is a boolean that enables automatic redirection to OIDC auth in // the DB Console. -var OIDCAutoLogin = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.TenantWritable, - OIDCAutoLoginSettingName, - "if true, logged-out visitors to the DB Console will be "+ - "automatically redirected to the OIDC login endpoint", - false, - ).WithPublic() - return s -}() +var OIDCAutoLogin = settings.RegisterBoolSetting( + settings.TenantWritable, + OIDCAutoLoginSettingName, + "if true, logged-out visitors to the DB Console will be "+ + "automatically redirected to the OIDC login endpoint", + false, + settings.WithPublic, +) // OIDCGenerateClusterSSOTokenEnabled enables or disables generating JWT auth // tokens for cluster SSO with OIDC. -var OIDCGenerateClusterSSOTokenEnabled = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.TenantWritable, - OIDCGenerateClusterSSOTokenEnabledSettingName, - "enables or disables using OIDC to generate JWT auth tokens for cluster SSO", - false, - ) - return s -}() +var OIDCGenerateClusterSSOTokenEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + OIDCGenerateClusterSSOTokenEnabledSettingName, + "enables or disables using OIDC to generate JWT auth tokens for cluster SSO", + false, +) type tokenToUse int64 @@ -311,41 +284,32 @@ const ( // OIDCGenerateClusterSSOTokenUseToken selects which OIDC callback token to use // for cluster SSO. -var OIDCGenerateClusterSSOTokenUseToken = func() *settings.EnumSetting { - s := settings.RegisterEnumSetting( - settings.TenantWritable, - OIDCGenerateClusterSSOTokenUseTokenSettingName, - "selects which OIDC callback token to use for cluster SSO", - "id_token", - map[int64]string{ - int64(useIdToken): "id_token", - int64(useAccessToken): "access_token", - }, - ) - return s -}() +var OIDCGenerateClusterSSOTokenUseToken = settings.RegisterEnumSetting( + settings.TenantWritable, + OIDCGenerateClusterSSOTokenUseTokenSettingName, + "selects which OIDC callback token to use for cluster SSO", + "id_token", + map[int64]string{ + int64(useIdToken): "id_token", + int64(useAccessToken): "access_token", + }, +) // OIDCGenerateClusterSSOTokenSQLHost stores the host name or address to be used // for making SQL connections to the cluster, for display purposes only. -var OIDCGenerateClusterSSOTokenSQLHost = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - OIDCGenerateClusterSSOTokenSQLHostSettingName, - "stores the host name or address to be used for making SQL connections to the cluster, for display purposes only", - "localhost", - ) - return s -}() +var OIDCGenerateClusterSSOTokenSQLHost = settings.RegisterStringSetting( + settings.TenantWritable, + OIDCGenerateClusterSSOTokenSQLHostSettingName, + "stores the host name or address to be used for making SQL connections to the cluster, for display purposes only", + "localhost", +) // OIDCGenerateClusterSSOTokenSQLPort stores the port number to be used for making // SQL connections to the cluster, for display purposes only. -var OIDCGenerateClusterSSOTokenSQLPort = func() *settings.IntSetting { - s := settings.RegisterIntSetting( - settings.TenantWritable, - OIDCGenerateClusterSSOTokenSQLPortSettingName, - "stores the port number to be used for making SQL connections to the cluster, for display purposes only", - 26257, - settings.NonNegativeIntWithMaximum(65535), - ) - return s -}() +var OIDCGenerateClusterSSOTokenSQLPort = settings.RegisterIntSetting( + settings.TenantWritable, + OIDCGenerateClusterSSOTokenSQLPortSettingName, + "stores the port number to be used for making SQL connections to the cluster, for display purposes only", + 26257, + settings.NonNegativeIntWithMaximum(65535), +) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go index 2111a3e1ea36..3d5b3d244bb1 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go @@ -48,12 +48,12 @@ import ( "github.com/cockroachdb/logtags" ) -var minimumFlushInterval = settings.RegisterPublicDurationSettingWithExplicitUnit( +var minimumFlushInterval = settings.RegisterDurationSettingWithExplicitUnit( settings.TenantWritable, "bulkio.stream_ingestion.minimum_flush_interval", "the minimum timestamp between flushes; flushes may still occur if internal buffers fill up", 5*time.Second, - nil, /* validateFn */ + settings.WithPublic, ) var maxKVBufferSize = settings.RegisterByteSizeSetting( diff --git a/pkg/ccl/utilccl/license_check.go b/pkg/ccl/utilccl/license_check.go index 5098abff5b02..7061efbcf912 100644 --- a/pkg/ccl/utilccl/license_check.go +++ b/pkg/ccl/utilccl/license_check.go @@ -31,24 +31,23 @@ import ( "github.com/cockroachdb/redact" ) -var enterpriseLicense = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - "enterprise.license", - "the encoded cluster license", - "", +var enterpriseLicense = settings.RegisterStringSetting( + settings.TenantWritable, + "enterprise.license", + "the encoded cluster license", + "", + settings.WithValidateString( func(sv *settings.Values, s string) error { _, err := decode(s) return err }, - ) + ), // Even though string settings are non-reportable by default, we // still mark them explicitly in case a future code change flips the // default. - s.SetReportable(false) - s.SetVisibility(settings.Public) - return s -}() + settings.WithReportable(false), + settings.WithPublic, +) // enterpriseStatus determines whether the cluster is enabled // for enterprise features or if enterprise status depends on the license. diff --git a/pkg/cloud/azure/azure_storage.go b/pkg/cloud/azure/azure_storage.go index 49feefa66a78..cf41013b6d8c 100644 --- a/pkg/cloud/azure/azure_storage.go +++ b/pkg/cloud/azure/azure_storage.go @@ -45,7 +45,7 @@ var maxConcurrentUploadBuffers = settings.RegisterIntSetting( "controls the number of concurrent buffers that will be used by the Azure client when uploading chunks."+ "Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload", 1, -).WithPublic() + settings.WithPublic) // A note on Azure authentication: // diff --git a/pkg/cloud/cloud_io.go b/pkg/cloud/cloud_io.go index dadc8a6865ac..725940bb2f3f 100644 --- a/pkg/cloud/cloud_io.go +++ b/pkg/cloud/cloud_io.go @@ -39,14 +39,14 @@ var Timeout = settings.RegisterDurationSetting( "cloudstorage.timeout", "the timeout for import/export storage operations", 10*time.Minute, -).WithPublic() + settings.WithPublic) var httpCustomCA = settings.RegisterStringSetting( settings.TenantWritable, "cloudstorage.http.custom_ca", "custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage", "", -).WithPublic() + settings.WithPublic) // WriteChunkSize is used to control the size of each chunk that is buffered and // uploaded by the cloud storage client. diff --git a/pkg/clusterversion/setting.go b/pkg/clusterversion/setting.go index 2ad124ab9947..32fcecd3a477 100644 --- a/pkg/clusterversion/setting.go +++ b/pkg/clusterversion/setting.go @@ -67,9 +67,10 @@ func registerClusterVersionSetting() *clusterVersionSetting { settings.TenantWritable, KeyVersionSetting, "set the active cluster version in the format '.'", // hide optional `-, - &s.VersionSetting) - s.SetVisibility(settings.Public) - s.SetReportable(true) + &s.VersionSetting, + settings.WithPublic, + settings.WithReportable(true), + ) return s } @@ -239,37 +240,32 @@ func (cv *clusterVersionSetting) validateBinaryVersions( return nil } -var PreserveDowngradeVersion = registerPreserveDowngradeVersionSetting() - -func registerPreserveDowngradeVersionSetting() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - "cluster.preserve_downgrade_option", - "disable (automatic or manual) cluster version upgrade from the specified version until reset", - "", - func(sv *settings.Values, s string) error { - if sv == nil || s == "" { - return nil - } - clusterVersion := version.activeVersion(context.TODO(), sv).Version - downgradeVersion, err := roachpb.ParseVersion(s) - if err != nil { - return err - } - - // cluster.preserve_downgrade_option can only be set to the current cluster version. - if downgradeVersion != clusterVersion { - return errors.Errorf( - "cannot set cluster.preserve_downgrade_option to %s (cluster version is %s)", - s, clusterVersion) - } +var PreserveDowngradeVersion = settings.RegisterStringSetting( + settings.TenantWritable, + "cluster.preserve_downgrade_option", + "disable (automatic or manual) cluster version upgrade from the specified version until reset", + "", + settings.WithValidateString(func(sv *settings.Values, s string) error { + if sv == nil || s == "" { return nil - }, - ) - s.SetReportable(true) - s.SetVisibility(settings.Public) - return s -} + } + clusterVersion := version.activeVersion(context.TODO(), sv).Version + downgradeVersion, err := roachpb.ParseVersion(s) + if err != nil { + return err + } + + // cluster.preserve_downgrade_option can only be set to the current cluster version. + if downgradeVersion != clusterVersion { + return errors.Errorf( + "cannot set cluster.preserve_downgrade_option to %s (cluster version is %s)", + s, clusterVersion) + } + return nil + }), + settings.WithReportable(true), + settings.WithPublic, +) var metaPreserveDowngradeLastUpdated = metric.Metadata{ Name: "cluster.preserve-downgrade-option.last-updated", diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index 3a262af2647d..58ae1e6dae43 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -131,7 +131,7 @@ var ( "the amount of time for which records for completed jobs are retained", defaultRetentionTime, settings.PositiveDuration, - ).WithPublic() + settings.WithPublic) cancellationsUpdateLimitSetting = settings.RegisterIntSetting( settings.TenantWritable, diff --git a/pkg/keyvisualizer/keyvissettings/settings.go b/pkg/keyvisualizer/keyvissettings/settings.go index c178ce1383f0..376d7fa46a14 100644 --- a/pkg/keyvisualizer/keyvissettings/settings.go +++ b/pkg/keyvisualizer/keyvissettings/settings.go @@ -11,7 +11,6 @@ package keyvissettings import ( - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/settings" @@ -40,10 +39,5 @@ var MaxBuckets = settings.RegisterIntSetting( "keyvisualizer.max_buckets", "the maximum number of buckets in a sample", 256, - func(i int64) error { - if i < 1 || i > 1024 { - return fmt.Errorf("expected max_buckets to be in range [1, 1024], got %d", i) - } - return nil - }, + settings.IntInRange(1, 1024), ) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 9441c9eb2abb..ba4db0cb6412 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -261,14 +261,8 @@ var rangeDescriptorCacheSize = settings.RegisterIntSetting( "kv.range_descriptor_cache.size", "maximum number of entries in the range descriptor cache", 1e6, - func(v int64) error { - // Set a minimum value to avoid a cache that is too small to be useful. - const minVal = 64 - if v < minVal { - return errors.Errorf("cannot be set to a value less than %d", minVal) - } - return nil - }, + // Set a minimum value to avoid a cache that is too small to be useful. + settings.NonNegativeIntWithMinimum(64), ) // senderConcurrencyLimit controls the maximum number of asynchronous send diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index fc51df340f05..6bc6c66fd504 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -79,7 +79,7 @@ var rangefeedRangeStuckThreshold = settings.RegisterDurationSetting( "restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.closed_timestamp.side_transport_interval takes precedence)", time.Minute, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) func maxConcurrentCatchupScans(sv *settings.Values) int { l := catchupScanConcurrency.Get(sv) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go index 9c3aa5f71de1..52d222423a18 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go @@ -82,7 +82,7 @@ var TrackedWritesMaxSize = settings.RegisterIntSetting( "kv.transaction.max_intents_bytes", "maximum number of bytes used to track locks in transactions", 1<<22, /* 4 MB */ -).WithPublic() + settings.WithPublic) // rejectTxnOverTrackedWritesBudget dictates what happens when a txn exceeds // kv.transaction.max_intents_bytes. @@ -92,7 +92,7 @@ var rejectTxnOverTrackedWritesBudget = settings.RegisterBoolSetting( "if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) "+ "are rejected instead of having their lock spans imprecisely compressed", false, -).WithPublic() + settings.WithPublic) // txnPipeliner is a txnInterceptor that pipelines transactional writes by using // asynchronous consensus. The interceptor then tracks all writes that have been diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index 18f76febc789..a8f948dcfd06 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -41,7 +41,7 @@ var MaxTxnRefreshSpansBytes = settings.RegisterIntSetting( "kv.transaction.max_refresh_spans_bytes", "maximum number of bytes used to track refresh spans in serializable transactions", 1<<22, /* 4 MB */ -).WithPublic() + settings.WithPublic) // txnSpanRefresher is a txnInterceptor that collects the read spans of a // serializable transaction in the event it gets a serializable retry error. It diff --git a/pkg/kv/kvprober/settings.go b/pkg/kv/kvprober/settings.go index 1ddc18d1cced..202368a630f6 100644 --- a/pkg/kv/kvprober/settings.go +++ b/pkg/kv/kvprober/settings.go @@ -14,7 +14,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/errors" ) // kv.prober.bypass_admission_control controls whether kvprober's requests @@ -44,12 +43,8 @@ var readInterval = settings.RegisterDurationSetting( "how often each node sends a read probe to the KV layer on average (jitter is added); "+ "note that a very slow read can block kvprober from sending additional probes; "+ "kv.prober.read.timeout controls the max time kvprober can be blocked", - 1*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 1*time.Second, + settings.PositiveDuration) var readTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -59,12 +54,9 @@ var readTimeout = settings.RegisterDurationSetting( "if this much time elapses without success, a KV read probe will be treated as an error; "+ "note that a very slow read can block kvprober from sending additional probes"+ "this setting controls the max time kvprober can be blocked", - 2*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 2*time.Second, + settings.PositiveDuration, +) var writeEnabled = settings.RegisterBoolSetting( settings.TenantWritable, @@ -78,12 +70,9 @@ var writeInterval = settings.RegisterDurationSetting( "how often each node sends a write probe to the KV layer on average (jitter is added); "+ "note that a very slow read can block kvprober from sending additional probes; "+ "kv.prober.write.timeout controls the max time kvprober can be blocked", - 5*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 5*time.Second, + settings.PositiveDuration, +) var writeTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -93,24 +82,18 @@ var writeTimeout = settings.RegisterDurationSetting( "if this much time elapses without success, a KV write probe will be treated as an error; "+ "note that a very slow read can block kvprober from sending additional probes"+ "this setting controls the max time kvprober can be blocked", - 4*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 4*time.Second, + settings.PositiveDuration, +) var scanMeta2Timeout = settings.RegisterDurationSetting( settings.TenantWritable, "kv.prober.planner.scan_meta2.timeout", "timeout on scanning meta2 via db.Scan with max rows set to "+ "kv.prober.planner.num_steps_to_plan_at_once", - 2*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 2*time.Second, + settings.PositiveDuration, +) var numStepsToPlanAtOnce = settings.RegisterIntSetting( settings.TenantWritable, @@ -120,12 +103,8 @@ var numStepsToPlanAtOnce = settings.RegisterIntSetting( "each planning run, so setting this to a small number will lead to "+ "close-to-lexical probing; already made plans are held in memory, so "+ "large values are advised against", - 100, func(i int64) error { - if i <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 100, + settings.PositiveInt) var quarantinePoolSize = settings.RegisterIntSetting( settings.TenantWritable, @@ -134,12 +113,8 @@ var quarantinePoolSize = settings.RegisterIntSetting( "pool holds Steps for ranges that have been probed and timed out; If "+ "the quarantine pool is full, probes that fail will not be added to "+ " the pool", - 100, func(i int64) error { - if i <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 100, + settings.PositiveInt) var quarantineWriteEnabled = settings.RegisterBoolSetting( settings.TenantWritable, @@ -157,9 +132,5 @@ var quarantineWriteInterval = settings.RegisterDurationSetting( "on average (jitter is added); "+ "note that a very slow read can block kvprober from sending additional probes; "+ "kv.prober.write.timeout controls the max time kvprober can be blocked", - 10*time.Second, func(duration time.Duration) error { - if duration <= 0 { - return errors.New("param must be >0") - } - return nil - }) + 10*time.Second, + settings.PositiveDuration) diff --git a/pkg/kv/kvserver/allocator/BUILD.bazel b/pkg/kv/kvserver/allocator/BUILD.bazel index eaa1816a318a..f9bb73413a0d 100644 --- a/pkg/kv/kvserver/allocator/BUILD.bazel +++ b/pkg/kv/kvserver/allocator/BUILD.bazel @@ -13,7 +13,6 @@ go_library( "//pkg/kv/kvserver/constraint", "//pkg/roachpb", "//pkg/settings", - "@com_github_cockroachdb_errors//:errors", "@io_etcd_go_raft_v3//:raft", ], ) diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go index c2cb5c7095ec..b3c8469c94f5 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator.go @@ -53,7 +53,7 @@ var LeaseRebalanceThreshold = settings.RegisterFloatSetting( "minimum fraction away from the mean a store's lease count can be before "+ "it is considered for lease-transfers", 0.05, -).WithPublic() + settings.WithPublic) // LeaseRebalanceThresholdMin is the absolute number of leases above/below the // mean lease count that a store can have before considered overfull/underfull. @@ -83,7 +83,7 @@ var EnableLoadBasedLeaseRebalancing = settings.RegisterBoolSetting( "kv.allocator.load_based_lease_rebalancing.enabled", "set to enable rebalancing of range leases based on load and latency", true, -).WithPublic() + settings.WithPublic) // leaseRebalancingAggressiveness enables users to tweak how aggressive their // cluster is at moving leases towards the localities where the most requests diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go index 6caadbb9736c..ed2997788cd7 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" ) const ( @@ -124,18 +123,15 @@ const ( // RangeRebalanceThreshold is the minimum ratio of a store's range count to // the mean range count at which that store is considered overfull or underfull // of ranges. -var RangeRebalanceThreshold = func() *settings.FloatSetting { - s := settings.RegisterFloatSetting( - settings.SystemOnly, - "kv.allocator.range_rebalance_threshold", - "minimum fraction away from the mean a store's range count can be before "+ - "it is considered overfull or underfull", - 0.05, - settings.NonNegativeFloat, - ) - s.SetVisibility(settings.Public) - return s -}() +var RangeRebalanceThreshold = settings.RegisterFloatSetting( + settings.SystemOnly, + "kv.allocator.range_rebalance_threshold", + "minimum fraction away from the mean a store's range count can be before "+ + "it is considered overfull or underfull", + 0.05, + settings.NonNegativeFloat, + settings.WithPublic, +) // ReplicaIOOverloadThreshold is the maximum IO overload score of a candidate // store before being excluded as a candidate for rebalancing replicas or @@ -236,18 +232,7 @@ var maxDiskUtilizationThreshold = settings.RegisterFloatSetting( "this should be set higher than "+ "`kv.allocator.rebalance_to_max_disk_utilization_threshold`", defaultMaxDiskUtilizationThreshold, - func(f float64) error { - if f > 0.99 { - return errors.Errorf( - "Cannot set kv.allocator.max_disk_utilization_threshold " + - "greater than 0.99") - } - if f < 0.0 { - return errors.Errorf( - "Cannot set kv.allocator.max_disk_utilization_threshold less than 0") - } - return nil - }, + settings.FloatInRange(0, 0.99), ) // rebalanceToMaxDiskUtilizationThreshold: if the fraction used of a store @@ -264,19 +249,7 @@ var rebalanceToMaxDiskUtilizationThreshold = settings.RegisterFloatSetting( "target; this should be set lower than "+ "`kv.allocator.max_disk_utilization_threshold`", defaultRebalanceToMaxDiskUtilizationThreshold, - func(f float64) error { - if f > 0.99 { - return errors.Errorf( - "Cannot set kv.allocator.rebalance_to_max_disk_utilization_threshold " + - "greater than 0.99") - } - if f < 0.0 { - return errors.Errorf( - "Cannot set kv.allocator.rebalance_to_max_disk_utilization_threshold " + - "less than 0") - } - return nil - }, + settings.FloatInRange(0, 0.99), ) // ScorerOptions defines the interface for the two heuristics that trigger diff --git a/pkg/kv/kvserver/allocator/base.go b/pkg/kv/kvserver/allocator/base.go index 9a47a1e25281..b234bae15d65 100644 --- a/pkg/kv/kvserver/allocator/base.go +++ b/pkg/kv/kvserver/allocator/base.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/errors" "go.etcd.io/raft/v3" ) @@ -105,61 +104,38 @@ type TestingKnobs struct { // rangeRebalanceThreshold because QPS can naturally vary over time as // workloads change and clients come and go, so we need to be a little more // forgiving to avoid thrashing. -var QPSRebalanceThreshold = func() *settings.FloatSetting { - s := settings.RegisterFloatSetting( - settings.SystemOnly, - "kv.allocator.qps_rebalance_threshold", - "minimum fraction away from the mean a store's QPS (such as queries per second) can be before it is considered overfull or underfull", - 0.10, - settings.NonNegativeFloat, - func(f float64) error { - if f < 0.01 { - return errors.Errorf("cannot set kv.allocator.qps_rebalance_threshold to less than 0.01") - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() +var QPSRebalanceThreshold = settings.RegisterFloatSetting( + settings.SystemOnly, + "kv.allocator.qps_rebalance_threshold", + "minimum fraction away from the mean a store's QPS (such as queries per second) can be before it is considered overfull or underfull", + 0.10, + settings.NonNegativeFloatWithMinimum(0.01), + settings.WithPublic, +) // CPURebalanceThreshold is the minimum ratio of a store's cpu time to the mean // cpu time at which that store is considered overfull or underfull of cpu // usage. -var CPURebalanceThreshold = func() *settings.FloatSetting { - s := settings.RegisterFloatSetting( - settings.SystemOnly, - "kv.allocator.store_cpu_rebalance_threshold", - "minimum fraction away from the mean a store's cpu usage can be before it is considered overfull or underfull", - 0.10, - settings.NonNegativeFloat, - func(f float64) error { - if f < 0.01 { - return errors.Errorf("cannot set kv.allocator.store_cpu_rebalance_threshold to less than 0.01") - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() +var CPURebalanceThreshold = settings.RegisterFloatSetting( + settings.SystemOnly, + "kv.allocator.store_cpu_rebalance_threshold", + "minimum fraction away from the mean a store's cpu usage can be before it is considered overfull or underfull", + 0.10, + settings.NonNegativeFloatWithMinimum(0.01), + settings.WithPublic, +) // LoadBasedRebalanceInterval controls how frequently each store checks for // load-base lease/replica rebalancing opportunties. -var LoadBasedRebalanceInterval = settings.RegisterPublicDurationSettingWithExplicitUnit( +var LoadBasedRebalanceInterval = settings.RegisterDurationSettingWithExplicitUnit( settings.SystemOnly, "kv.allocator.load_based_rebalancing_interval", "the rough interval at which each store will check for load-based lease / replica rebalancing opportunities", defaultLoadBasedRebalancingInterval, - func(d time.Duration) error { - // Setting this interval to a very low duration is generally going to be a - // bad idea without any real benefit, so let's disallow that. - const min = 10 * time.Second - if d < min { - return errors.Errorf("must specify a minimum of %s", min) - } - return nil - }, + // Setting this interval to a very low duration is generally going to be a + // bad idea without any real benefit, so let's disallow that. + settings.NonNegativeDurationWithMinimum(10*time.Second), + settings.WithPublic, ) // MinQPSDifferenceForTransfers is the minimum QPS difference that the store @@ -172,18 +148,15 @@ var LoadBasedRebalanceInterval = settings.RegisterPublicDurationSettingWithExpli // be above or below the mean to be considered overfull or underfull // respectively. This is to make lease and replica transfers less sensitive to // the jitters in any given workload. -var MinQPSDifferenceForTransfers = func() *settings.FloatSetting { - s := settings.RegisterFloatSetting( - settings.SystemOnly, - "kv.allocator.min_qps_difference_for_transfers", - "the minimum qps difference that must exist between any two stores"+ - " for the allocator to allow a lease or replica transfer between them", - 2*MinQPSThresholdDifference, - settings.NonNegativeFloat, - ) - s.SetVisibility(settings.Reserved) - return s -}() +var MinQPSDifferenceForTransfers = settings.RegisterFloatSetting( + settings.SystemOnly, + "kv.allocator.min_qps_difference_for_transfers", + "the minimum qps difference that must exist between any two stores"+ + " for the allocator to allow a lease or replica transfer between them", + 2*MinQPSThresholdDifference, + settings.NonNegativeFloat, + settings.WithVisibility(settings.Reserved), +) // transferLeaseGoal dictates whether a call to TransferLeaseTarget should // improve locality of access, convergence of lease counts or convergence of diff --git a/pkg/kv/kvserver/batcheval/cmd_export.go b/pkg/kv/kvserver/batcheval/cmd_export.go index fd4e245a92f9..dfd239a6fe87 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export.go +++ b/pkg/kv/kvserver/batcheval/cmd_export.go @@ -45,7 +45,7 @@ var ExportRequestTargetFileSize = settings.RegisterByteSizeSetting( SSTTargetSizeSetting, MaxExportOverageSetting, ), 16<<20, -).WithPublic() + settings.WithPublic) // MaxExportOverageSetting is the cluster setting name for the // ExportRequestMaxAllowedFileSizeOverage setting. @@ -63,7 +63,7 @@ var ExportRequestMaxAllowedFileSizeOverage = settings.RegisterByteSizeSetting( SSTTargetSizeSetting, MaxExportOverageSetting, ), 64<<20, /* 64 MiB */ -).WithPublic() + settings.WithPublic) func init() { RegisterReadOnlyCommand(kvpb.Export, declareKeysExport, evalExport) diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index d0e1c54c49bc..559a731aba3c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -69,7 +69,7 @@ var MaxLockWaitQueueLength = settings.RegisterIntSetting( "wait-queue is already equal to or exceeding this length, requests will be rejected "+ "eagerly instead of entering the queue and waiting. Set to 0 to disable.", 0, - func(v int64) error { + settings.WithValidateInt(func(v int64) error { if v < 0 { return errors.Errorf("cannot be set to a negative value: %d", v) } @@ -83,7 +83,7 @@ var MaxLockWaitQueueLength = settings.RegisterIntSetting( return errors.Errorf("cannot be set below %d: %d", minSafeMaxLength, v) } return nil - }, + }), ) // DiscoveredLocksThresholdToConsultTxnStatusCache sets a threshold as mentioned diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index bbca9363ef8a..6ccf895ddb23 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -44,7 +44,7 @@ var consistencyCheckRate = settings.RegisterByteSizeSetting( "negatively impact performance.", 8<<20, // 8MB settings.PositiveInt, -).WithPublic() + settings.WithPublic) // consistencyCheckRateBurstFactor we use this to set the burst parameter on the // quotapool.RateLimiter. It seems overkill to provide a user setting for this, diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 803d26451851..095f3010f0c0 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -71,12 +71,7 @@ var IntentAgeThreshold = settings.RegisterDurationSetting( "kv.gc.intent_age_threshold", "intents older than this threshold will be resolved when encountered by the MVCC GC queue", 2*time.Hour, - func(d time.Duration) error { - if d < 2*time.Minute { - return errors.New("intent age threshold must be >= 2 minutes") - } - return nil - }, + settings.NonNegativeDurationWithMinimum(2*time.Minute), ) // TxnCleanupThreshold is the threshold after which a transaction is diff --git a/pkg/kv/kvserver/kvadmission/kvadmission.go b/pkg/kv/kvserver/kvadmission/kvadmission.go index f0a61eea5aa9..84ec04388f25 100644 --- a/pkg/kv/kvserver/kvadmission/kvadmission.go +++ b/pkg/kv/kvserver/kvadmission/kvadmission.go @@ -14,7 +14,6 @@ package kvadmission import ( "context" - "fmt" "sync" "time" @@ -45,17 +44,7 @@ var elasticCPUDurationPerExportRequest = settings.RegisterDurationSetting( "kvadmission.elastic_cpu.duration_per_export_request", "controls how many CPU tokens are allotted for each export request", admission.MaxElasticCPUDuration, - func(duration time.Duration) error { - if duration < admission.MinElasticCPUDuration { - return fmt.Errorf("minimum CPU duration allowed per export request is %s, got %s", - admission.MinElasticCPUDuration, duration) - } - if duration > admission.MaxElasticCPUDuration { - return fmt.Errorf("maximum CPU duration allowed per export request is %s, got %s", - admission.MaxElasticCPUDuration, duration) - } - return nil - }, + settings.DurationInRange(admission.MinElasticCPUDuration, admission.MaxElasticCPUDuration), ) // elasticCPUDurationPerRangefeedScanUnit controls how many CPU tokens are @@ -66,17 +55,7 @@ var elasticCPUDurationPerRangefeedScanUnit = settings.RegisterDurationSetting( "kvadmission.elastic_cpu.duration_per_rangefeed_scan_unit", "controls how many CPU tokens are allotted for each unit of work during rangefeed catchup scans", admission.MaxElasticCPUDuration, - func(duration time.Duration) error { - if duration < admission.MinElasticCPUDuration { - return fmt.Errorf("minimum CPU duration allowed is %s, got %s", - admission.MinElasticCPUDuration, duration) - } - if duration > admission.MaxElasticCPUDuration { - return fmt.Errorf("maximum CPU duration allowed is %s, got %s", - admission.MaxElasticCPUDuration, duration) - } - return nil - }, + settings.DurationInRange(admission.MinElasticCPUDuration, admission.MaxElasticCPUDuration), ) // rangefeedCatchupScanElasticControlEnabled determines whether rangefeed catch @@ -94,7 +73,8 @@ var ProvisionedBandwidth = settings.RegisterByteSizeSetting( settings.SystemOnly, "kvadmission.store.provisioned_bandwidth", "if set to a non-zero value, this is used as the provisioned bandwidth (in bytes/s), "+ "for each store. It can be over-ridden on a per-store basis using the --store flag", - 0).WithPublic() + 0, + settings.WithPublic) // FlowTokenDropInterval determines the frequency at which we check for pending // flow token dispatches to nodes we're no longer connected to, in order to drop diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go index 9ded7387c1e2..75a1ffb82219 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go @@ -404,7 +404,7 @@ const ( maxTokensPerStream kvflowcontrol.Tokens = 64 << 20 // 64 MiB ) -func validateTokenRange(b int64) error { +var validateTokenRange = settings.WithValidateInt(func(b int64) error { t := kvflowcontrol.Tokens(b) if t < minTokensPerStream { return fmt.Errorf("minimum flowed tokens allowed is %s, got %s", minTokensPerStream, t) @@ -413,7 +413,7 @@ func validateTokenRange(b int64) error { return fmt.Errorf("maximum flow tokens allowed is %s, got %s", maxTokensPerStream, t) } return nil -} +}) func (c *Controller) getTokensForStream(stream kvflowcontrol.Stream) tokensPerWorkClass { c.mu.Lock() diff --git a/pkg/kv/kvserver/kvserverbase/BUILD.bazel b/pkg/kv/kvserver/kvserverbase/BUILD.bazel index d3350dab765a..42acd7cb249d 100644 --- a/pkg/kv/kvserver/kvserverbase/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverbase/BUILD.bazel @@ -22,7 +22,6 @@ go_library( "//pkg/settings/cluster", "//pkg/util/errorutil", "//pkg/util/hlc", - "//pkg/util/humanizeutil", "//pkg/util/log", "//pkg/util/quotapool", "//pkg/util/syncutil", diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index b83ddd4a00a0..20855375fdf9 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -21,9 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/quotapool" - "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -284,13 +282,7 @@ var SplitByLoadMergeDelay = settings.RegisterDurationSetting( "kv.range_split.by_load_merge_delay", "the delay that range splits created due to load will wait before considering being merged away", 5*time.Minute, - func(v time.Duration) error { - const minDelay = 5 * time.Second - if v < minDelay { - return errors.Errorf("cannot be set to a value below %s", minDelay) - } - return nil - }, + settings.NonNegativeDurationWithMinimum(5*time.Second), ) const ( @@ -309,10 +301,5 @@ var MaxCommandSize = settings.RegisterByteSizeSetting( "kv.raft.command.max_size", "maximum size of a raft command", MaxCommandSizeDefault, - func(size int64) error { - if size < MaxCommandSizeFloor { - return fmt.Errorf("max_size must be greater than %s", humanizeutil.IBytes(MaxCommandSizeFloor)) - } - return nil - }, + settings.ByteSizeWithMinimum(MaxCommandSizeFloor), ) diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index 24b0d97eabc6..be938a62da19 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -48,25 +48,31 @@ const ( timeAfterNodeSuspectSettingName = "server.time_after_store_suspect" ) +// Setting this to less than the interval for gossiping stores is a big +// no-no, since this value is compared to the age of the most recent gossip +// from each store to determine whether that store is live. Put a buffer of +// 15 seconds on top to allow time for gossip to propagate. +const minTimeUntilNodeDead = gossip.StoresInterval + 15*time.Second + // TimeUntilNodeDead wraps "server.time_until_store_dead". var TimeUntilNodeDead = settings.RegisterDurationSetting( settings.TenantWritable, timeUntilNodeDeadSettingName, "the time after which if there is no new gossiped information about a store, it is considered dead", 5*time.Minute, - func(v time.Duration) error { - // Setting this to less than the interval for gossiping stores is a big - // no-no, since this value is compared to the age of the most recent gossip - // from each store to determine whether that store is live. Put a buffer of - // 15 seconds on top to allow time for gossip to propagate. - const minTimeUntilNodeDead = gossip.StoresInterval + 15*time.Second - if v < minTimeUntilNodeDead { - return errors.Errorf("cannot set %s to less than %v: %v", - timeUntilNodeDeadSettingName, minTimeUntilNodeDead, v) - } - return nil - }, -).WithPublic() + settings.NonNegativeDurationWithMinimum(minTimeUntilNodeDead), + settings.WithPublic, +) + +// Setting this to less than the interval for gossiping stores is a big +// no-no, since this value is compared to the age of the most recent gossip +// from each store to determine whether that store is live. +const minTimeUntilNodeSuspect = gossip.StoresInterval + +// We enforce a maximum value of 5 minutes for this settings, as setting this +// to high may result in a prolonged period of unavailability as a recovered +// store will not be able to acquire leases or replicas for a long time. +const maxTimeAfterNodeSuspect = 5 * time.Minute // TimeAfterNodeSuspect measures how long we consider a store suspect since // it's last failure. @@ -76,27 +82,7 @@ var TimeAfterNodeSuspect = settings.RegisterDurationSetting( "the amount of time we consider a node suspect for after it becomes unavailable."+ " A suspect node is typically treated the same as an unavailable node.", 30*time.Second, - func(v time.Duration) error { - // Setting this to less than the interval for gossiping stores is a big - // no-no, since this value is compared to the age of the most recent gossip - // from each store to determine whether that store is live. - const minTimeUntilNodeSuspect = gossip.StoresInterval - if v < minTimeUntilNodeSuspect { - return errors.Errorf("cannot set %s to less than %v: %v", - timeAfterNodeSuspectSettingName, minTimeUntilNodeSuspect, v) - } - return nil - }, func(v time.Duration) error { - // We enforce a maximum value of 5 minutes for this settings, as setting this - // to high may result in a prolonged period of unavailability as a recovered - // store will not be able to acquire leases or replicas for a long time. - const maxTimeAfterNodeSuspect = 5 * time.Minute - if v > maxTimeAfterNodeSuspect { - return errors.Errorf("cannot set %s to more than %v: %v", - timeAfterNodeSuspectSettingName, maxTimeAfterNodeSuspect, v) - } - return nil - }, + settings.DurationInRange(minTimeUntilNodeSuspect, maxTimeAfterNodeSuspect), ) var ( diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go index fbbe6a9ca680..0a007a26ff38 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go @@ -36,7 +36,7 @@ var ReconcileInterval = settings.RegisterDurationSetting( "the frequency for reconciling jobs with protected timestamp records", 5*time.Minute, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // StatusFunc is used to check on the status of a Record based on its Meta // field. diff --git a/pkg/kv/kvserver/protectedts/settings.go b/pkg/kv/kvserver/protectedts/settings.go index 97eec60ce01f..92780de471a7 100644 --- a/pkg/kv/kvserver/protectedts/settings.go +++ b/pkg/kv/kvserver/protectedts/settings.go @@ -27,6 +27,7 @@ var MaxBytes = settings.RegisterIntSetting( "if non-zero the limit of the number of bytes of spans and metadata which can be protected", 1<<20, // 1 MiB settings.NonNegativeInt, + settings.WithVisibility(settings.Reserved), ) // MaxSpans controls the maximum number of spans which can be protected @@ -37,6 +38,7 @@ var MaxSpans = settings.RegisterIntSetting( "if non-zero the limit of the number of spans which can be protected", 32768, settings.NonNegativeInt, + settings.WithVisibility(settings.Reserved), ) // PollInterval defines how frequently the protectedts state is polled by the @@ -47,8 +49,3 @@ var PollInterval = settings.RegisterDurationSetting( // TODO(ajwerner): better description. "the interval at which the protectedts subsystem state is polled", 2*time.Minute, settings.NonNegativeDuration) - -func init() { - MaxBytes.SetVisibility(settings.Reserved) - MaxSpans.SetVisibility(settings.Reserved) -} diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index e58e6b0ec560..c3ba4a1fe672 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -59,12 +59,9 @@ var queueGuaranteedProcessingTimeBudget = settings.RegisterDurationSetting( "the guaranteed duration before which the processing of a queue may "+ "time out", defaultProcessTimeout, + settings.WithVisibility(settings.Reserved), ) -func init() { - queueGuaranteedProcessingTimeBudget.SetVisibility(settings.Reserved) -} - func defaultProcessTimeoutFunc(cs *cluster.Settings, _ replicaInQueue) time.Duration { return queueGuaranteedProcessingTimeBudget.Get(&cs.SV) } diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index 7c96c7381ec7..639ccbe258c6 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -117,15 +117,13 @@ import ( // to https://github.com/cockroachdb/cockroach/issues/78412 we have changed // the default to false for v22.1. // TODO(sumeer): update the above comment when we have a revised plan. -var looselyCoupledTruncationEnabled = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.SystemOnly, - "kv.raft_log.loosely_coupled_truncation.enabled", - "set to true to loosely couple the raft log truncation", - false) - s.SetVisibility(settings.Reserved) - return s -}() +var looselyCoupledTruncationEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.raft_log.loosely_coupled_truncation.enabled", + "set to true to loosely couple the raft log truncation", + false, + settings.WithVisibility(settings.Reserved), +) const ( // raftLogQueueTimerDuration is the duration between truncations. diff --git a/pkg/kv/kvserver/rebalance_objective.go b/pkg/kv/kvserver/rebalance_objective.go index 9c5b7eb82cda..5b6a11a9f5df 100644 --- a/pkg/kv/kvserver/rebalance_objective.go +++ b/pkg/kv/kvserver/rebalance_objective.go @@ -113,7 +113,7 @@ var LoadBasedRebalancingObjective = settings.RegisterEnumSetting( "`cpu` the cluster will attempt to balance cpu usage among stores", "cpu", LoadBasedRebalancingObjectiveMap, -).WithPublic() + settings.WithPublic) // ToDimension returns the equivalent allocator load dimension of a rebalancing // objective. diff --git a/pkg/kv/kvserver/replica_backpressure.go b/pkg/kv/kvserver/replica_backpressure.go index fe68b3f7209b..f154e3d5f64a 100644 --- a/pkg/kv/kvserver/replica_backpressure.go +++ b/pkg/kv/kvserver/replica_backpressure.go @@ -33,12 +33,7 @@ var backpressureRangeSizeMultiplier = settings.RegisterFloatSetting( "multiple of range_max_bytes that a range is allowed to grow to without "+ "splitting before writes to that range are blocked, or 0 to disable", 2.0, - func(v float64) error { - if v != 0 && v < 1 { - return errors.Errorf("backpressure multiplier cannot be smaller than 1: %f", v) - } - return nil - }, + settings.NonNegativeFloatWithMinimumOrZeroDisable(1), ) // backpressureByteTolerance exists to deal with the fact that lowering the diff --git a/pkg/kv/kvserver/replica_circuit_breaker.go b/pkg/kv/kvserver/replica_circuit_breaker.go index 2950724f3db1..a92884305ab7 100644 --- a/pkg/kv/kvserver/replica_circuit_breaker.go +++ b/pkg/kv/kvserver/replica_circuit_breaker.go @@ -51,24 +51,16 @@ var defaultReplicaCircuitBreakerSlowReplicationThreshold = envutil.EnvOrDefaultD 4*base.SlowRequestThreshold, ) -var replicaCircuitBreakerSlowReplicationThreshold = settings.RegisterPublicDurationSettingWithExplicitUnit( +var replicaCircuitBreakerSlowReplicationThreshold = settings.RegisterDurationSettingWithExplicitUnit( settings.SystemOnly, "kv.replica_circuit_breaker.slow_replication_threshold", "duration after which slow proposals trip the per-Replica circuit breaker (zero duration disables breakers)", defaultReplicaCircuitBreakerSlowReplicationThreshold, - func(d time.Duration) error { - // Setting the breaker duration too low could be very dangerous to cluster - // health (breaking things to the point where the cluster setting can't be - // changed), so enforce a sane minimum. - const min = 500 * time.Millisecond - if d == 0 { - return nil - } - if d <= min { - return errors.Errorf("must specify a minimum of %s", min) - } - return nil - }, + settings.WithPublic, + // Setting the breaker duration too low could be very dangerous to cluster + // health (breaking things to the point where the cluster setting can't be + // changed), so enforce a sane minimum. + settings.NonNegativeDurationWithMinimumOrZeroDisable(500*time.Millisecond), ) // Telemetry counter to count number of trip events. diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index 99432ff77247..0e9db1d7d537 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -31,7 +31,7 @@ var FollowerReadsEnabled = settings.RegisterBoolSetting( "kv.closed_timestamp.follower_reads_enabled", "allow (all) replicas to serve consistent historical reads based on closed timestamp information", true, -).WithPublic() + settings.WithPublic) // BatchCanBeEvaluatedOnFollower determines if a batch consists exclusively of // requests that can be evaluated on a follower replica, given a sufficiently diff --git a/pkg/kv/kvserver/replica_raft_overload.go b/pkg/kv/kvserver/replica_raft_overload.go index a545a4779d76..c9dcc97e71f6 100644 --- a/pkg/kv/kvserver/replica_raft_overload.go +++ b/pkg/kv/kvserver/replica_raft_overload.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "go.etcd.io/raft/v3/tracker" ) @@ -32,16 +31,7 @@ var pauseReplicationIOThreshold = settings.RegisterFloatSetting( "admission.kv.pause_replication_io_threshold", "pause replication to non-essential followers when their I/O admission control score exceeds the given threshold (zero to disable)", 0, - func(v float64) error { - if v == 0 { - return nil - } - const min = 0.3 - if v < min { - return errors.Errorf("minimum admissible nonzero value is %f", min) - } - return nil - }, + settings.NonNegativeFloatWithMinimumOrZeroDisable(0.3), ) type ioThresholdMapI interface { diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 607f874495d9..eb4d8d5e4d80 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -49,7 +49,7 @@ var RangefeedEnabled = settings.RegisterBoolSetting( "kv.rangefeed.enabled", "if set, rangefeed registration is enabled", false, -).WithPublic() + settings.WithPublic) // RangeFeedRefreshInterval controls the frequency with which we deliver closed // timestamp updates to rangefeeds. diff --git a/pkg/kv/kvserver/replica_split_load.go b/pkg/kv/kvserver/replica_split_load.go index a2e03af9c2ba..90ae61ce65dd 100644 --- a/pkg/kv/kvserver/replica_split_load.go +++ b/pkg/kv/kvserver/replica_split_load.go @@ -32,7 +32,7 @@ var SplitByLoadEnabled = settings.RegisterBoolSetting( "kv.range_split.by_load_enabled", "allow automatic splits of ranges based on where load is concentrated", true, -).WithPublic() + settings.WithPublic) // SplitByLoadQPSThreshold wraps "kv.range_split.load_qps_threshold". var SplitByLoadQPSThreshold = settings.RegisterIntSetting( @@ -40,7 +40,7 @@ var SplitByLoadQPSThreshold = settings.RegisterIntSetting( "kv.range_split.load_qps_threshold", "the QPS over which, the range becomes a candidate for load based splitting", 2500, // 2500 req/s -).WithPublic() + settings.WithPublic) // SplitByLoadCPUThreshold wraps "kv.range_split.load_cpu_threshold". The // default threshold of 500ms translates to a replica utilizing 50% of a CPU @@ -58,21 +58,15 @@ var SplitByLoadCPUThreshold = settings.RegisterDurationSetting( "kv.range_split.load_cpu_threshold", "the CPU use per second over which, the range becomes a candidate for load based splitting", 500*time.Millisecond, - func(threshold time.Duration) error { - // We enforce a minimum because of recursive splitting that may occur if - // the threshold is set too low. There is a fixed CPU overhead for a - // replica. At the moment no split key will be produced unless there are - // more than 100 samples (batch requests) to that replica, however the - // memory overhead of tracking split keys in split/weighted_finder.go is - // noticeable and a finder is created after exceeding this threshold. - if threshold < 10*time.Millisecond { - return errors.Errorf( - "Cannot set `kv.range_split.load_cpu_threshold` less than 10ms", - ) - } - return nil - }, -).WithPublic() + // We enforce a minimum because of recursive splitting that may occur if + // the threshold is set too low. There is a fixed CPU overhead for a + // replica. At the moment no split key will be produced unless there are + // more than 100 samples (batch requests) to that replica, however the + // memory overhead of tracking split keys in split/weighted_finder.go is + // noticeable and a finder is created after exceeding this threshold. + settings.NonNegativeDurationWithMinimum(10*time.Millisecond), + settings.WithPublic, +) func (obj LBRebalancingObjective) ToSplitObjective() split.SplitObjective { switch obj { diff --git a/pkg/kv/kvserver/replicastats/replica_stats.go b/pkg/kv/kvserver/replicastats/replica_stats.go index a47e2eb780f9..3c7ff7188f39 100644 --- a/pkg/kv/kvserver/replicastats/replica_stats.go +++ b/pkg/kv/kvserver/replicastats/replica_stats.go @@ -46,7 +46,7 @@ var AddSSTableRequestSizeFactor = settings.RegisterIntSetting( // increase accounted QPS by 1. Typically AddSSTableRequests are ~1mb in size, accounted as 20 // QPS. 50000, -).WithPublic() + settings.WithPublic) // LocalityOracle provides a mapping between a node ID and it's corresponding // locality. diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go index 0b5619f3a1e3..eca8ec593c85 100644 --- a/pkg/kv/kvserver/reports/reporter.go +++ b/pkg/kv/kvserver/reports/reporter.go @@ -53,7 +53,7 @@ var ReporterInterval = settings.RegisterDurationSetting( "replication_critical_localities reports (set to 0 to disable)", time.Minute, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // Reporter periodically produces a couple of reports on the cluster's data // distribution: the system tables: replication_constraint_stats, diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 4fb367179aea..0135f5370818 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -185,7 +185,7 @@ var bulkIOWriteLimit = settings.RegisterByteSizeSetting( "kv.bulk_io_write.max_rate", "the rate limit (bytes/sec) to use for writes to disk on behalf of bulk io ops", 1<<40, // 1 TiB -).WithPublic() + settings.WithPublic) // addSSTableRequestLimit limits concurrent AddSSTable requests. var addSSTableRequestLimit = settings.RegisterIntSetting( @@ -240,25 +240,16 @@ var queueAdditionOnSystemConfigUpdateBurst = settings.RegisterIntSetting( ) // leaseTransferWait is the timeout for a single iteration of draining range leases. -var leaseTransferWait = func() *settings.DurationSetting { - s := settings.RegisterDurationSetting( - settings.SystemOnly, - leaseTransferWaitSettingName, - "the timeout for a single iteration of the range lease transfer phase of draining "+ - "(note that the --drain-wait parameter for cockroach node drain may need adjustment "+ - "after changing this setting)", - 5*time.Second, - func(v time.Duration) error { - if v < 0 { - return errors.Errorf("cannot set %s to a negative duration: %s", - leaseTransferWaitSettingName, v) - } - return nil - }, - ) - s.SetVisibility(settings.Public) - return s -}() +var leaseTransferWait = settings.RegisterDurationSetting( + settings.SystemOnly, + leaseTransferWaitSettingName, + "the timeout for a single iteration of the range lease transfer phase of draining "+ + "(note that the --drain-wait parameter for cockroach node drain may need adjustment "+ + "after changing this setting)", + 5*time.Second, + settings.NonNegativeDuration, + settings.WithPublic, +) const leaseTransferWaitSettingName = "server.shutdown.lease_transfer_wait" @@ -1232,18 +1223,14 @@ type StoreConfig struct { // (e.g., split, merge, add/remove voter/non-voter) into the system.rangelog // table and node join and restart events into system.eventolog table. // Decommissioning events are not controlled by this setting. -var logRangeAndNodeEventsEnabled = func() *settings.BoolSetting { - s := settings.RegisterBoolSetting( - settings.SystemOnly, - "kv.log_range_and_node_events.enabled", - "set to true to transactionally log range events"+ - " (e.g., split, merge, add/remove voter/non-voter) into system.rangelog"+ - "and node join and restart events into system.eventolog", - true, - ) - s.SetVisibility(settings.Public) - return s -}() +var logRangeAndNodeEventsEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.log_range_and_node_events.enabled", + "set to true to transactionally log range events"+ + " (e.g., split, merge, add/remove voter/non-voter) into system.rangelog"+ + "and node join and restart events into system.eventolog", + true, + settings.WithPublic) // ConsistencyTestingKnobs is a BatchEvalTestingKnobs struct used to control the // behavior of the consistency checker for tests. diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index fbbe3f3fd29c..492c6a445bd9 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -80,7 +80,7 @@ var LoadBasedRebalancingMode = settings.RegisterEnumSetting( int64(LBRebalancingLeasesOnly): "leases", int64(LBRebalancingLeasesAndReplicas): "leases and replicas", }, -).WithPublic() + settings.WithPublic) // LBRebalancingMode controls if and when we do store-level rebalancing // based on load. diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 19703c317faf..e93bbe5e6b4b 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -71,12 +71,13 @@ const ( // TODO(baptist): Remove in v24.1, no longer read in v23.2. func init() { - settings.RegisterBoolSetting( + _ = settings.RegisterBoolSetting( settings.SystemOnly, "kv.snapshot_prioritization.enabled", "deprecated no longer used", true, - ).SetRetired() + settings.Retired, + ) } // snapshotMetrics contains metrics on the number and size of snapshots in @@ -1528,23 +1529,19 @@ var rebalanceSnapshotRate = settings.RegisterByteSizeSetting( "kv.snapshot_rebalance.max_rate", "the rate limit (bytes/sec) to use for rebalance and upreplication snapshots", 32<<20, // 32mb/s - func(v int64) error { - if v < minSnapshotRate { - return errors.Errorf("snapshot rate cannot be set to a value below %s: %s", - humanizeutil.IBytes(minSnapshotRate), humanizeutil.IBytes(v)) - } - return nil - }, -).WithPublic() + settings.ByteSizeWithMinimum(minSnapshotRate), + settings.WithPublic, +) // TODO(baptist): Remove in v24.1, no longer read in v23.2. func init() { - settings.RegisterByteSizeSetting( + _ = settings.RegisterByteSizeSetting( settings.SystemOnly, "kv.snapshot_recovery.max_rate", "use kv.snapshot_rebalance.max_rate instead", 32<<20, // 32mb/s - ).SetRetired() + settings.Retired, + ) } // snapshotSenderBatchSize is the size that key-value batches are allowed to @@ -1652,15 +1649,7 @@ var snapshotReservationQueueTimeoutFraction = settings.RegisterFloatSetting( "the fraction of a snapshot's total timeout that it is allowed to spend "+ "queued on the receiver waiting for a reservation", 0.4, - func(v float64) error { - const min, max = 0.25, 1.0 - if v < min { - return errors.Errorf("cannot set to a value less than %f: %f", min, v) - } else if v > max { - return errors.Errorf("cannot set to a value greater than %f: %f", max, v) - } - return nil - }, + settings.FloatInRange(0.25, 1.0), ) // snapshotSSTWriteSyncRate is the size of chunks to write before fsync-ing. diff --git a/pkg/kv/kvserver/tenantrate/settings.go b/pkg/kv/kvserver/tenantrate/settings.go index 6589fff23293..c5c6c257034f 100644 --- a/pkg/kv/kvserver/tenantrate/settings.go +++ b/pkg/kv/kvserver/tenantrate/settings.go @@ -14,7 +14,6 @@ import ( "runtime" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/errors" ) // Config contains the configuration of the rate limiter. @@ -79,12 +78,7 @@ var ( "per-tenant rate limit in KV Compute Units per second if positive, "+ "or KV Compute Units per second per CPU if negative", -200, - func(v float64) error { - if v == 0 { - return errors.New("cannot set to zero value") - } - return nil - }, + settings.NonZeroFloat, ) kvcuBurstLimitSeconds = settings.RegisterFloatSetting( diff --git a/pkg/multitenant/tenant_config.go b/pkg/multitenant/tenant_config.go index ac30ae5c34c3..0db95937003c 100644 --- a/pkg/multitenant/tenant_config.go +++ b/pkg/multitenant/tenant_config.go @@ -27,7 +27,7 @@ var DefaultTenantSelect = settings.RegisterStringSetting( DefaultTenantSelectSettingName, "name of the tenant to use to serve requests when clients don't specify a tenant", catconstants.SystemTenantName, -).WithPublic() + settings.WithPublic) // VerifyTenantService determines whether there should be an advisory // interlock between changes to the tenant service and changes to the diff --git a/pkg/multitenant/tenantcostmodel/settings.go b/pkg/multitenant/tenantcostmodel/settings.go index 266c249cc0f0..790e20d7968d 100644 --- a/pkg/multitenant/tenantcostmodel/settings.go +++ b/pkg/multitenant/tenantcostmodel/settings.go @@ -109,17 +109,14 @@ var ( settings.NonNegativeFloat, ) - CrossRegionNetworkCostSetting = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantReadOnly, - "tenant_cost_model.cross_region_network_cost", - "network cost table for cross-region traffic", - "", - validateRegionalCostMultiplierTableSetting, - ) - s.SetReportable(true) - return s - }() + CrossRegionNetworkCostSetting = settings.RegisterStringSetting( + settings.TenantReadOnly, + "tenant_cost_model.cross_region_network_cost", + "network cost table for cross-region traffic", + "", + settings.WithValidateString(validateRegionalCostMultiplierTableSetting), + settings.WithReportable(true), + ) // List of config settings, used by SetOnChange. configSettings = [...]settings.NonMaskedSetting{ diff --git a/pkg/roachpb/span_stats.go b/pkg/roachpb/span_stats.go index 18e5586721e2..cfe0854f8b1b 100644 --- a/pkg/roachpb/span_stats.go +++ b/pkg/roachpb/span_stats.go @@ -11,7 +11,6 @@ package roachpb import ( - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/settings" @@ -59,12 +58,7 @@ var RangeDescPageSize = settings.RegisterIntSetting( "server.span_stats.range_desc_page_size", "the page size when iterating through range descriptors", 100, - func(i int64) error { - if i < 5 || i > 25000 { - return fmt.Errorf("expected range_desc_page_size to be in range [5, 25000], got %d", i) - } - return nil - }, + settings.IntInRange(5, 25000), ) func (m *SpanStats) Add(other *SpanStats) { diff --git a/pkg/security/cert_expiry_cache.go b/pkg/security/cert_expiry_cache.go index 9203d29d553e..655bb9cdb7de 100644 --- a/pkg/security/cert_expiry_cache.go +++ b/pkg/security/cert_expiry_cache.go @@ -37,7 +37,7 @@ var ClientCertExpirationCacheCapacity = settings.RegisterIntSetting( "server.client_cert_expiration_cache.capacity", "the maximum number of client cert expirations stored", 1000, -).WithPublic() + settings.WithPublic) // ClientCertExpirationCache contains a cache of gauge objects keyed by // SQL username strings. It is a FIFO cache that stores gauges valued by diff --git a/pkg/security/password.go b/pkg/security/password.go index 888a0dddbc77..956c4b5afdb1 100644 --- a/pkg/security/password.go +++ b/pkg/security/password.go @@ -45,12 +45,12 @@ var BcryptCost = settings.RegisterIntSetting( // It incurs a password check latency of ~60ms on AMD 3950X 3.7GHz. // For reference, value 11 incurs ~110ms latency on the same hw, value 12 incurs ~390ms. password.DefaultBcryptCost, - func(i int64) error { + settings.WithValidateInt(func(i int64) error { if i < int64(bcrypt.MinCost) || i > int64(bcrypt.MaxCost) { return bcrypt.InvalidCostError(int(i)) } return nil - }).WithPublic() + }), settings.WithPublic) // BcryptCostSettingName is the name of the cluster setting BcryptCost. const BcryptCostSettingName = "server.user_login.password_hashes.default_cost.crdb_bcrypt" @@ -73,12 +73,8 @@ var SCRAMCost = settings.RegisterIntSetting( // For further discussion, see the explanation on bcryptCostToSCRAMIterCount // below. password.DefaultSCRAMCost, - func(i int64) error { - if i < password.ScramMinCost || i > password.ScramMaxCost { - return errors.Newf("cost not in allowed range (%d,%d)", password.ScramMinCost, password.ScramMaxCost) - } - return nil - }).WithPublic() + settings.IntInRange(password.ScramMinCost, password.ScramMaxCost), + settings.WithPublic) // SCRAMCostSettingName is the name of the cluster setting SCRAMCost. const SCRAMCostSettingName = "server.user_login.password_hashes.default_cost.scram_sha_256" @@ -112,7 +108,7 @@ var PasswordHashMethod = settings.RegisterEnumSetting( int64(password.HashBCrypt): password.HashBCrypt.String(), int64(password.HashSCRAMSHA256): password.HashSCRAMSHA256.String(), }, -).WithPublic() + settings.WithPublic) // GetConfiguredPasswordCost returns the configured hashing cost // for the given method. @@ -155,7 +151,7 @@ var MinPasswordLength = settings.RegisterIntSetting( "Note that a value lower than 1 is ignored: passwords cannot be empty in any case.", 1, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // AutoUpgradePasswordHashes is the cluster setting that configures whether to // automatically re-encode stored passwords using crdb-bcrypt to scram-sha-256. @@ -165,7 +161,7 @@ var AutoUpgradePasswordHashes = settings.RegisterBoolSetting( "if server.user_login.password_encryption=scram-sha-256, this controls "+ "whether to automatically re-encode stored passwords using crdb-bcrypt to scram-sha-256", true, -).WithPublic() + settings.WithPublic) // AutoDowngradePasswordHashes is the cluster setting that configures whether to // automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt. @@ -175,7 +171,7 @@ var AutoDowngradePasswordHashes = settings.RegisterBoolSetting( "if server.user_login.password_encryption=crdb-bcrypt, this controls "+ "whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt", true, -).WithPublic() + settings.WithPublic) // AutoRehashOnSCRAMCostChange is the cluster setting that configures whether to // automatically re-encode stored passwords using scram-sha-256 to use a new @@ -187,7 +183,7 @@ var AutoRehashOnSCRAMCostChange = settings.RegisterBoolSetting( "the cost in a stored hash, this controls whether to automatically re-encode "+ "stored passwords using scram-sha-256 with the new default cost", true, -).WithPublic() + settings.WithPublic) // expensiveHashComputeSemOnce wraps a semaphore that limits the // number of concurrent calls to the bcrypt and sha256 hash diff --git a/pkg/security/tls_settings.go b/pkg/security/tls_settings.go index ca5e011d13ae..0015952d0e06 100644 --- a/pkg/security/tls_settings.go +++ b/pkg/security/tls_settings.go @@ -45,14 +45,15 @@ var ocspMode = settings.RegisterEnumSetting( "use OCSP to check whether TLS certificates are revoked. If the OCSP "+ "server is unreachable, in strict mode all certificates will be rejected "+ "and in lax mode all certificates will be accepted.", - "off", map[int64]string{ocspOff: "off", ocspLax: "lax", ocspStrict: "strict"}).WithPublic() + "off", map[int64]string{ocspOff: "off", ocspLax: "lax", ocspStrict: "strict"}, + settings.WithPublic) var ocspTimeout = settings.RegisterDurationSetting( settings.TenantWritable, "security.ocsp.timeout", "timeout before considering the OCSP server unreachable", 3*time.Second, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) type clusterTLSSettings struct { settings *cluster.Settings diff --git a/pkg/server/authserver/authentication.go b/pkg/server/authserver/authentication.go index 6a7f357eebcb..f1ef750358aa 100644 --- a/pkg/server/authserver/authentication.go +++ b/pkg/server/authserver/authentication.go @@ -100,7 +100,7 @@ var WebSessionTimeout = settings.RegisterDurationSetting( "the duration that a newly created web session will be valid", 7*24*time.Hour, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) type authenticationServer struct { cfg *base.Config diff --git a/pkg/server/clock_monotonicity.go b/pkg/server/clock_monotonicity.go index 82ca03f1c1cb..d1a72a8c1e0a 100644 --- a/pkg/server/clock_monotonicity.go +++ b/pkg/server/clock_monotonicity.go @@ -28,7 +28,7 @@ var ( "server.clock.forward_jump_check_enabled", "if enabled, forward clock jumps > max_offset/2 will cause a panic", false, - ).WithPublic() + settings.WithPublic) persistHLCUpperBoundInterval = settings.RegisterDurationSetting( settings.TenantWritable, @@ -40,7 +40,7 @@ var ( "time across server restarts. Not setting this or setting a value of 0 disables this "+ "feature.", 0, - ).WithPublic() + settings.WithPublic) ) // startMonitoringForwardClockJumps starts a background task to monitor forward diff --git a/pkg/server/cluster_settings.go b/pkg/server/cluster_settings.go index bd1902754fdc..d097992cdb6f 100644 --- a/pkg/server/cluster_settings.go +++ b/pkg/server/cluster_settings.go @@ -20,7 +20,7 @@ var SQLStatsResponseMax = settings.RegisterIntSetting( "the maximum number of statements and transaction stats returned in a CombinedStatements request", 20000, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // SQLStatsShowInternal controls if statistics for internal executions should be returned in sql stats APIs, // including: CombinedStatementStats, ListSessions, and ListLocalSessions. @@ -31,7 +31,7 @@ var SQLStatsShowInternal = settings.RegisterBoolSetting( "internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display "+ "statistics on the SQL Activity pages", false, -).WithPublic() + settings.WithPublic) // StatsActivityUIEnabled controls if the combined statement stats uses // the system.statement_activity and system.transaction_activity which diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index f4caee78e89c..ab247858787c 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -54,14 +54,11 @@ func init() { // Endpoint is the entry point under which the debug tools are housed. const Endpoint = "/debug/" -var _ = func() *settings.StringSetting { - // This setting definition still exists so as to not break - // deployment scripts that set it unconditionally. - v := settings.RegisterStringSetting( - settings.TenantWritable, "server.remote_debugging.mode", "unused", "local") - v.SetRetired() - return v -}() +// This setting definition still exists so as to not break +// deployment scripts that set it unconditionally. +var _ = settings.RegisterStringSetting( + settings.TenantWritable, "server.remote_debugging.mode", "unused", "local", + settings.Retired) // Server serves the /debug/* family of tools. type Server struct { diff --git a/pkg/server/diagnostics/reporter.go b/pkg/server/diagnostics/reporter.go index 4fa61cd538b4..efadf2641515 100644 --- a/pkg/server/diagnostics/reporter.go +++ b/pkg/server/diagnostics/reporter.go @@ -63,7 +63,7 @@ var reportFrequency = settings.RegisterDurationSetting( "interval at which diagnostics data should be reported", time.Hour, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // Reporter is a helper struct that phones home to report usage and diagnostics. type Reporter struct { diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 61c898be4e68..8e1623067333 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -45,7 +45,7 @@ var ( "after changing this setting)", 10*time.Second, settings.NonNegativeDurationWithMaximum(10*time.Hour), - ).WithPublic() + settings.WithPublic) drainWait = settings.RegisterDurationSetting( settings.TenantWritable, @@ -57,7 +57,7 @@ var ( "wait time for health probes to notice that the node is not ready.)", 0*time.Second, settings.NonNegativeDurationWithMaximum(10*time.Hour), - ).WithPublic() + settings.WithPublic) connectionWait = settings.RegisterDurationSetting( settings.TenantWritable, @@ -68,7 +68,7 @@ var ( "after changing this setting)", 0*time.Second, settings.NonNegativeDurationWithMaximum(10*time.Hour), - ).WithPublic() + settings.WithPublic) jobRegistryWait = settings.RegisterDurationSetting( settings.TenantWritable, @@ -77,7 +77,7 @@ var ( "to notice drain request and to perform orderly shutdown", 10*time.Second, settings.NonNegativeDurationWithMaximum(10*time.Minute), - ).WithPublic() + settings.WithPublic) ) // Drain puts the node into the specified drain mode(s) and optionally diff --git a/pkg/server/hot_ranges.go b/pkg/server/hot_ranges.go index 90889cb197e0..cd44f7380baa 100644 --- a/pkg/server/hot_ranges.go +++ b/pkg/server/hot_ranges.go @@ -27,4 +27,4 @@ var HotRangesRequestNodeTimeout = settings.RegisterDurationSetting( "the duration allowed for a single node to return hot range data before the request is cancelled; if set to 0, there is no timeout", time.Minute*5, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) diff --git a/pkg/server/node.go b/pkg/server/node.go index bae8ee804698..9425f1be8582 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -229,7 +229,7 @@ var ( "external.graphite.endpoint", "if nonempty, push server metrics to the Graphite or Carbon server at the specified host:port", "", - ).WithPublic() + settings.WithPublic) // graphiteInterval is how often metrics are pushed to Graphite, if enabled. graphiteInterval = settings.RegisterDurationSetting( settings.TenantWritable, @@ -237,13 +237,13 @@ var ( "the interval at which metrics are pushed to Graphite (if enabled)", 10*time.Second, settings.NonNegativeDurationWithMaximum(maxGraphiteInterval), - ).WithPublic() + settings.WithPublic) RedactServerTracesForSecondaryTenants = settings.RegisterBoolSetting( settings.SystemOnly, "server.secondary_tenants.redact_trace.enabled", "controls if server side traces are redacted for tenant operations", true, - ).WithPublic() + settings.WithPublic) slowRequestHistoricalStackThreshold = settings.RegisterDurationSetting( settings.SystemOnly, diff --git a/pkg/server/profiler/cluster_settings.go b/pkg/server/profiler/cluster_settings.go index 6c28c94696d9..83bcc08bff32 100644 --- a/pkg/server/profiler/cluster_settings.go +++ b/pkg/server/profiler/cluster_settings.go @@ -25,4 +25,4 @@ var ActiveQueryDumpsEnabled = settings.RegisterBoolSetting( "diagnostics.active_query_dumps.enabled", "experimental: enable dumping of anonymized active queries to disk when node is under memory pressure", true, -).WithPublic() + settings.WithPublic) diff --git a/pkg/server/profiler/profilestore.go b/pkg/server/profiler/profilestore.go index 6cf3e60da1ae..97a641fe423d 100644 --- a/pkg/server/profiler/profilestore.go +++ b/pkg/server/profiler/profilestore.go @@ -44,18 +44,18 @@ var ( ) func init() { - s := settings.RegisterIntSetting( + _ = settings.RegisterIntSetting( settings.TenantWritable, - "server.heap_profile.max_profiles", "use server.mem_profile.max_profiles instead", 5) - s.SetRetired() + "server.heap_profile.max_profiles", "use server.mem_profile.max_profiles instead", 5, + settings.Retired) - b := settings.RegisterByteSizeSetting( + _ = settings.RegisterByteSizeSetting( settings.TenantWritable, "server.heap_profile.total_dump_size_limit", "use server.mem_profile.total_dump_size_limit instead", 128<<20, // 128MiB + settings.Retired, ) - b.SetRetired() } // profileStore represents the directory where heap profiles are stored. diff --git a/pkg/server/server_http.go b/pkg/server/server_http.go index e286f4960c3f..6dcdf81d6c59 100644 --- a/pkg/server/server_http.go +++ b/pkg/server/server_http.go @@ -75,7 +75,7 @@ var HSTSEnabled = settings.RegisterBoolSetting( "access the DB Console. Ensure that TLS is correctly configured "+ "prior to enabling.", false, -).WithPublic() + settings.WithPublic) const hstsHeaderKey = "Strict-Transport-Security" diff --git a/pkg/server/server_systemlog_gc.go b/pkg/server/server_systemlog_gc.go index 8769dae28c94..d7952392eae9 100644 --- a/pkg/server/server_systemlog_gc.go +++ b/pkg/server/server_systemlog_gc.go @@ -34,14 +34,14 @@ var ( "the period at which log-like system tables are checked for old entries", time.Hour, settings.NonNegativeDuration, - ).WithPublic() + settings.WithPublic) systemLogGCLimit = settings.RegisterIntSetting( settings.TenantWritable, "server.log_gc.max_deletions_per_cycle", "the maximum number of entries to delete on each purge of log-like system tables", 1000, - ).WithPublic() + settings.WithPublic) // rangeLogTTL is the TTL for rows in system.rangelog. If non zero, range log // entries are periodically garbage collected. @@ -50,7 +50,7 @@ var ( "server.rangelog.ttl", "if nonzero, entries in system.rangelog older than this duration are periodically purged", 30*24*time.Hour, // 30 days - ).WithPublic() + settings.WithPublic) // eventLogTTL is the TTL for rows in system.eventlog. If non zero, event log // entries are periodically garbage collected. @@ -59,14 +59,14 @@ var ( "server.eventlog.ttl", "if nonzero, entries in system.eventlog older than this duration are periodically purged", 90*24*time.Hour, // 90 days - ).WithPublic() + settings.WithPublic) webSessionPurgeTTL = settings.RegisterDurationSetting( settings.TenantWritable, "server.web_session.purge.ttl", "if nonzero, entries in system.web_sessions older than this duration are periodically purged", time.Hour, - ).WithPublic() + settings.WithPublic) ) // gcSystemLog deletes entries in the given system log table between diff --git a/pkg/server/settings_test.go b/pkg/server/settings_test.go index 95e3ed18bb86..86429e8dcd2e 100644 --- a/pkg/server/settings_test.go +++ b/pkg/server/settings_test.go @@ -33,28 +33,30 @@ const durationKey = "testing.duration" const byteSizeKey = "testing.bytesize" const enumKey = "testing.enum" -var strA = settings.RegisterValidatedStringSetting( - settings.TenantWritable, strKey, "desc", "", func(sv *settings.Values, v string) error { +var strA = settings.RegisterStringSetting( + settings.TenantWritable, strKey, "desc", "", + settings.WithValidateString(func(sv *settings.Values, v string) error { if len(v) > 15 { return errors.Errorf("can't set %s to string longer than 15: %s", strKey, v) } return nil - }) + })) var intA = settings.RegisterIntSetting( - settings.TenantWritable, intKey, "desc", 1, func(v int64) error { + settings.TenantWritable, intKey, "desc", 1, + settings.WithValidateInt(func(v int64) error { if v < 0 { return errors.Errorf("can't set %s to a negative value: %d", intKey, v) } return nil - - }) + })) var durationA = settings.RegisterDurationSetting( - settings.TenantWritable, durationKey, "desc", time.Minute, func(v time.Duration) error { + settings.TenantWritable, durationKey, "desc", time.Minute, + settings.WithValidateDuration(func(v time.Duration) error { if v < 0 { return errors.Errorf("can't set %s to a negative duration: %s", durationKey, v) } return nil - }) + })) var byteSizeA = settings.RegisterByteSizeSetting( settings.TenantWritable, byteSizeKey, "desc", 1024*1024, ) diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index 2cda3c054a98..b120eeabf374 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -90,7 +90,8 @@ type storeMetrics interface { var ChildMetricsEnabled = settings.RegisterBoolSetting( settings.TenantWritable, "server.child_metrics.enabled", "enables the exporting of child metrics, additional prometheus time series with extra labels", - false).WithPublic() + false, + settings.WithPublic) // MetricsRecorder is used to periodically record the information in a number of // metric registries. diff --git a/pkg/settings/BUILD.bazel b/pkg/settings/BUILD.bazel index 24e85be267e2..15596de2d6cb 100644 --- a/pkg/settings/BUILD.bazel +++ b/pkg/settings/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "float.go", "int.go", "masked.go", + "options.go", "protobuf.go", "registry.go", "setting.go", @@ -44,6 +45,7 @@ go_test( name = "settings_test", size = "small", srcs = [ + "duration_test.go", "encoding_test.go", "internal_test.go", "settings_test.go", diff --git a/pkg/settings/bool.go b/pkg/settings/bool.go index 4b3973564135..242e347b332f 100644 --- a/pkg/settings/bool.go +++ b/pkg/settings/bool.go @@ -97,17 +97,12 @@ func (b *BoolSetting) setToDefault(ctx context.Context, sv *Values) { b.set(ctx, sv, b.defaultValue) } -// WithPublic sets public visibility and can be chained. -func (b *BoolSetting) WithPublic() *BoolSetting { - b.SetVisibility(Public) - return b -} - // RegisterBoolSetting defines a new setting with type bool. func RegisterBoolSetting( - class Class, key InternalKey, desc string, defaultValue bool, + class Class, key InternalKey, desc string, defaultValue bool, opts ...SettingOption, ) *BoolSetting { setting := &BoolSetting{defaultValue: defaultValue} register(class, key, desc, setting) + setting.apply(opts) return setting } diff --git a/pkg/settings/byte_size.go b/pkg/settings/byte_size.go index cabb655b76cc..05db7e7bed43 100644 --- a/pkg/settings/byte_size.go +++ b/pkg/settings/byte_size.go @@ -42,29 +42,33 @@ func (b *ByteSizeSetting) DecodeToString(encoded string) (string, error) { return string(humanizeutil.IBytes(iv)), nil } -// WithPublic sets public visibility and can be chained. -func (b *ByteSizeSetting) WithPublic() *ByteSizeSetting { - b.SetVisibility(Public) - return b -} - // RegisterByteSizeSetting defines a new setting with type bytesize and any // supplied validation function(s). If no validation functions are given, then // the non-negative int validation is performed. func RegisterByteSizeSetting( - class Class, key InternalKey, desc string, defaultValue int64, validateFns ...func(int64) error, + class Class, key InternalKey, desc string, defaultValue int64, opts ...SettingOption, ) *ByteSizeSetting { - - var validateFn = func(v int64) error { - if len(validateFns) > 0 { - for _, fn := range validateFns { + validateFn := func(v int64) error { + hasExplicitValidationFn := false + for _, opt := range opts { + if opt.validateProtoFn != nil || + opt.validateFloat64Fn != nil || + opt.validateStringFn != nil || + opt.validateDurationFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateInt64Fn; fn != nil { + hasExplicitValidationFn = true if err := fn(v); err != nil { return errors.Wrapf(err, "invalid value for %s", key) } } - return nil } - return NonNegativeInt(v) + if !hasExplicitValidationFn { + // Default validation. + return nonNegativeIntInternal(v) + } + return nil } if err := validateFn(defaultValue); err != nil { @@ -75,5 +79,17 @@ func RegisterByteSizeSetting( validateFn: validateFn, }} register(class, key, desc, setting) + setting.apply(opts) return setting } + +// ByteSizeWithMinimum can be passed to RegisterByteSizeSetting. +func ByteSizeWithMinimum(minVal int64) SettingOption { + return WithValidateInt(func(v int64) error { + if v < minVal { + return errors.Errorf("cannot be set to a value lower than %v", + humanizeutil.IBytes(minVal)) + } + return nil + }) +} diff --git a/pkg/settings/common.go b/pkg/settings/common.go index 4423109d643f..12b464f793b5 100644 --- a/pkg/settings/common.go +++ b/pkg/settings/common.go @@ -87,28 +87,21 @@ func (c *common) ValueOrigin(ctx context.Context, sv *Values) ValueOrigin { return sv.getValueOrigin(ctx, c.slot) } -// SetReportable indicates whether a setting's value can show up in SHOW ALL -// CLUSTER SETTINGS and telemetry reports. -// -// The setting can still be used with SET and SHOW if the exact -// setting name is known. Use SetReportable(false) for data that must -// be hidden from standard setting report, telemetry and -// troubleshooting screenshots, such as license data or keys. -// -// All string settings are also non-reportable by default and must be -// opted in to reports manually with SetReportable(true). -func (c *common) SetReportable(reportable bool) { +// setReportable configures the reportability. +// Refer to the WithReportable option for details. +func (c *common) setReportable(reportable bool) { c.nonReportable = !reportable } -// SetVisibility customizes the visibility of a setting. -func (c *common) SetVisibility(v Visibility) { +// setVisibility customizes the visibility of a setting. +// Refer to the WithVisibility option for details. +func (c *common) setVisibility(v Visibility) { c.visibility = v } -// SetRetired marks the setting as obsolete. It also hides +// setRetired marks the setting as obsolete. It also hides // it from the output of SHOW CLUSTER SETTINGS. -func (c *common) SetRetired() { +func (c *common) setRetired() { c.description = "do not use - " + c.description c.retired = true } diff --git a/pkg/settings/duration.go b/pkg/settings/duration.go index ca36ce228757..b0e271084c0c 100644 --- a/pkg/settings/duration.go +++ b/pkg/settings/duration.go @@ -127,101 +127,107 @@ func (d *DurationSetting) setToDefault(ctx context.Context, sv *Values) { } } -// WithPublic sets the visibility to public and can be chained. -func (d *DurationSetting) WithPublic() *DurationSetting { - d.SetVisibility(Public) - return d -} - // RegisterDurationSetting defines a new setting with type duration. func RegisterDurationSetting( - class Class, - key InternalKey, - desc string, - defaultValue time.Duration, - validateFns ...func(time.Duration) error, + class Class, key InternalKey, desc string, defaultValue time.Duration, opts ...SettingOption, ) *DurationSetting { - var validateFn func(time.Duration) error - if len(validateFns) > 0 { - validateFn = func(v time.Duration) error { - for _, fn := range validateFns { - if err := fn(v); err != nil { + validateFn := func(val time.Duration) error { + for _, opt := range opts { + if opt.validateInt64Fn != nil || + opt.validateFloat64Fn != nil || + opt.validateStringFn != nil || + opt.validateProtoFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateDurationFn; fn != nil { + if err := fn(val); err != nil { return errors.Wrapf(err, "invalid value for %s", key) } } - return nil } + return nil } - if validateFn != nil { - if err := validateFn(defaultValue); err != nil { - panic(errors.Wrap(err, "invalid default")) - } + if err := validateFn(defaultValue); err != nil { + panic(errors.Wrap(err, "invalid default")) } setting := &DurationSetting{ defaultValue: defaultValue, validateFn: validateFn, } register(class, key, desc, setting) + setting.apply(opts) return setting } // RegisterPublicDurationSettingWithExplicitUnit defines a new // public setting with type duration which requires an explicit unit when being // set. -func RegisterPublicDurationSettingWithExplicitUnit( - class Class, - key InternalKey, - desc string, - defaultValue time.Duration, - validateFn func(time.Duration) error, +func RegisterDurationSettingWithExplicitUnit( + class Class, key InternalKey, desc string, defaultValue time.Duration, opts ...SettingOption, ) *DurationSettingWithExplicitUnit { - var fn func(time.Duration) error - - if validateFn != nil { - fn = func(v time.Duration) error { - return errors.Wrapf(validateFn(v), "invalid value for %s", key) + validateFn := func(val time.Duration) error { + for _, opt := range opts { + if opt.validateInt64Fn != nil || + opt.validateFloat64Fn != nil || + opt.validateStringFn != nil || + opt.validateProtoFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateDurationFn; fn != nil { + if err := fn(val); err != nil { + return errors.Wrapf(err, "invalid value for %s", key) + } + } } + return nil + } + if err := validateFn(defaultValue); err != nil { + panic(errors.Wrap(err, "invalid default")) } - setting := &DurationSettingWithExplicitUnit{ DurationSetting{ defaultValue: defaultValue, - validateFn: fn, + validateFn: validateFn, }, } - setting.SetVisibility(Public) register(class, key, desc, setting) + setting.apply(opts) return setting } // NonNegativeDuration can be passed to RegisterDurationSetting. -func NonNegativeDuration(v time.Duration) error { +var NonNegativeDuration SettingOption = WithValidateDuration(nonNegativeDurationInternal) + +func nonNegativeDurationInternal(v time.Duration) error { if v < 0 { return errors.Errorf("cannot be set to a negative duration: %s", v) } return nil } -// NonNegativeDurationWithMaximum returns a validation function that can be -// passed to RegisterDurationSetting. -func NonNegativeDurationWithMaximum(maxValue time.Duration) func(time.Duration) error { - return func(v time.Duration) error { - if v < 0 { - return errors.Errorf("cannot be set to a negative duration: %s", v) - } - if v > maxValue { - return errors.Errorf("cannot be set to a value larger than %s", maxValue) +// DurationInRange returns a validation function that checks the value +// is within the given bounds (inclusive). +func DurationInRange(minVal, maxVal time.Duration) SettingOption { + return WithValidateDuration(func(v time.Duration) error { + if v < minVal || v > maxVal { + return errors.Errorf("expected value in range [%v, %v], got: %v", minVal, maxVal, v) } return nil - } + }) +} + +// NonNegativeDurationWithMaximum returns a validation function that can be +// passed to RegisterDurationSetting. +func NonNegativeDurationWithMaximum(maxValue time.Duration) SettingOption { + return DurationInRange(0, maxValue) } // NonNegativeDurationWithMinimum returns a validation function can be passed // to RegisterDurationSetting. -func NonNegativeDurationWithMinimum(minValue time.Duration) func(time.Duration) error { - return func(v time.Duration) error { - if err := NonNegativeDuration(v); err != nil { +func NonNegativeDurationWithMinimum(minValue time.Duration) SettingOption { + return WithValidateDuration(func(v time.Duration) error { + if err := nonNegativeDurationInternal(v); err != nil { return err } if v < minValue { @@ -229,13 +235,29 @@ func NonNegativeDurationWithMinimum(minValue time.Duration) func(time.Duration) minValue) } return nil - } + }) +} + +// NonNegativeDurationWithMinimumOrZeroDisable returns a validation +// function can be passed to RegisterDurationSetting. It verifies the value +// is at least the given minimum, or zero to disable. +func NonNegativeDurationWithMinimumOrZeroDisable(minValue time.Duration) SettingOption { + return WithValidateDuration(func(v time.Duration) error { + if err := nonNegativeDurationInternal(v); err != nil { + return err + } + if v != 0 && v < minValue { + return errors.Errorf("cannot be set to a value smaller than %s", + minValue) + } + return nil + }) } // PositiveDuration can be passed to RegisterDurationSetting. -func PositiveDuration(v time.Duration) error { +var PositiveDuration SettingOption = WithValidateDuration(func(v time.Duration) error { if v <= 0 { return errors.Errorf("cannot be set to a non-positive duration: %s", v) } return nil -} +}) diff --git a/pkg/settings/duration_test.go b/pkg/settings/duration_test.go new file mode 100644 index 000000000000..41d6bb09203a --- /dev/null +++ b/pkg/settings/duration_test.go @@ -0,0 +1,27 @@ +// Copyright 2023 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 settings + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestNonNegativeDurationWithMinimum(t *testing.T) { + validatorOpt := NonNegativeDurationWithMinimum(time.Minute) + validator := validatorOpt.validateDurationFn + require.NoError(t, validator(time.Minute)) + require.NoError(t, validator(2*time.Minute)) + require.Error(t, validator(59*time.Second)) + require.Error(t, validator(-1*time.Second)) +} diff --git a/pkg/settings/enum.go b/pkg/settings/enum.go index 5d96d37e9a31..39ddbaac0d79 100644 --- a/pkg/settings/enum.go +++ b/pkg/settings/enum.go @@ -116,15 +116,14 @@ func enumValuesToDesc(enumValues map[int64]string) string { return buffer.String() } -// WithPublic sets public visibility and can be chained. -func (e *EnumSetting) WithPublic() *EnumSetting { - e.SetVisibility(Public) - return e -} - // RegisterEnumSetting defines a new setting with type int. func RegisterEnumSetting( - class Class, key InternalKey, desc string, defaultValue string, enumValues map[int64]string, + class Class, + key InternalKey, + desc string, + defaultValue string, + enumValues map[int64]string, + opts ...SettingOption, ) *EnumSetting { enumValuesLower := make(map[int64]string) var i int64 @@ -147,5 +146,6 @@ func RegisterEnumSetting( } register(class, key, fmt.Sprintf("%s %s", desc, enumValuesToDesc(enumValues)), setting) + setting.apply(opts) return setting } diff --git a/pkg/settings/float.go b/pkg/settings/float.go index 18d1fa145f2e..1e858c0cb7e6 100644 --- a/pkg/settings/float.go +++ b/pkg/settings/float.go @@ -117,70 +117,117 @@ func (f *FloatSetting) setToDefault(ctx context.Context, sv *Values) { } } -// WithPublic sets public visibility and can be chained. -func (f *FloatSetting) WithPublic() *FloatSetting { - f.SetVisibility(Public) - return f -} - // RegisterFloatSetting defines a new setting with type float. func RegisterFloatSetting( - class Class, - key InternalKey, - desc string, - defaultValue float64, - validateFns ...func(float64) error, + class Class, key InternalKey, desc string, defaultValue float64, opts ...SettingOption, ) *FloatSetting { - var validateFn func(float64) error - if len(validateFns) > 0 { - validateFn = func(v float64) error { - for _, fn := range validateFns { + validateFn := func(v float64) error { + for _, opt := range opts { + if opt.validateDurationFn != nil || + opt.validateInt64Fn != nil || + opt.validateStringFn != nil || + opt.validateProtoFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateFloat64Fn; fn != nil { if err := fn(v); err != nil { - return errors.Wrapf(err, "invalid value for %s", key) + return err } } - return nil } + return nil } - if validateFn != nil { - if err := validateFn(defaultValue); err != nil { - panic(errors.Wrap(err, "invalid default")) - } + if err := validateFn(defaultValue); err != nil { + panic(errors.Wrap(err, "invalid default")) } setting := &FloatSetting{ defaultValue: defaultValue, validateFn: validateFn, } register(class, key, desc, setting) + setting.apply(opts) return setting } // NonNegativeFloat can be passed to RegisterFloatSetting. -func NonNegativeFloat(v float64) error { - if v < 0 { - return errors.Errorf("cannot set to a negative value: %f", v) - } - return nil +var NonNegativeFloat SettingOption = NonNegativeFloatWithMinimum(0) + +// NonNegativeFloatWithMinimum can be passed to RegisterFloatSetting. +func NonNegativeFloatWithMinimum(minVal float64) SettingOption { + return WithValidateFloat(func(v float64) error { + if v < 0 { + return errors.Errorf("cannot set to a negative value: %f", v) + } + if v < minVal { + return errors.Errorf("cannot set to a value lower than %f: %f", minVal, v) + } + return nil + }) } -// NonNegativeFloatWithMaximum can be passed to RegisterFloatSetting. -func NonNegativeFloatWithMaximum(maxValue float64) func(float64) error { - return func(v float64) error { +// NonNegativeFloatWithMinimumOrZeroDisable is an option that verifies +// the value is at least the given minimum, or zero to diassble. +func NonNegativeFloatWithMinimumOrZeroDisable(minVal float64) SettingOption { + return WithValidateFloat(func(v float64) error { if v < 0 { - return errors.Errorf("cannot be set to a negative value: %f", v) + return errors.Errorf("cannot set to a negative value: %f", v) } - if v > maxValue { - return errors.Errorf("cannot be set to a value larger than %f", maxValue) + if v != 0 && v < minVal { + return errors.Errorf("cannot set to a value lower than %f: %f", minVal, v) } return nil - } + }) +} + +// NonNegativeFloatWithMaximum can be passed to RegisterFloatSetting. +func NonNegativeFloatWithMaximum(maxValue float64) SettingOption { + return FloatInRange(0, maxValue) } // PositiveFloat can be passed to RegisterFloatSetting. -func PositiveFloat(v float64) error { +var PositiveFloat SettingOption = WithValidateFloat(func(v float64) error { if v <= 0 { return errors.Errorf("cannot set to a non-positive value: %f", v) } return nil +}) + +// NonZeroFloat can be passed to RegisterFloatSetting. +var NonZeroFloat SettingOption = WithValidateFloat(func(v float64) error { + if v == 0 { + return errors.New("cannot set to zero value") + } + return nil +}) + +// Fraction requires the setting to be in the interval [0, 1]. It can +// be passed to RegisterFloatSetting. +var Fraction SettingOption = FloatInRange(0, 1) + +// FloatInRange returns a validation function that checks the value +// is within the given bounds (inclusive). +func FloatInRange(minVal, maxVal float64) SettingOption { + return WithValidateFloat(func(v float64) error { + if v < minVal || v > maxVal { + return errors.Errorf("expected value in range [%f, %f], got: %f", minVal, maxVal, v) + } + return nil + }) +} + +// FractionUpperExclusive requires the setting to be in the interval +// [0, 1]. It can be passed to RegisterFloatSetting. +var FractionUpperExclusive SettingOption = FloatInRangeUpperExclusive(0, 1) + +// FloatInRangeUpperExclusive returns a validation function that +// checks the value is within the given bounds (inclusive lower, +// exclusive upper). +func FloatInRangeUpperExclusive(minVal, maxVal float64) SettingOption { + return WithValidateFloat(func(v float64) error { + if v < minVal || v >= maxVal { + return errors.Errorf("expected value in range [%f, %f), got: %f", minVal, maxVal, v) + } + return nil + }) } diff --git a/pkg/settings/int.go b/pkg/settings/int.go index 108a96472c7c..af6c6df37cc2 100644 --- a/pkg/settings/int.go +++ b/pkg/settings/int.go @@ -117,64 +117,91 @@ func (i *IntSetting) setToDefault(ctx context.Context, sv *Values) { // RegisterIntSetting defines a new setting with type int with a // validation function. func RegisterIntSetting( - class Class, key InternalKey, desc string, defaultValue int64, validateFns ...func(int64) error, + class Class, key InternalKey, desc string, defaultValue int64, opts ...SettingOption, ) *IntSetting { - var composed func(int64) error - if len(validateFns) > 0 { - composed = func(v int64) error { - for _, validateFn := range validateFns { - if err := validateFn(v); err != nil { - return errors.Wrapf(err, "invalid value for %s", key) + validateFn := func(val int64) error { + for _, opt := range opts { + if opt.validateDurationFn != nil || + opt.validateFloat64Fn != nil || + opt.validateStringFn != nil || + opt.validateProtoFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateInt64Fn; fn != nil { + if err := fn(val); err != nil { + return err } } - return nil } + return nil } - if composed != nil { - if err := composed(defaultValue); err != nil { - panic(errors.Wrap(err, "invalid default")) - } + if err := validateFn(defaultValue); err != nil { + panic(errors.Wrap(err, "invalid default")) } setting := &IntSetting{ defaultValue: defaultValue, - validateFn: composed, + validateFn: validateFn, } register(class, key, desc, setting) + setting.apply(opts) return setting } -// WithPublic sets public visibility and can be chained. -func (i *IntSetting) WithPublic() *IntSetting { - i.SetVisibility(Public) - return i -} - -// PositiveInt can be passed to RegisterIntSetting -func PositiveInt(v int64) error { - if v < 1 { +// PositiveInt can be passed to RegisterIntSetting. +var PositiveInt SettingOption = WithValidateInt(func(v int64) error { + if v <= 0 { return errors.Errorf("cannot be set to a non-positive value: %d", v) } return nil -} +}) // NonNegativeInt can be passed to RegisterIntSetting. -func NonNegativeInt(v int64) error { +var NonNegativeInt SettingOption = WithValidateInt(nonNegativeIntInternal) + +func nonNegativeIntInternal(v int64) error { if v < 0 { return errors.Errorf("cannot be set to a negative value: %d", v) } return nil } +// NonNegativeIntWithMinimum can be passed to RegisterIntSetting. +func NonNegativeIntWithMinimum(minVal int64) SettingOption { + return WithValidateInt(func(v int64) error { + if err := nonNegativeIntInternal(v); err != nil { + return err + } + if v < minVal { + return errors.Errorf("cannot be set to a value lower than %d: %d", minVal, v) + } + return nil + }) +} + // NonNegativeIntWithMaximum returns a validation function that can be // passed to RegisterIntSetting. -func NonNegativeIntWithMaximum(maxValue int64) func(int64) error { - return func(v int64) error { - if v < 0 { - return errors.Errorf("cannot be set to a negative integer: %d", v) +func NonNegativeIntWithMaximum(maxValue int64) SettingOption { + return IntInRange(0, maxValue) +} + +// IntInRange returns a validation function that checks the value +// is within the given bounds (inclusive). +func IntInRange(minVal, maxVal int64) SettingOption { + return WithValidateInt(func(v int64) error { + if v < minVal || v > maxVal { + return errors.Errorf("expected value in range [%d, %d], got: %d", minVal, maxVal, v) } - if v > maxValue { - return errors.Errorf("cannot be set to a value larger than %d", maxValue) + return nil + }) +} + +// IntInRangeOrZeroDisable returns a validation function that checks the value +// is within the given bounds (inclusive) or is zero (disabled). +func IntInRangeOrZeroDisable(minVal, maxVal int64) SettingOption { + return WithValidateInt(func(v int64) error { + if v != 0 && (v < minVal || v > maxVal) { + return errors.Errorf("expected value in range [%d, %d] or 0 to disable, got: %d", minVal, maxVal, v) } return nil - } + }) } diff --git a/pkg/settings/options.go b/pkg/settings/options.go new file mode 100644 index 000000000000..5255cfbb6445 --- /dev/null +++ b/pkg/settings/options.go @@ -0,0 +1,88 @@ +// Copyright 2021 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 settings + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/util/protoutil" +) + +// SettingOption is the type of an option that can be passed to Register. +type SettingOption struct { + commonOpt func(*common) + validateDurationFn func(time.Duration) error + validateInt64Fn func(int64) error + validateFloat64Fn func(float64) error + validateStringFn func(*Values, string) error + validateProtoFn func(*Values, protoutil.Message) error +} + +// WithVisibility customizes the visibility of a setting. +func WithVisibility(v Visibility) SettingOption { + return SettingOption{commonOpt: func(c *common) { + c.setVisibility(v) + }} +} + +// WithPublic sets public visibility. +var WithPublic SettingOption = WithVisibility(Public) + +// WithReportable indicates whether a setting's value can show up in SHOW ALL +// CLUSTER SETTINGS and telemetry reports. +func WithReportable(reportable bool) SettingOption { + return SettingOption{commonOpt: func(c *common) { + c.setReportable(reportable) + }} +} + +// Retired marks the setting as obsolete. It also hides it from the +// output of SHOW CLUSTER SETTINGS. Note: in many case the setting +// definition can be removed outright, and its name added to the +// retiredSettings map (in settings/registry.go). The Retired option +// exists for cases where there is a need to maintain compatibility +// with autommation. +var Retired SettingOption = SettingOption{commonOpt: func(c *common) { + c.setRetired() +}} + +// WithValidateDuration adds a validation function for a duration setting. +func WithValidateDuration(fn func(time.Duration) error) SettingOption { + return SettingOption{validateDurationFn: fn} +} + +// WithValidateInt adds a validation function for an int64 setting. +func WithValidateInt(fn func(int64) error) SettingOption { + return SettingOption{validateInt64Fn: fn} +} + +// WithValidateFloat adds a validation function for a float64 setting. +func WithValidateFloat(fn func(float64) error) SettingOption { + return SettingOption{validateFloat64Fn: fn} +} + +// WithValidateString adds a validation function for a string setting. +func WithValidateString(fn func(*Values, string) error) SettingOption { + return SettingOption{validateStringFn: fn} +} + +// WithValidateProto adds a validation function for a proto setting. +func WithValidateProto(fn func(*Values, protoutil.Message) error) SettingOption { + return SettingOption{validateProtoFn: fn} +} + +func (c *common) apply(opts []SettingOption) { + for _, opt := range opts { + if opt.commonOpt != nil { + opt.commonOpt(c) + } + } +} diff --git a/pkg/settings/protobuf.go b/pkg/settings/protobuf.go index 3d0e06156251..c4ab0947cfb8 100644 --- a/pkg/settings/protobuf.go +++ b/pkg/settings/protobuf.go @@ -126,12 +126,6 @@ func (s *ProtobufSetting) setToDefault(ctx context.Context, sv *Values) { } } -// WithPublic sets public visibility and can be chained. -func (s *ProtobufSetting) WithPublic() *ProtobufSetting { - s.SetVisibility(Public) - return s -} - // MarshalToJSON returns a JSON representation of the protobuf. func (s *ProtobufSetting) MarshalToJSON(p protoutil.Message) (string, error) { jsonEncoder := jsonpb.Marshaler{EmitDefaults: false} @@ -154,24 +148,23 @@ func (s *ProtobufSetting) UnmarshalFromJSON(jsonEncoded string) (protoutil.Messa // RegisterProtobufSetting defines a new setting with type protobuf. func RegisterProtobufSetting( - class Class, key InternalKey, desc string, defaultValue protoutil.Message, -) *ProtobufSetting { - return RegisterValidatedProtobufSetting(class, key, desc, defaultValue, nil) -} - -// RegisterValidatedProtobufSetting defines a new setting with type protobuf -// with a validation function. -func RegisterValidatedProtobufSetting( - class Class, - key InternalKey, - desc string, - defaultValue protoutil.Message, - validateFn func(*Values, protoutil.Message) error, + class Class, key InternalKey, desc string, defaultValue protoutil.Message, opts ...SettingOption, ) *ProtobufSetting { - if validateFn != nil { - if err := validateFn(nil, defaultValue); err != nil { - panic(errors.Wrap(err, "invalid default")) + validateFn := func(sv *Values, p protoutil.Message) error { + for _, opt := range opts { + if opt.validateInt64Fn != nil || + opt.validateFloat64Fn != nil || + opt.validateStringFn != nil || + opt.validateDurationFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateProtoFn; fn != nil { + if err := fn(sv, p); err != nil { + return err + } + } } + return nil } setting := &ProtobufSetting{ defaultValue: defaultValue, @@ -180,14 +173,14 @@ func RegisterValidatedProtobufSetting( // By default, all protobuf settings are considered to contain PII and are // thus non-reportable (to exclude them from telemetry reports). - setting.SetReportable(false) + setting.setReportable(false) register(class, key, desc, setting) + setting.apply(opts) return setting } // Defeat the unused linter. var _ = (*ProtobufSetting).Default -var _ = (*ProtobufSetting).WithPublic // newProtoMessage creates a new protocol message object, given its fully // qualified name. diff --git a/pkg/settings/settings_test.go b/pkg/settings/settings_test.go index b541d574eb8a..28d5f73f5b02 100644 --- a/pkg/settings/settings_test.go +++ b/pkg/settings/settings_test.go @@ -155,7 +155,7 @@ var i1A = settings.RegisterIntSetting(settings.TenantWritable, "i.1", "desc", 0) var i2A = settings.RegisterIntSetting(settings.TenantWritable, "i.2", "desc", 5) var fA = settings.RegisterFloatSetting(settings.TenantReadOnly, "f", "desc", 5.4) var dA = settings.RegisterDurationSetting(settings.TenantWritable, "d", "desc", time.Second) -var duA = settings.RegisterPublicDurationSettingWithExplicitUnit(settings.TenantWritable, "d_with_explicit_unit", "desc", time.Second, settings.NonNegativeDuration) +var duA = settings.RegisterDurationSettingWithExplicitUnit(settings.TenantWritable, "d_with_explicit_unit", "desc", time.Second, settings.NonNegativeDuration, settings.WithPublic) var _ = settings.RegisterDurationSetting(settings.TenantWritable, "d_with_maximum", "desc", time.Second, settings.NonNegativeDurationWithMaximum(time.Hour)) var eA = settings.RegisterEnumSetting(settings.SystemOnly, "e", "desc", "foo", map[int64]string{1: "foo", 2: "bar", 3: "baz"}) var byteSize = settings.RegisterByteSizeSetting(settings.TenantWritable, "zzz", "desc", mb) @@ -166,37 +166,29 @@ var mA = func() *settings.VersionSetting { }() func init() { - settings.RegisterBoolSetting(settings.SystemOnly, "sekretz", "desc", false).SetReportable(false) - settings.RegisterBoolSetting(settings.SystemOnly, "rezervedz", "desc", false).SetVisibility(settings.Reserved) + _ = settings.RegisterBoolSetting(settings.SystemOnly, "sekretz", "desc", false, settings.WithReportable(false)) + _ = settings.RegisterBoolSetting(settings.SystemOnly, "rezervedz", "desc", false, settings.WithVisibility(settings.Reserved)) } -var strVal = settings.RegisterValidatedStringSetting(settings.SystemOnly, - "str.val", "desc", "", func(sv *settings.Values, v string) error { +var strVal = settings.RegisterStringSetting(settings.SystemOnly, + "str.val", "desc", "", settings.WithValidateString(func(sv *settings.Values, v string) error { for _, c := range v { if !unicode.IsLetter(c) { return errors.Errorf("not all runes of %s are letters: %c", v, c) } } return nil - }) + })) var dVal = settings.RegisterDurationSetting(settings.SystemOnly, "dVal", "desc", time.Second, settings.NonNegativeDuration) var fVal = settings.RegisterFloatSetting(settings.SystemOnly, "fVal", "desc", 5.4, settings.NonNegativeFloat) var byteSizeVal = settings.RegisterByteSizeSetting(settings.SystemOnly, "byteSize.Val", "desc", mb) var iVal = settings.RegisterIntSetting(settings.SystemOnly, - "i.Val", "desc", 0, func(v int64) error { + "i.Val", "desc", 0, settings.WithValidateInt(func(v int64) error { if v < 0 { return errors.Errorf("int cannot be negative") } return nil - }) - -func TestNonNegativeDurationWithMinimum(t *testing.T) { - validator := settings.NonNegativeDurationWithMinimum(time.Minute) - require.NoError(t, validator(time.Minute)) - require.NoError(t, validator(2*time.Minute)) - require.Error(t, validator(59*time.Second)) - require.Error(t, validator(-1*time.Second)) -} + })) func TestValidation(t *testing.T) { ctx := context.Background() diff --git a/pkg/settings/string.go b/pkg/settings/string.go index eb7fa3fc3b8e..cae0d93efbca 100644 --- a/pkg/settings/string.go +++ b/pkg/settings/string.go @@ -100,32 +100,25 @@ func (s *StringSetting) setToDefault(ctx context.Context, sv *Values) { } } -// WithPublic sets public visibility and can be chained. -func (s *StringSetting) WithPublic() *StringSetting { - s.SetVisibility(Public) - return s -} - // RegisterStringSetting defines a new setting with type string. func RegisterStringSetting( - class Class, key InternalKey, desc string, defaultValue string, -) *StringSetting { - return RegisterValidatedStringSetting(class, key, desc, defaultValue, nil) -} - -// RegisterValidatedStringSetting defines a new setting with type string with a -// validation function. -func RegisterValidatedStringSetting( - class Class, - key InternalKey, - desc string, - defaultValue string, - validateFn func(*Values, string) error, + class Class, key InternalKey, desc string, defaultValue string, opts ...SettingOption, ) *StringSetting { - if validateFn != nil { - if err := validateFn(nil, defaultValue); err != nil { - panic(errors.Wrap(err, "invalid default")) + validateFn := func(sv *Values, val string) error { + for _, opt := range opts { + if opt.validateDurationFn != nil || + opt.validateFloat64Fn != nil || + opt.validateInt64Fn != nil || + opt.validateProtoFn != nil { + panic(errors.AssertionFailedf("wrong validator type")) + } + if fn := opt.validateStringFn; fn != nil { + if err := fn(sv, val); err != nil { + return err + } + } } + return nil } setting := &StringSetting{ defaultValue: defaultValue, @@ -134,7 +127,8 @@ func RegisterValidatedStringSetting( // By default all string settings are considered to perhaps contain // PII and are thus non-reportable (to exclude them from telemetry // reports). - setting.SetReportable(false) + setting.setReportable(false) register(class, key, desc, setting) + setting.apply(opts) return setting } diff --git a/pkg/settings/version.go b/pkg/settings/version.go index cfec2f4e0652..f5a448b9bd50 100644 --- a/pkg/settings/version.go +++ b/pkg/settings/version.go @@ -181,6 +181,9 @@ func (v *VersionSetting) setToDefault(ctx context.Context, sv *Values) {} // RegisterVersionSetting adds the provided version setting to the global // registry. -func RegisterVersionSetting(class Class, key InternalKey, desc string, setting *VersionSetting) { +func RegisterVersionSetting( + class Class, key InternalKey, desc string, setting *VersionSetting, opts ...SettingOption, +) { register(class, key, desc, setting) + setting.apply(opts) } diff --git a/pkg/spanconfig/spanconfigreporter/reporter.go b/pkg/spanconfig/spanconfigreporter/reporter.go index aa1f2bee27ef..9005e494b19d 100644 --- a/pkg/spanconfig/spanconfigreporter/reporter.go +++ b/pkg/spanconfig/spanconfigreporter/reporter.go @@ -14,7 +14,6 @@ package spanconfigreporter import ( "context" - "fmt" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -33,12 +32,7 @@ var rangeDescPageSize = settings.RegisterIntSetting( "spanconfig.reporter.range_desc_page_size", "pa", 100, - func(i int64) error { - if i < 5 || i > 25000 { - return fmt.Errorf("expected range_desc_page_size to be in range [5, 25000], got %d", i) - } - return nil - }, + settings.IntInRange(5, 25000), ) var conformanceReportRateLimit = settings.RegisterFloatSetting( @@ -46,12 +40,7 @@ var conformanceReportRateLimit = settings.RegisterFloatSetting( "spanconfig.reporter.report_rate_limit", "the number of calls per second allowed to SpanConfigConformance", 1.0, - func(f float64) error { - if f < 0 || f > 1e9 { - return fmt.Errorf("expected report_rate_limit to be in range [0, 1e9], got %f", f) - } - return nil - }, + settings.FloatInRange(0, 1e9), ) // Reporter is used to figure out whether ranges backing specific spans conform diff --git a/pkg/spanconfig/spanconfigstore/span_store.go b/pkg/spanconfig/spanconfigstore/span_store.go index 3b2c9adc5096..9c74977a18d3 100644 --- a/pkg/spanconfig/spanconfigstore/span_store.go +++ b/pkg/spanconfig/spanconfigstore/span_store.go @@ -33,7 +33,7 @@ var tenantCoalesceAdjacentSetting = settings.RegisterBoolSetting( "spanconfig.tenant_coalesce_adjacent.enabled", `collapse adjacent ranges with the same span configs across all secondary tenant keyspaces`, true, -).WithPublic() + settings.WithPublic) // StorageCoalesceAdjacentSetting is a public cluster setting that controls // whether we coalesce adjacent ranges outside of the secondary tenant keyspaces @@ -43,7 +43,7 @@ var StorageCoalesceAdjacentSetting = settings.RegisterBoolSetting( "spanconfig.storage_coalesce_adjacent.enabled", `collapse adjacent ranges with the same span configs, for the ranges specific to the system tenant`, true, -).WithPublic() + settings.WithPublic) // spanConfigStore is an in-memory data structure to store and retrieve // SpanConfigs associated with a single span. Internally it makes use of a diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index caf6abac98f8..f644724bb3db 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -56,7 +56,7 @@ var boundsEnabled = settings.RegisterBoolSetting( "spanconfig.bounds.enabled", "dictates whether span config bounds are consulted when serving span configs for secondary tenants", true, -).WithPublic() + settings.WithPublic) // Store is an in-memory data structure to store, retrieve, and incrementally // update the span configuration state. Internally, it makes use of an interval diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 0e10e9fb05d2..3c61a6c107e0 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -294,7 +294,7 @@ var allowDropFinalRegion = settings.RegisterBoolSetting( "sql.multiregion.drop_primary_region.enabled", "allows dropping the PRIMARY REGION of a database if it is the last region", true, -).WithPublic() + settings.WithPublic) // AlterDatabaseDropRegion transforms a tree.AlterDatabaseDropRegion into a plan node. func (p *planner) AlterDatabaseDropRegion( diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index eaec233aa548..eb81a40fb624 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -75,7 +75,7 @@ var changeOwnPasswordEnabled = settings.RegisterBoolSetting( "sql.auth.change_own_password.enabled", "controls whether a user is allowed to change their own password, even if they have no other privileges", false, -).WithPublic() + settings.WithPublic) // AlterRole represents a `ALTER ROLE ... [WITH] OPTION` statement. // Privileges: CREATEROLE privilege. diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index 8e34221f6e4a..c6ef3253af1e 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -648,7 +648,7 @@ var useSingleQueryForRoleMembershipCache = settings.RegisterBoolSetting( "sql.auth.resolve_membership_single_scan.enabled", "determines whether to populate the role membership cache with a single scan", defaultSingleQueryForRoleMembershipCache, -).WithPublic() + settings.WithPublic) // resolveMemberOfWithAdminOption performs the actual recursive role membership lookup. func resolveMemberOfWithAdminOption( diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 691138618be2..8a2a6dd136da 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -65,20 +65,13 @@ var LeaseDuration = settings.RegisterDurationSetting( "mean duration of sql descriptor leases, this actual duration is jitterred", base.DefaultDescriptorLeaseDuration) -func between0and1inclusive(f float64) error { - if f < 0 || f > 1 { - return errors.Errorf("value %f must be between 0 and 1", f) - } - return nil -} - // LeaseJitterFraction controls the percent jitter around sql lease durations var LeaseJitterFraction = settings.RegisterFloatSetting( settings.TenantWritable, "sql.catalog.descriptor_lease_jitter_fraction", "mean duration of sql descriptor leases, this actual duration is jitterred", base.DefaultDescriptorLeaseJitterFraction, - between0and1inclusive) + settings.Fraction) // WaitForNoVersion returns once there are no unexpired leases left // for any version of the descriptor. diff --git a/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go b/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go index ffbd13b7cfa8..db266d0bd486 100644 --- a/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go +++ b/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go @@ -38,18 +38,19 @@ const SchemaTelemetryScheduleName = "sql-schema-telemetry" // SchemaTelemetryRecurrence is the cron-tab string specifying the recurrence // for schema telemetry job. -var SchemaTelemetryRecurrence = settings.RegisterValidatedStringSetting( +var SchemaTelemetryRecurrence = settings.RegisterStringSetting( settings.TenantReadOnly, "sql.schema.telemetry.recurrence", "cron-tab recurrence for SQL schema telemetry job", "@weekly", /* defaultValue */ - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { if _, err := cron.ParseStandard(s); err != nil { return errors.Wrap(err, "invalid cron expression") } return nil - }, -).WithPublic() + }), + settings.WithPublic, +) // ErrDuplicatedSchedules indicates that there is already a schedule for SQL // schema telemetry jobs existing in the system.scheduled_jobs table. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index 2776f80a6042..137c0480780a 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -311,7 +311,7 @@ var DefaultHashShardedIndexBucketCount = settings.RegisterIntSetting( "used as bucket count if bucket count is not specified in hash sharded index definition", 16, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // GetShardColumnName generates a name for the hidden shard column to be used to create a // hash sharded index. diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 6b215c560b4a..ab883869a6fa 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -14,7 +14,6 @@ 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/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -1821,8 +1820,8 @@ func (desc *wrapper) validateMinStaleRows(vea catalog.ValidationErrorAccumulator 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)) + if *value < 0 { + vea.Report(errors.Newf("invalid integer value for %s: cannot be negative", settingName)) } } } @@ -1833,8 +1832,8 @@ func (desc *wrapper) validateFractionStaleRows( 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)) + if *value < 0 { + vea.Report(errors.Newf("invalid float value for %s: cannot be negative", settingName)) } } } diff --git a/pkg/sql/closed_session_cache.go b/pkg/sql/closed_session_cache.go index 22b2bcc7d03b..7d731a128c70 100644 --- a/pkg/sql/closed_session_cache.go +++ b/pkg/sql/closed_session_cache.go @@ -34,7 +34,7 @@ var ClosedSessionCacheCapacity = settings.RegisterIntSetting( "sql.closed_session_cache.capacity", "the maximum number of sessions in the cache", 1000, // TODO(gtr): Totally arbitrary for now, adjust later. -).WithPublic() + settings.WithPublic) // ClosedSessionCacheTimeToLive is the cluster setting that controls the maximum time // to live for a session's information in the cache, in seconds. @@ -43,7 +43,7 @@ var ClosedSessionCacheTimeToLive = settings.RegisterIntSetting( "sql.closed_session_cache.time_to_live", "the maximum time to live, in seconds", 3600, // One hour -).WithPublic() + settings.WithPublic) // ClosedSessionCache is an in-memory FIFO cache for closed sessions. type ClosedSessionCache struct { diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 4b55a804fcd1..77b37466bd36 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -97,7 +97,7 @@ var maxNumNonAdminConnections = settings.RegisterIntSetting( "(note: this will only limit future connection attempts and will not affect already established connections). "+ "Negative values result in unlimited number of connections. Superusers are not affected by this limit.", -1, // Postgres defaults to 100, but we default to -1 to match our previous behavior of unlimited. -).WithPublic() + settings.WithPublic) // Note(alyshan): This setting is not public. It is intended to be used by Cockroach Cloud to limit // connections to serverless clusters while still being able to connect from the Cockroach Cloud control plane. diff --git a/pkg/sql/contention/cluster_settings.go b/pkg/sql/contention/cluster_settings.go index 18d405f65af2..ff6aeabbf275 100644 --- a/pkg/sql/contention/cluster_settings.go +++ b/pkg/sql/contention/cluster_settings.go @@ -33,7 +33,7 @@ var StoreCapacity = settings.RegisterByteSizeSetting( "sql.contention.event_store.capacity", "the in-memory storage capacity per-node of contention event store", 64*1024*1024, // 64 MB per node. -).WithPublic() + settings.WithPublic) // DurationThreshold is the cluster setting for the threshold of // contention durations. Only the contention events whose duration exceeds the @@ -44,4 +44,4 @@ var DurationThreshold = settings.RegisterDurationSetting( "minimum contention duration to cause the contention events to be collected "+ "into crdb_internal.transaction_contention_events", 0, -).WithPublic() + settings.WithPublic) diff --git a/pkg/sql/contention/txnidcache/cluster_settings.go b/pkg/sql/contention/txnidcache/cluster_settings.go index c739b468be6e..3b7750d836c0 100644 --- a/pkg/sql/contention/txnidcache/cluster_settings.go +++ b/pkg/sql/contention/txnidcache/cluster_settings.go @@ -18,4 +18,4 @@ var MaxSize = settings.RegisterByteSizeSetting( `sql.contention.txn_id_cache.max_size`, "the maximum byte size TxnID cache will use (set to 0 to disable)", 64*1024*1024, // 64MiB -).WithPublic() + settings.WithPublic) diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index b8079221efe5..30bfdcb37ad2 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -50,7 +50,7 @@ var createStatsPostEvents = settings.RegisterBoolSetting( "sql.stats.post_events.enabled", "if set, an event is logged for every CREATE STATISTICS job", false, -).WithPublic() + settings.WithPublic) // featureStatsEnabled is used to enable and disable the CREATE STATISTICS and // ANALYZE features. @@ -59,7 +59,7 @@ var featureStatsEnabled = settings.RegisterBoolSetting( "feature.stats.enabled", "set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) const nonIndexColHistogramBuckets = 2 diff --git a/pkg/sql/deprecatedshowranges/condition.go b/pkg/sql/deprecatedshowranges/condition.go index b29cc0981c52..4292fb23a33e 100644 --- a/pkg/sql/deprecatedshowranges/condition.go +++ b/pkg/sql/deprecatedshowranges/condition.go @@ -105,7 +105,7 @@ var ShowRangesDeprecatedBehaviorSetting = settings.RegisterBoolSetting( " NB: the new SHOW RANGES interface has richer WITH options "+ "than pre-v23.1 SHOW RANGES.", false, -).WithPublic() + settings.WithPublic) const envVarName = "COCKROACH_FORCE_DEPRECATED_SHOW_RANGE_BEHAVIOR" diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index c48a84024073..72f0e223c502 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -68,7 +68,7 @@ var PublicSchemaCreatePrivilegeEnabled = settings.RegisterBoolSetting( "determines whether to grant all users the CREATE privileges on the public "+ "schema when it is created", true, -).WithPublic() + settings.WithPublic) // createDatabase takes Database descriptor and creates it if needed, // incrementing the descriptor counter. Returns true if the descriptor @@ -347,7 +347,7 @@ var DefaultPrimaryRegion = settings.RegisterStringSetting( `if not empty, all databases created without a PRIMARY REGION will `+ `implicitly have the given PRIMARY REGION`, "", -).WithPublic() + settings.WithPublic) // SecondaryTenantsMultiRegionAbstractionsEnabledSettingName is the name of the // cluster setting that governs secondary tenant multi-region abstraction usage. diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index c549bbdf2f8f..53c46078f86f 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -53,7 +53,7 @@ var histogramSamples = settings.RegisterIntSetting( "number of rows sampled for histogram construction during table statistics collection", 10000, settings.NonNegativeIntWithMaximum(math.MaxUint32), -).WithPublic() + settings.WithPublic) // maxTimestampAge is the maximum allowed age of a scan timestamp during table // stats collection, used when creating statistics AS OF SYSTEM TIME. The diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 50cecc9f3426..ea9cf170bbe6 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -71,15 +71,7 @@ var settingDistSQLNumRunners = settings.RegisterIntSetting( // The choice of the default multiple of 4 was made in order to get the // original value of 16 on machines with 4 CPUs. 4*int64(runtime.GOMAXPROCS(0)), /* defaultValue */ - func(v int64) error { - if v < 0 { - return errors.Errorf("cannot be set to a negative value: %d", v) - } - if v > distSQLNumRunnersMax { - return errors.Errorf("cannot be set to a value exceeding %d: %d", distSQLNumRunnersMax, v) - } - return nil - }, + settings.IntInRange(0, distSQLNumRunnersMax), ) // Somewhat arbitrary upper bound. diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index 5f3b7d23c7bb..7de178d2c91b 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -437,7 +437,7 @@ var eventLogSystemTableEnabled = settings.RegisterBoolSetting( "server.eventlog.enabled", "if set, logged notable events are also stored in the table system.eventlog", true, -).WithPublic() + settings.WithPublic) // EventLogTestingKnobs provides hooks and knobs for event logging. type EventLogTestingKnobs struct { diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index feb33b75fa42..2695695ddfd6 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -51,15 +51,16 @@ var logStatementsExecuteEnabled = settings.RegisterBoolSetting( "sql.trace.log_statement_execute", "set to true to enable logging of executed statements", false, -).WithPublic() + settings.WithPublic) -var slowQueryLogThreshold = settings.RegisterPublicDurationSettingWithExplicitUnit( +var slowQueryLogThreshold = settings.RegisterDurationSettingWithExplicitUnit( settings.TenantWritable, "sql.log.slow_query.latency_threshold", "when set to non-zero, log statements whose service latency exceeds "+ "the threshold to a secondary logger on each node", 0, settings.NonNegativeDuration, + settings.WithPublic, ) var slowInternalQueryLogEnabled = settings.RegisterBoolSetting( @@ -69,7 +70,7 @@ var slowInternalQueryLogEnabled = settings.RegisterBoolSetting( "are logged to a separate log. Must have the slow query log enabled for this "+ "setting to have any effect.", false, -).WithPublic() + settings.WithPublic) var slowQueryLogFullTableScans = settings.RegisterBoolSetting( settings.TenantWritable, @@ -78,7 +79,7 @@ var slowQueryLogFullTableScans = settings.RegisterBoolSetting( "slow query log even if they do not meet the latency threshold. Must have the slow query "+ "log enabled for this setting to have any effect.", false, -).WithPublic() + settings.WithPublic) var adminAuditLogEnabled = settings.RegisterBoolSetting( settings.TenantWritable, @@ -98,7 +99,7 @@ var telemetryLoggingEnabled = settings.RegisterBoolSetting( // setting directly after tenant creation without significant overhead in terms // of time and code. envutil.EnvOrDefaultBool("COCKROACH_SQL_TELEMETRY_QUERY_SAMPLING_ENABLED", false), -).WithPublic() + settings.WithPublic) type executorType int diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 111f917d7d16..fd89f9634260 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -131,7 +131,7 @@ var ClusterOrganization = settings.RegisterStringSetting( "cluster.organization", "organization name", "", -).WithPublic() + settings.WithPublic) // ClusterIsInternal returns true if the cluster organization contains // "Cockroach Labs", indicating an internal cluster. @@ -141,36 +141,33 @@ func ClusterIsInternal(sv *settings.Values) bool { // ClusterSecret is a cluster specific secret. This setting is // non-reportable. -var ClusterSecret = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.TenantWritable, - "cluster.secret", - "cluster specific secret", - "", - ) +var ClusterSecret = settings.RegisterStringSetting( + settings.TenantWritable, + "cluster.secret", + "cluster specific secret", + "", // Even though string settings are non-reportable by default, we // still mark them explicitly in case a future code change flips the // default. - s.SetReportable(false) - return s -}() + settings.WithReportable(false), +) // defaultIntSize controls how a "naked" INT type will be parsed. // TODO(bob): Change this to 4 in v2.3; https://github.com/cockroachdb/cockroach/issues/32534 // TODO(bob): Remove or n-op this in v2.4: https://github.com/cockroachdb/cockroach/issues/32844 -var defaultIntSize = func() *settings.IntSetting { - s := settings.RegisterIntSetting( - settings.TenantWritable, - "sql.defaults.default_int_size", - "the size, in bytes, of an INT type", 8, func(i int64) error { - if i != 4 && i != 8 { - return errors.New("only 4 or 8 are valid values") - } - return nil - }).WithPublic() - s.SetVisibility(settings.Public) - return s -}() +var defaultIntSize = settings.RegisterIntSetting( + settings.TenantWritable, + "sql.defaults.default_int_size", + "the size, in bytes, of an INT type", + 8, + settings.WithValidateInt(func(i int64) error { + if i != 4 && i != 8 { + return errors.New("only 4 or 8 are valid values") + } + return nil + }), + settings.WithPublic, +) const allowCrossDatabaseFKsSetting = "sql.cross_db_fks.enabled" @@ -179,7 +176,7 @@ var allowCrossDatabaseFKs = settings.RegisterBoolSetting( allowCrossDatabaseFKsSetting, "if true, creating foreign key references across databases is allowed", false, -).WithPublic() + settings.WithPublic) const allowCrossDatabaseViewsSetting = "sql.cross_db_views.enabled" @@ -188,7 +185,7 @@ var allowCrossDatabaseViews = settings.RegisterBoolSetting( allowCrossDatabaseViewsSetting, "if true, creating views that refer to other databases is allowed", false, -).WithPublic() + settings.WithPublic) const allowCrossDatabaseSeqOwnerSetting = "sql.cross_db_sequence_owners.enabled" @@ -197,7 +194,7 @@ var allowCrossDatabaseSeqOwner = settings.RegisterBoolSetting( allowCrossDatabaseSeqOwnerSetting, "if true, creating sequences owned by tables from other databases is allowed", false, -).WithPublic() + settings.WithPublic) const allowCrossDatabaseSeqReferencesSetting = "sql.cross_db_sequence_references.enabled" @@ -206,7 +203,7 @@ var allowCrossDatabaseSeqReferences = settings.RegisterBoolSetting( allowCrossDatabaseSeqReferencesSetting, "if true, sequences referenced by tables from other databases are allowed", false, -).WithPublic() + settings.WithPublic) // SecondaryTenantZoneConfigsEnabled controls if secondary tenants are allowed // to set zone configurations. It has no effect for the system tenant. @@ -255,7 +252,7 @@ var traceTxnThreshold = settings.RegisterDurationSetting( "this setting is coarser-grained than sql.trace.stmt.enable_threshold "+ "because it applies to all statements within a transaction as well as "+ "client communication (e.g. retries)", 0, -).WithPublic() + settings.WithPublic) // TraceStmtThreshold is identical to traceTxnThreshold except it applies to // individual statements in a transaction. The motivation for this setting is @@ -270,7 +267,7 @@ var TraceStmtThreshold = settings.RegisterDurationSetting( "this setting applies to individual statements within a transaction and "+ "is therefore finer-grained than sql.trace.txn.enable_threshold", 0, -).WithPublic() + settings.WithPublic) // traceSessionEventLogEnabled can be used to enable the event log // that is normally kept for every SQL connection. The event log has a @@ -282,7 +279,7 @@ var traceSessionEventLogEnabled = settings.RegisterBoolSetting( "set to true to enable session tracing; "+ "note that enabling this may have a negative performance impact", false, -).WithPublic() + settings.WithPublic) // ReorderJoinsLimitClusterSettingName is the name of the cluster setting for // the maximum number of joins to reorder. @@ -295,24 +292,16 @@ var ReorderJoinsLimitClusterValue = settings.RegisterIntSetting( ReorderJoinsLimitClusterSettingName, "default number of joins to reorder", opt.DefaultJoinOrderLimit, - func(limit int64) error { - if limit < 0 || limit > opt.MaxReorderJoinsLimit { - return pgerror.Newf(pgcode.InvalidParameterValue, - "cannot set %s to a value less than 0 or greater than %v", - ReorderJoinsLimitClusterSettingName, - opt.MaxReorderJoinsLimit, - ) - } - return nil - }, -).WithPublic() + settings.IntInRange(0, opt.MaxReorderJoinsLimit), + settings.WithPublic, +) var requireExplicitPrimaryKeysClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.require_explicit_primary_keys.enabled", "default value for requiring explicit primary keys in CREATE TABLE statements", false, -).WithPublic() + settings.WithPublic) var placementEnabledClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, @@ -336,21 +325,21 @@ var onUpdateRehomeRowEnabledClusterMode = settings.RegisterBoolSetting( "default value for on_update_rehome_row;"+ " enables ON UPDATE rehome_row() expressions to trigger on updates", true, -).WithPublic() + settings.WithPublic) var temporaryTablesEnabledClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.experimental_temporary_tables.enabled", "default value for experimental_enable_temp_tables; allows for use of temporary tables by default", false, -).WithPublic() + settings.WithPublic) var implicitColumnPartitioningEnabledClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.experimental_implicit_column_partitioning.enabled", "default value for experimental_enable_temp_tables; allows for the use of implicit column partitioning", false, -).WithPublic() + settings.WithPublic) var overrideMultiRegionZoneConfigClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, @@ -358,7 +347,7 @@ var overrideMultiRegionZoneConfigClusterMode = settings.RegisterBoolSetting( "default value for override_multi_region_zone_config; "+ "allows for overriding the zone configs of a multi-region table or database", false, -).WithPublic() + settings.WithPublic) var maxHashShardedIndexRangePreSplit = settings.RegisterIntSetting( settings.SystemOnly, @@ -366,14 +355,14 @@ var maxHashShardedIndexRangePreSplit = settings.RegisterIntSetting( "max pre-split ranges to have when adding hash sharded index to an existing table", 16, settings.PositiveInt, -).WithPublic() + settings.WithPublic) var zigzagJoinClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.zigzag_join.enabled", "default value for enable_zigzag_join session setting; allows use of zig-zag join by default", true, -).WithPublic() + settings.WithPublic) var optDrivenFKCascadesClusterLimit = settings.RegisterIntSetting( settings.TenantWritable, @@ -381,14 +370,14 @@ var optDrivenFKCascadesClusterLimit = settings.RegisterIntSetting( "default value for foreign_key_cascades_limit session setting; limits the number of cascading operations that run as part of a single query", 10000, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) var preferLookupJoinsForFKs = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.prefer_lookup_joins_for_fks.enabled", "default value for prefer_lookup_joins_for_fks session setting; causes foreign key operations to use lookup joins when possible", false, -).WithPublic() + settings.WithPublic) // optUseHistogramsClusterMode controls the cluster default for whether // histograms are used by the optimizer for cardinality estimation. @@ -401,7 +390,7 @@ var optUseHistogramsClusterMode = settings.RegisterBoolSetting( "sql.defaults.optimizer_use_histograms.enabled", "default value for optimizer_use_histograms session setting; enables usage of histograms in the optimizer by default", true, -).WithPublic() + settings.WithPublic) // optUseMultiColStatsClusterMode controls the cluster default for whether // multi-column stats are used by the optimizer for cardinality estimation. @@ -414,7 +403,7 @@ var optUseMultiColStatsClusterMode = settings.RegisterBoolSetting( "sql.defaults.optimizer_use_multicol_stats.enabled", "default value for optimizer_use_multicol_stats session setting; enables usage of multi-column stats in the optimizer by default", true, -).WithPublic() + settings.WithPublic) // localityOptimizedSearchMode controls the cluster default for the use of // locality optimized search. If enabled, the optimizer will try to plan scans @@ -427,21 +416,21 @@ var localityOptimizedSearchMode = settings.RegisterBoolSetting( "default value for locality_optimized_partitioned_index_scan session setting; "+ "enables searching for rows in the current region before searching remote regions", true, -).WithPublic() + settings.WithPublic) var implicitSelectForUpdateClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.implicit_select_for_update.enabled", "default value for enable_implicit_select_for_update session setting; enables FOR UPDATE locking during the row-fetch phase of mutation statements", true, -).WithPublic() + settings.WithPublic) var insertFastPathClusterMode = settings.RegisterBoolSetting( settings.TenantWritable, "sql.defaults.insert_fast_path.enabled", "default value for enable_insert_fast_path session setting; enables a specialized insert path", true, -).WithPublic() + settings.WithPublic) var experimentalAlterColumnTypeGeneralMode = settings.RegisterBoolSetting( settings.TenantWritable, @@ -449,7 +438,7 @@ var experimentalAlterColumnTypeGeneralMode = settings.RegisterBoolSetting( "default value for experimental_alter_column_type session setting; "+ "enables the use of ALTER COLUMN TYPE for general conversions", false, -).WithPublic() + settings.WithPublic) var clusterStatementTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -460,7 +449,7 @@ var clusterStatementTimeout = settings.RegisterDurationSetting( "there is no timeout", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) var clusterLockTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -472,7 +461,7 @@ var clusterLockTimeout = settings.RegisterDurationSetting( "perform a non-locking read on a key; if set to 0, there is no timeout", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) var clusterIdleInSessionTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -483,7 +472,7 @@ var clusterIdleInSessionTimeout = settings.RegisterDurationSetting( "if set to 0, there is no timeout", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) var clusterIdleInTransactionSessionTimeout = settings.RegisterDurationSetting( settings.TenantWritable, @@ -493,7 +482,7 @@ var clusterIdleInTransactionSessionTimeout = settings.RegisterDurationSetting( "session is terminated; if set to 0, there is no timeout", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // TODO(rytaft): remove this once unique without index constraints are fully // supported. @@ -503,7 +492,7 @@ var experimentalUniqueWithoutIndexConstraintsMode = settings.RegisterBoolSetting "default value for experimental_enable_unique_without_index_constraints session setting;"+ "disables unique without index constraints by default", false, -).WithPublic() + settings.WithPublic) var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting( settings.TenantWritable, @@ -517,25 +506,25 @@ var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting( int64(sessiondatapb.UseNewSchemaChangerUnsafe): "unsafe", int64(sessiondatapb.UseNewSchemaChangerUnsafeAlways): "unsafe_always", }, -).WithPublic() + settings.WithPublic) var stubCatalogTablesEnabledClusterValue = settings.RegisterBoolSetting( settings.TenantWritable, `sql.defaults.stub_catalog_tables.enabled`, `default value for stub_catalog_tables session setting`, true, -).WithPublic() + settings.WithPublic) -var experimentalComputedColumnRewrites = settings.RegisterValidatedStringSetting( +var experimentalComputedColumnRewrites = settings.RegisterStringSetting( settings.TenantWritable, "sql.defaults.experimental_computed_column_rewrites", "allows rewriting computed column expressions in CREATE TABLE and ALTER TABLE statements; "+ "the format is: '(before expression) -> (after expression) [, (before expression) -> (after expression) ...]'", "", /* defaultValue */ - func(_ *settings.Values, val string) error { + settings.WithValidateString(func(_ *settings.Values, val string) error { _, err := schemaexpr.ParseComputedColumnRewrites(val) return err - }, + }), ) var propagateInputOrdering = settings.RegisterBoolSetting( @@ -552,13 +541,9 @@ var settingWorkMemBytes = settings.RegisterByteSizeSetting( "sql.distsql.temp_storage.workmem", "maximum amount of memory in bytes a processor can use before falling back to temp storage", execinfra.DefaultMemoryLimit, /* 64MiB */ - func(v int64) error { - if v <= 1 { - return errors.Errorf("can only be set to a value greater than 1: %d", v) - } - return nil - }, -).WithPublic() + settings.PositiveInt, + settings.WithPublic, +) // ExperimentalDistSQLPlanningClusterSettingName is the name for the cluster // setting that controls experimentalDistSQLPlanningClusterMode below. @@ -576,7 +561,7 @@ var experimentalDistSQLPlanningClusterMode = settings.RegisterEnumSetting( int64(sessiondatapb.ExperimentalDistSQLPlanningOff): "off", int64(sessiondatapb.ExperimentalDistSQLPlanningOn): "on", }, -).WithPublic() + settings.WithPublic) // VectorizeClusterSettingName is the name for the cluster setting that controls // the VectorizeClusterMode below. @@ -596,7 +581,7 @@ var VectorizeClusterMode = settings.RegisterEnumSetting( } return m }(), -).WithPublic() + settings.WithPublic) // DistSQLClusterExecMode controls the cluster default for when DistSQL is used. var DistSQLClusterExecMode = settings.RegisterEnumSetting( @@ -610,7 +595,7 @@ var DistSQLClusterExecMode = settings.RegisterEnumSetting( int64(sessiondatapb.DistSQLOn): "on", int64(sessiondatapb.DistSQLAlways): "always", }, -).WithPublic() + settings.WithPublic) // SerialNormalizationMode controls how the SERIAL type is interpreted in table // definitions. @@ -626,14 +611,14 @@ var SerialNormalizationMode = settings.RegisterEnumSetting( int64(sessiondatapb.SerialUsesSQLSequences): "sql_sequence", int64(sessiondatapb.SerialUsesCachedSQLSequences): "sql_sequence_cached", }, -).WithPublic() + settings.WithPublic) var disallowFullTableScans = settings.RegisterBoolSetting( settings.TenantWritable, `sql.defaults.disallow_full_table_scans.enabled`, "setting to true rejects queries that have planned a full table scan", false, -).WithPublic() + settings.WithPublic) // intervalStyle controls intervals representation. var intervalStyle = settings.RegisterEnumSetting( @@ -648,7 +633,7 @@ var intervalStyle = settings.RegisterEnumSetting( } return ret }(), -).WithPublic() + settings.WithPublic) var dateStyleEnumMap = map[int64]string{ 0: "ISO, MDY", @@ -663,7 +648,7 @@ var dateStyle = settings.RegisterEnumSetting( "default value for DateStyle session setting", pgdate.DefaultDateStyle().SQLString(), dateStyleEnumMap, -).WithPublic() + settings.WithPublic) var txnRowsWrittenLog = settings.RegisterIntSetting( settings.TenantWritable, @@ -673,7 +658,7 @@ var txnRowsWrittenLog = settings.RegisterIntSetting( "SQL_INTERNAL_PERF for internal transactions); use 0 to disable", 0, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) var txnRowsWrittenErr = settings.RegisterIntSetting( settings.TenantWritable, @@ -683,7 +668,7 @@ var txnRowsWrittenErr = settings.RegisterIntSetting( "event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable", 0, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) var txnRowsReadLog = settings.RegisterIntSetting( settings.TenantWritable, @@ -693,7 +678,7 @@ var txnRowsReadLog = settings.RegisterIntSetting( "SQL_INTERNAL_PERF for internal transactions); use 0 to disable", 0, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) var txnRowsReadErr = settings.RegisterIntSetting( settings.TenantWritable, @@ -703,7 +688,7 @@ var txnRowsReadErr = settings.RegisterIntSetting( "event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable", 0, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // This is a float setting (rather than an int setting) because the optimizer // uses floating point for calculating row estimates. @@ -714,14 +699,14 @@ var largeFullScanRows = settings.RegisterFloatSetting( "the maximum table size allowed for a full scan when disallow_full_table_scans "+ "is set to true", 1000.0, -).WithPublic() + settings.WithPublic) var costScansWithDefaultColSize = settings.RegisterBoolSetting( settings.TenantWritable, `sql.defaults.cost_scans_with_default_col_size.enabled`, "setting to true uses the same size for all columns to compute scan cost", false, -).WithPublic() + settings.WithPublic) var enableSuperRegions = settings.RegisterBoolSetting( settings.TenantWritable, @@ -729,7 +714,7 @@ var enableSuperRegions = settings.RegisterBoolSetting( "default value for enable_super_regions; "+ "allows for the usage of super regions", false, -).WithPublic() + settings.WithPublic) var overrideAlterPrimaryRegionInSuperRegion = settings.RegisterBoolSetting( settings.TenantWritable, @@ -738,7 +723,7 @@ var overrideAlterPrimaryRegionInSuperRegion = settings.RegisterBoolSetting( "allows for altering the primary region even if the primary region is a "+ "member of a super region", false, -).WithPublic() + settings.WithPublic) var errNoTransactionInProgress = pgerror.New(pgcode.NoActiveSQLTransaction, "there is no transaction in progress") var errTransactionInProgress = pgerror.New(pgcode.ActiveSQLTransaction, "there is already a transaction in progress") diff --git a/pkg/sql/export.go b/pkg/sql/export.go index c7e616b294f4..1c1d0e8ef7d3 100644 --- a/pkg/sql/export.go +++ b/pkg/sql/export.go @@ -99,7 +99,7 @@ var featureExportEnabled = settings.RegisterBoolSetting( "feature.export.enabled", "set to true to enable exports, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) // ConstructExport is part of the exec.Factory interface. func (ef *execFactory) ConstructExport( diff --git a/pkg/sql/idxusage/cluster_settings.go b/pkg/sql/idxusage/cluster_settings.go index 89c67668cd2b..c9b7b3b6f712 100644 --- a/pkg/sql/idxusage/cluster_settings.go +++ b/pkg/sql/idxusage/cluster_settings.go @@ -16,4 +16,4 @@ import "github.com/cockroachdb/cockroach/pkg/settings" var Enable = settings.RegisterBoolSetting( settings.TenantWritable, "sql.metrics.index_usage_stats.enabled", "collect per index usage statistics", true, /* defaultValue */ -).WithPublic() + settings.WithPublic) diff --git a/pkg/sql/importer/export_base.go b/pkg/sql/importer/export_base.go index fbf3518512da..3c9a8dd4033e 100644 --- a/pkg/sql/importer/export_base.go +++ b/pkg/sql/importer/export_base.go @@ -13,7 +13,6 @@ package importer import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/mon" - "github.com/cockroachdb/errors" ) // eventMemoryMultipier is the multiplier for the amount of memory needed to @@ -25,12 +24,7 @@ var eventMemoryMultipier = settings.RegisterFloatSetting( "export.event_memory_multiplier", "the amount of memory required to export a datum is multiplied by this factor", 3, - func(v float64) error { - if v < 1 { - return errors.New("value must be at least 1") - } - return nil - }, + settings.NonNegativeFloatWithMinimum(1), ) // ExportTestingKnobs contains testing knobs for Export. diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index c34fbff4c62b..ccc6d9dbfc97 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -209,7 +209,7 @@ var featureImportEnabled = settings.RegisterBoolSetting( "feature.import.enabled", "set to true to enable imports, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) func validateFormatOptions( format string, specified map[string]string, formatAllowed map[string]struct{}, diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index b830381f3b0b..af04bc6801bd 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -57,12 +57,7 @@ var collectTxnStatsSampleRate = settings.RegisterFloatSetting( "sql.txn_stats.sample_rate", "the probability that a given transaction will collect execution statistics (displayed in the DB Console)", 0.01, - func(f float64) error { - if f < 0 || f > 1 { - return errors.New("value must be between 0 and 1 inclusive") - } - return nil - }, + settings.Fraction, ) // instrumentationHelper encapsulates the logic around extracting information diff --git a/pkg/sql/notice.go b/pkg/sql/notice.go index 2eebde2c21da..7454197fa534 100644 --- a/pkg/sql/notice.go +++ b/pkg/sql/notice.go @@ -26,7 +26,7 @@ var NoticesEnabled = settings.RegisterBoolSetting( "sql.notices.enabled", "enable notices in the server/client protocol being sent", true, -).WithPublic() + settings.WithPublic) // noticeSender is a subset of RestrictedCommandResult which allows // sending notices. diff --git a/pkg/sql/opt/optbuilder/mutation_builder_unique.go b/pkg/sql/opt/optbuilder/mutation_builder_unique.go index f0aada453948..e8dc532e3642 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder_unique.go +++ b/pkg/sql/opt/optbuilder/mutation_builder_unique.go @@ -30,7 +30,7 @@ var UniquenessChecksForGenRandomUUIDClusterMode = settings.RegisterBoolSetting( "if enabled, uniqueness checks may be planned for mutations of UUID columns updated with"+ " gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability", false, -).WithPublic() + settings.WithPublic) // buildUniqueChecksForInsert builds uniqueness check queries for an insert. // These check queries are used to enforce UNIQUE WITHOUT INDEX constraints. diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index 03bdcdae5ecc..5f81263f518a 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -35,7 +35,7 @@ var multipleModificationsOfTableEnabled = settings.RegisterBoolSetting( "modified multiple times by a single statement (multiple INSERT subqueries without ON "+ "CONFLICT cannot cause corruption and are always allowed)", false, -).WithPublic() + settings.WithPublic) // windowAggregateFrame() returns a frame that any aggregate built as a window // can use. diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index dc98f8d8d002..63f938132e20 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -496,7 +496,7 @@ var AutoSelectPasswordAuth = settings.RegisterBoolSetting( "server.user_login.cert_password_method.auto_scram_promotion.enabled", "whether to automatically promote cert-password authentication to use SCRAM", true, -).WithPublic() + settings.WithPublic) // authAutoSelectPasswordProtocol is the AuthMethod constructor used // for HBA method "cert-password" when the SQL client does not provide diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go index 2dc942241ae9..f8a0f6c20afc 100644 --- a/pkg/sql/pgwire/hba_conf.go +++ b/pkg/sql/pgwire/hba_conf.go @@ -80,17 +80,14 @@ const serverHBAConfSetting = "server.host_based_authentication.configuration" // connAuthConf is the cluster setting that holds the HBA // configuration. -var connAuthConf = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - serverHBAConfSetting, - "host-based authentication configuration to use during connection authentication", - "", - checkHBASyntaxBeforeUpdatingSetting, - ) - s.SetVisibility(settings.Public) - return s -}() +var connAuthConf = settings.RegisterStringSetting( + settings.TenantWritable, + serverHBAConfSetting, + "host-based authentication configuration to use during connection authentication", + "", + settings.WithValidateString(checkHBASyntaxBeforeUpdatingSetting), + settings.WithPublic, +) // loadLocalHBAConfigUponRemoteSettingChange initializes the local // node's cache of the HBA configuration each time the cluster setting diff --git a/pkg/sql/pgwire/ident_map_conf.go b/pkg/sql/pgwire/ident_map_conf.go index c7c0e9ff1408..6a7e5adbb815 100644 --- a/pkg/sql/pgwire/ident_map_conf.go +++ b/pkg/sql/pgwire/ident_map_conf.go @@ -25,20 +25,18 @@ import ( const serverIdentityMapSetting = "server.identity_map.configuration" // ConnIdentityMapConf maps system-identities to database-usernames using the pg_ident.conf format. -var ConnIdentityMapConf = func() *settings.StringSetting { - s := settings.RegisterValidatedStringSetting( - settings.TenantWritable, - serverIdentityMapSetting, - "system-identity to database-username mappings", - "", - func(values *settings.Values, s string) error { - _, err := identmap.From(strings.NewReader(s)) - return err - }, - ) - s.SetVisibility(settings.Public) - return s -}() +var ConnIdentityMapConf = settings.RegisterStringSetting( + settings.TenantWritable, + serverIdentityMapSetting, + "system-identity to database-username mappings", + "", + settings.WithValidateString(func(values *settings.Values, s string) error { + _, err := identmap.From(strings.NewReader(s)) + return err + }, + ), + settings.WithPublic, +) // loadLocalIdentityMapUponRemoteSettingChange initializes the local // node's cache of the identity map configuration each time the cluster diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index dd251dd72457..2166e9411f82 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -62,12 +62,7 @@ var ReadBufferMaxMessageSizeClusterSetting = settings.RegisterByteSizeSetting( readBufferMaxMessageSizeClusterSettingName, "maximum buffer size to allow for ingesting sql statements. Connections must be restarted for this to take effect.", defaultMaxReadBufferMessageSize, - func(val int64) error { - if val < minReadBufferMessageSize { - return errors.Newf("buffer message size must be at least %s", humanize.Bytes(minReadBufferMessageSize)) - } - return nil - }, + settings.ByteSizeWithMinimum(minReadBufferMessageSize), ) // FormatCode represents a pgwire data format. diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 96b5e6ab174b..806cb78d1f85 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -74,19 +74,21 @@ var connResultsBufferSize = settings.RegisterByteSizeSetting( "Updating the setting only affects new connections. "+ "Setting to 0 disables any buffering.", 16<<10, // 16 KiB -).WithPublic() + settings.WithPublic) var logConnAuth = settings.RegisterBoolSetting( settings.TenantWritable, sql.ConnAuditingClusterSettingName, "if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes)", - false).WithPublic() + false, + settings.WithPublic) var logSessionAuth = settings.RegisterBoolSetting( settings.TenantWritable, sql.AuthAuditingClusterSettingName, "if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)", - false).WithPublic() + false, + settings.WithPublic) const ( // ErrSSLRequired is returned when a client attempts to connect to a diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 1ae0f6639955..b806b2f05932 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/errors" ) const ( @@ -51,21 +50,9 @@ var maxRowSizeLog = settings.RegisterByteSizeSetting( "write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF "+ "if the mutating statement was internal); use 0 to disable", kvserverbase.MaxCommandSizeDefault, - func(size int64) error { - if size != 0 && size < maxRowSizeFloor { - return errors.Newf( - "cannot set sql.guardrails.max_row_size_log to %v, must be 0 or >= %v", - size, maxRowSizeFloor, - ) - } else if size > maxRowSizeCeil { - return errors.Newf( - "cannot set sql.guardrails.max_row_size_log to %v, must be <= %v", - size, maxRowSizeCeil, - ) - } - return nil - }, -).WithPublic() + settings.IntInRangeOrZeroDisable(maxRowSizeFloor, maxRowSizeCeil), + settings.WithPublic, +) var maxRowSizeErr = settings.RegisterByteSizeSetting( settings.TenantWritable, @@ -73,21 +60,9 @@ var maxRowSizeErr = settings.RegisterByteSizeSetting( "maximum size of row (or column family if multiple column families are in use) that SQL can "+ "write to the database, above which an error is returned; use 0 to disable", 512<<20, /* 512 MiB */ - func(size int64) error { - if size != 0 && size < maxRowSizeFloor { - return errors.Newf( - "cannot set sql.guardrails.max_row_size_err to %v, must be 0 or >= %v", - size, maxRowSizeFloor, - ) - } else if size > maxRowSizeCeil { - return errors.Newf( - "cannot set sql.guardrails.max_row_size_err to %v, must be <= %v", - size, maxRowSizeCeil, - ) - } - return nil - }, -).WithPublic() + settings.IntInRangeOrZeroDisable(maxRowSizeFloor, maxRowSizeCeil), + settings.WithPublic, +) // RowHelper has the common methods for table row manipulations. type RowHelper struct { diff --git a/pkg/sql/schema_change_cluster_setting.go b/pkg/sql/schema_change_cluster_setting.go index f6112d6e7a31..d9022474e971 100644 --- a/pkg/sql/schema_change_cluster_setting.go +++ b/pkg/sql/schema_change_cluster_setting.go @@ -27,7 +27,7 @@ var featureSchemaChangeEnabled = settings.RegisterBoolSetting( "feature.schema_change.enabled", "set to true to enable schema changes, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, -).WithPublic() + settings.WithPublic) // checkSchemaChangeEnabled is a method that wraps the featureflag.CheckEnabled // method specifically for all features that are categorized as schema changes. diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go index 2f50450c11aa..7f87afffae6e 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go @@ -27,32 +27,30 @@ type schemaStatementControl map[string]bool // for the declarative schema changer. Users can specify statement // tags for each statement and a "!" symbol in front can have the opposite // effect to force enable fully unimplemented features. -var schemaChangerDisabledStatements = func() *settings.StringSetting { - return settings.RegisterValidatedStringSetting( - settings.TenantWritable, - "sql.schema.force_declarative_statements", - "allows force enabling / disabling declarative schema changer for specific statements", - "", - func(values *settings.Values, s string) error { - if s == "" { - return nil +var schemaChangerDisabledStatements = settings.RegisterStringSetting( + settings.TenantWritable, + "sql.schema.force_declarative_statements", + "allows force enabling / disabling declarative schema changer for specific statements", + "", + settings.WithValidateString(func(values *settings.Values, s string) error { + if s == "" { + return nil + } + // First split the string into individual tags. + tags := strings.Split(s, ",") + for _, tag := range tags { + tag = strings.ToUpper(strings.TrimSpace(tag)) + if len(tag) > 0 && (tag[0] == '+' || tag[0] == '!') { + tag = tag[1:] + } else { + return errors.Errorf("tag is not properly formatted, must start with '+' or '!' (%s)", tag) } - // First split the string into individual tags. - tags := strings.Split(s, ",") - for _, tag := range tags { - tag = strings.ToUpper(strings.TrimSpace(tag)) - if len(tag) > 0 && (tag[0] == '+' || tag[0] == '!') { - tag = tag[1:] - } else { - return errors.Errorf("tag is not properly formatted, must start with '+' or '!' (%s)", tag) - } - if _, ok := supportedStatementTags[tag]; !ok { - return errors.Errorf("statement tag %q is not controlled by the declarative schema changer", tag) - } + if _, ok := supportedStatementTags[tag]; !ok { + return errors.Errorf("statement tag %q is not controlled by the declarative schema changer", tag) } - return nil - }) -}() + } + return nil + })) // CheckStatementControl if a statement is forced to disabled or enabled. If a // statement is disabled then an not implemented error will be panicked. Otherwise, diff --git a/pkg/sql/sem/eval/settings.go b/pkg/sql/sem/eval/settings.go index a997e9ceeeb5..ae1a90610333 100644 --- a/pkg/sql/sem/eval/settings.go +++ b/pkg/sql/sem/eval/settings.go @@ -25,7 +25,7 @@ var experimentalBox2DClusterSetting = settings.RegisterBoolSetting( experimentalBox2DClusterSettingName, "enables the use of certain experimental box2d comparison operators", false, -).WithPublic() + settings.WithPublic) func checkExperimentalBox2DComparisonOperatorEnabled(s *cluster.Settings) error { if !experimentalBox2DClusterSetting.Get(&s.SV) { diff --git a/pkg/sql/sessioninit/cache.go b/pkg/sql/sessioninit/cache.go index 3dd974cdfa69..792bad9eae75 100644 --- a/pkg/sql/sessioninit/cache.go +++ b/pkg/sql/sessioninit/cache.go @@ -41,7 +41,7 @@ var CacheEnabled = settings.RegisterBoolSetting( "enables a cache used during authentication to avoid lookups to system tables "+ "when retrieving per-user authentication-related information", true, -).WithPublic() + settings.WithPublic) // Cache is a shared cache for hashed passwords and other information used // during user authentication and session initialization. diff --git a/pkg/sql/sql_activity_update_job.go b/pkg/sql/sql_activity_update_job.go index 1e82c4f4fbcb..0ccf6f4e102c 100644 --- a/pkg/sql/sql_activity_update_job.go +++ b/pkg/sql/sql_activity_update_job.go @@ -63,7 +63,7 @@ var sqlStatsActivityMaxPersistedRows = settings.RegisterIntSetting( " activity that will be persisted in the system tables", 200000, /* defaultValue*/ settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) const numberOfTopColumns = 6 diff --git a/pkg/sql/sqlinstance/instancestorage/instancestorage.go b/pkg/sql/sqlinstance/instancestorage/instancestorage.go index 52ae8a879c0e..ed8ae8b7d17a 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancestorage.go +++ b/pkg/sql/sqlinstance/instancestorage/instancestorage.go @@ -62,17 +62,9 @@ var PreallocatedCount = settings.RegisterIntSetting( "server.instance_id.preallocated_count", "number of preallocated instance IDs within the system.sql_instances table", 10, - func(v int64) error { - // If this is 0, the assignment op will block forever if there are no - // preallocated instance IDs. - if v < 1 { - return errors.Errorf("cannot be less than 1: %d", v) - } - if v > math.MaxInt32 { - return errors.Errorf("cannot be more than %d: %d", math.MaxInt32, v) - } - return nil - }, + // If this is 0, the assignment op will block forever if there are no + // preallocated instance IDs. + settings.IntInRange(1, math.MaxInt32), ) var errNoPreallocatedRows = errors.New("no preallocated rows") diff --git a/pkg/sql/sqlliveness/slstorage/slstorage.go b/pkg/sql/sqlliveness/slstorage/slstorage.go index a444c6eae6d5..7cc85ac0ce60 100644 --- a/pkg/sql/sqlliveness/slstorage/slstorage.go +++ b/pkg/sql/sqlliveness/slstorage/slstorage.go @@ -60,12 +60,7 @@ var GCJitter = settings.RegisterFloatSetting( "server.sqlliveness.gc_jitter", "jitter fraction on the duration between attempts to delete extant sessions that have expired", .15, - func(f float64) error { - if f < 0 || f > 1 { - return errors.Errorf("%f is not in [0, 1]", f) - } - return nil - }, + settings.Fraction, ) // CacheSize is the size of the entries to store in the cache. diff --git a/pkg/sql/sqlstats/cluster_settings.go b/pkg/sql/sqlstats/cluster_settings.go index be6db3e12dfb..c5de5c427485 100644 --- a/pkg/sql/sqlstats/cluster_settings.go +++ b/pkg/sql/sqlstats/cluster_settings.go @@ -20,7 +20,7 @@ import ( var StmtStatsEnable = settings.RegisterBoolSetting( settings.TenantWritable, "sql.metrics.statement_details.enabled", "collect per-statement query statistics", true, -).WithPublic() + settings.WithPublic) // TxnStatsNumStmtFingerprintIDsToRecord limits the number of statementFingerprintIDs stored in // transactions statistics for a single transaction. This defaults to 1000, and @@ -38,7 +38,7 @@ var TxnStatsNumStmtFingerprintIDsToRecord = settings.RegisterIntSetting( var TxnStatsEnable = settings.RegisterBoolSetting( settings.TenantWritable, "sql.metrics.transaction_details.enabled", "collect per-application transaction statistics", true, -).WithPublic() + settings.WithPublic) // StatsCollectionLatencyThreshold specifies the minimum amount of time // consumed by a SQL statement before it is collected for statistics reporting. @@ -48,7 +48,7 @@ var StatsCollectionLatencyThreshold = settings.RegisterDurationSetting( "minimum execution time to cause statement statistics to be collected. "+ "If configured, no transaction stats are collected.", 0, -).WithPublic() + settings.WithPublic) // DumpStmtStatsToLogBeforeReset specifies whether we dump the statements // statistics to logs before being reset. @@ -57,7 +57,7 @@ var DumpStmtStatsToLogBeforeReset = settings.RegisterBoolSetting( "sql.metrics.statement_details.dump_to_logs", "dump collected statement statistics to node logs when periodically cleared", false, -).WithPublic() + settings.WithPublic) // SampleLogicalPlans specifies whether we periodically sample the logical plan // for each fingerprint. @@ -66,7 +66,7 @@ var SampleLogicalPlans = settings.RegisterBoolSetting( "sql.metrics.statement_details.plan_collection.enabled", "periodically save a logical plan for each fingerprint", false, -).WithPublic() + settings.WithPublic) // LogicalPlanCollectionPeriod specifies the interval between collections of // logical plans for each fingerprint. @@ -76,7 +76,7 @@ var LogicalPlanCollectionPeriod = settings.RegisterDurationSetting( "the time until a new logical plan is collected", 5*time.Minute, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // MaxMemSQLStatsStmtFingerprints specifies the maximum of unique statement // fingerprints we store in memory. @@ -85,7 +85,7 @@ var MaxMemSQLStatsStmtFingerprints = settings.RegisterIntSetting( "sql.metrics.max_mem_stmt_fingerprints", "the maximum number of statement fingerprints stored in memory", 100000, -).WithPublic() + settings.WithPublic) // MaxMemSQLStatsTxnFingerprints specifies the maximum of unique transaction // fingerprints we store in memory. @@ -94,7 +94,7 @@ var MaxMemSQLStatsTxnFingerprints = settings.RegisterIntSetting( "sql.metrics.max_mem_txn_fingerprints", "the maximum number of transaction fingerprints stored in memory", 100000, -).WithPublic() + settings.WithPublic) // MaxMemReportedSQLStatsStmtFingerprints specifies the maximum of unique statement // fingerprints we store in memory. @@ -103,7 +103,7 @@ var MaxMemReportedSQLStatsStmtFingerprints = settings.RegisterIntSetting( "sql.metrics.max_mem_reported_stmt_fingerprints", "the maximum number of reported statement fingerprints stored in memory", 100000, -).WithPublic() + settings.WithPublic) // MaxMemReportedSQLStatsTxnFingerprints specifies the maximum of unique transaction // fingerprints we store in memory. @@ -112,7 +112,7 @@ var MaxMemReportedSQLStatsTxnFingerprints = settings.RegisterIntSetting( "sql.metrics.max_mem_reported_txn_fingerprints", "the maximum number of reported transaction fingerprints stored in memory", 100000, -).WithPublic() + settings.WithPublic) // MaxSQLStatsStmtFingerprintsPerExplicitTxn specifies the maximum of unique statement // fingerprints we store for each explicit transaction. @@ -158,7 +158,7 @@ var MaxSQLStatReset = settings.RegisterDurationSetting( "if not collected by telemetry reporter. It has a max value of 24H.", time.Hour*2, settings.NonNegativeDurationWithMaximum(time.Hour*24), -).WithPublic() + settings.WithPublic) // SampleIndexRecommendation specifies whether we generate an index recommendation // for each fingerprint ID. @@ -167,7 +167,7 @@ var SampleIndexRecommendation = settings.RegisterBoolSetting( "sql.metrics.statement_details.index_recommendation_collection.enabled", "generate an index recommendation for each fingerprint ID", true, -).WithPublic() + settings.WithPublic) // MaxMemReportedSampleIndexRecommendations specifies the maximum of unique index // recommendations info we store in memory. @@ -176,7 +176,7 @@ var MaxMemReportedSampleIndexRecommendations = settings.RegisterIntSetting( "sql.metrics.statement_details.max_mem_reported_idx_recommendations", "the maximum number of reported index recommendation info stored in memory", 5000, -).WithPublic() + settings.WithPublic) // GatewayNodeEnabled specifies whether we save the gateway node id for each fingerprint // during sql stats collection, otherwise the value will be set to 0. @@ -186,4 +186,4 @@ var GatewayNodeEnabled = settings.RegisterBoolSetting( "save the gateway node for each statement fingerprint. If false, the value will "+ "be stored as 0.", true, -).WithPublic() + settings.WithPublic) diff --git a/pkg/sql/sqlstats/insights/insights.go b/pkg/sql/sqlstats/insights/insights.go index c316ccc5b90b..fb61e284308f 100644 --- a/pkg/sql/sqlstats/insights/insights.go +++ b/pkg/sql/sqlstats/insights/insights.go @@ -31,7 +31,7 @@ var ExecutionInsightsCapacity = settings.RegisterIntSetting( "the size of the per-node store of execution insights", 1000, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // LatencyThreshold configures the execution time beyond which a statement is // considered slow. A LatencyThreshold of 0 (the default) disables this @@ -41,7 +41,7 @@ var LatencyThreshold = settings.RegisterDurationSetting( "sql.insights.latency_threshold", "amount of time after which an executing statement is considered slow. Use 0 to disable.", 100*time.Millisecond, -).WithPublic() + settings.WithPublic) // AnomalyDetectionEnabled turns on a per-fingerprint heuristic-based // algorithm for marking statements as slow, attempting to capture elevated @@ -52,7 +52,7 @@ var AnomalyDetectionEnabled = settings.RegisterBoolSetting( "sql.insights.anomaly_detection.enabled", "enable per-fingerprint latency recording and anomaly detection", true, -).WithPublic() + settings.WithPublic) // AnomalyDetectionLatencyThreshold sets the bar above which we consider // statement executions worth inspecting for slow execution. A statement's @@ -66,7 +66,7 @@ var AnomalyDetectionLatencyThreshold = settings.RegisterDurationSetting( "statements must surpass this threshold to trigger anomaly detection and identification", 50*time.Millisecond, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // AnomalyDetectionMemoryLimit restricts the overall memory available for // tracking per-statement execution latencies. When changing this setting, keep @@ -77,7 +77,7 @@ var AnomalyDetectionMemoryLimit = settings.RegisterByteSizeSetting( "sql.insights.anomaly_detection.memory_limit", "the maximum amount of memory allowed for tracking statement latencies", 1024*1024, -).WithPublic() + settings.WithPublic) // HighRetryCountThreshold sets the number of times a slow statement must have // been retried to be marked as having a high retry count. @@ -87,7 +87,7 @@ var HighRetryCountThreshold = settings.RegisterIntSetting( "the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem", 10, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // Metrics holds running measurements of various insights-related runtime stats. type Metrics struct { diff --git a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go index 273b3b209318..eac4a5cee64f 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go +++ b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go @@ -27,7 +27,7 @@ var SQLStatsFlushInterval = settings.RegisterDurationSetting( "this value must be less than or equal to 1 hour", time.Minute*10, settings.NonNegativeDurationWithMaximum(time.Hour*24), -).WithPublic() + settings.WithPublic) // MinimumInterval is the cluster setting that controls the minimum interval // between each flush operation. If flush operations get triggered faster @@ -62,7 +62,7 @@ var SQLStatsFlushEnabled = settings.RegisterBoolSetting( "sql.stats.flush.enabled", "if set, SQL execution statistics are periodically flushed to disk", true, /* defaultValue */ -).WithPublic() + settings.WithPublic) // SQLStatsFlushJitter specifies the jitter fraction on the interval between // attempts to flush SQL Stats. @@ -75,12 +75,7 @@ var SQLStatsFlushJitter = settings.RegisterFloatSetting( "sql.stats.flush.jitter", "jitter fraction on the duration between sql stats flushes", 0.15, - func(f float64) error { - if f < 0 || f > 1 { - return errors.Newf("%f is not in [0, 1]", f) - } - return nil - }, + settings.Fraction, ) // SQLStatsMaxPersistedRows specifies maximum number of rows that will be @@ -91,22 +86,23 @@ var SQLStatsMaxPersistedRows = settings.RegisterIntSetting( "maximum number of rows of statement and transaction statistics that "+ "will be persisted in the system tables before compaction begins", 1000000, /* defaultValue */ -).WithPublic() + settings.WithPublic) // SQLStatsCleanupRecurrence is the cron-tab string specifying the recurrence // for SQL Stats cleanup job. -var SQLStatsCleanupRecurrence = settings.RegisterValidatedStringSetting( +var SQLStatsCleanupRecurrence = settings.RegisterStringSetting( settings.TenantWritable, "sql.stats.cleanup.recurrence", "cron-tab recurrence for SQL Stats cleanup job", "@hourly", /* defaultValue */ - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { if _, err := cron.ParseStandard(s); err != nil { return errors.Wrap(err, "invalid cron expression") } return nil - }, -).WithPublic() + }), + settings.WithPublic, +) // SQLStatsAggregationInterval is the cluster setting that controls the aggregation // interval for stats when we flush to disk. diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index aad2ae461d22..3e912224d1da 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -39,8 +39,6 @@ go_library( "//pkg/sql/memsize", "//pkg/sql/opt/cat", "//pkg/sql/parser", - "//pkg/sql/pgwire/pgcode", - "//pkg/sql/pgwire/pgerror", "//pkg/sql/rowenc", "//pkg/sql/rowenc/keyside", "//pkg/sql/sem/eval", diff --git a/pkg/sql/stats/automatic_stats.go b/pkg/sql/stats/automatic_stats.go index 8fe7fe703cef..3cb0214f5332 100644 --- a/pkg/sql/stats/automatic_stats.go +++ b/pkg/sql/stats/automatic_stats.go @@ -27,8 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/isql" - "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/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -45,7 +43,7 @@ var AutomaticStatisticsClusterMode = settings.RegisterBoolSetting( catpb.AutoStatsEnabledSettingName, "automatic statistics collection mode", true, -).WithPublic() + settings.WithPublic) // UseStatisticsOnSystemTables controls the cluster setting for enabling // statistics usage by the optimizer for planning queries involving system @@ -55,7 +53,7 @@ var UseStatisticsOnSystemTables = settings.RegisterBoolSetting( catpb.UseStatsOnSystemTables, "when true, enables use of statistics on system tables by the query optimizer", true, -).WithPublic() + settings.WithPublic) // AutomaticStatisticsOnSystemTables controls the cluster setting for enabling // automatic statistics collection on system tables. Auto stats must be enabled @@ -66,7 +64,7 @@ var AutomaticStatisticsOnSystemTables = settings.RegisterBoolSetting( catpb.AutoStatsOnSystemTables, "when true, enables automatic collection of statistics on system tables", true, -).WithPublic() + settings.WithPublic) // MultiColumnStatisticsClusterMode controls the cluster setting for enabling // automatic collection of multi-column statistics. @@ -75,7 +73,7 @@ var MultiColumnStatisticsClusterMode = settings.RegisterBoolSetting( "sql.stats.multi_column_collection.enabled", "multi-column statistics collection mode", true, -).WithPublic() + settings.WithPublic) // AutomaticStatisticsMaxIdleTime controls the maximum fraction of time that // the sampler processors will be idle when scanning large tables for automatic @@ -86,45 +84,33 @@ var AutomaticStatisticsMaxIdleTime = settings.RegisterFloatSetting( "sql.stats.automatic_collection.max_fraction_idle", "maximum fraction of time that automatic statistics sampler processors are idle", 0.9, - func(val float64) error { - if val < 0 || val >= 1 { - return pgerror.Newf(pgcode.InvalidParameterValue, - "sql.stats.automatic_collection.max_fraction_idle must be >= 0 and < 1 but found: %v", val) - } - return nil - }, + settings.FractionUpperExclusive, ) // AutomaticStatisticsFractionStaleRows controls the cluster setting for // the target fraction of rows in a table that should be stale before // statistics on that table are refreshed, in addition to the constant value // AutomaticStatisticsMinStaleRows. -var AutomaticStatisticsFractionStaleRows = func() *settings.FloatSetting { - s := settings.RegisterFloatSetting( - settings.TenantWritable, - catpb.AutoStatsFractionStaleSettingName, - "target fraction of stale rows per table that will trigger a statistics refresh", - 0.2, - settings.NonNegativeFloat, - ) - s.SetVisibility(settings.Public) - return s -}() +var AutomaticStatisticsFractionStaleRows = settings.RegisterFloatSetting( + settings.TenantWritable, + catpb.AutoStatsFractionStaleSettingName, + "target fraction of stale rows per table that will trigger a statistics refresh", + 0.2, + settings.NonNegativeFloat, + settings.WithPublic, +) // AutomaticStatisticsMinStaleRows controls the cluster setting for the target // number of rows that should be updated before a table is refreshed, in // addition to the fraction AutomaticStatisticsFractionStaleRows. -var AutomaticStatisticsMinStaleRows = func() *settings.IntSetting { - s := settings.RegisterIntSetting( - settings.TenantWritable, - catpb.AutoStatsMinStaleSettingName, - "target minimum number of stale rows per table that will trigger a statistics refresh", - 500, - settings.NonNegativeInt, - ) - s.SetVisibility(settings.Public) - return s -}() +var AutomaticStatisticsMinStaleRows = settings.RegisterIntSetting( + settings.TenantWritable, + catpb.AutoStatsMinStaleSettingName, + "target minimum number of stale rows per table that will trigger a statistics refresh", + 500, + settings.NonNegativeInt, + settings.WithPublic, +) // statsGarbageCollectionInterval controls the interval between running an // internal query to delete stats for dropped tables. diff --git a/pkg/sql/stats/delete_stats.go b/pkg/sql/stats/delete_stats.go index da138bcdf774..1792bf5e1541 100644 --- a/pkg/sql/stats/delete_stats.go +++ b/pkg/sql/stats/delete_stats.go @@ -44,7 +44,7 @@ var TableStatisticsRetentionPeriod = settings.RegisterDurationSetting( "sql.stats.non_default_columns.min_retention_period", "minimum retention period for table statistics collected on non-default columns", defaultKeepTime, -).WithPublic() + settings.WithPublic) // DeleteOldStatsForColumns deletes old statistics from the // system.table_statistics table. For the given tableID and columnIDs, diff --git a/pkg/sql/stats/forecast.go b/pkg/sql/stats/forecast.go index 2dd45d5164d5..958bbead0f89 100644 --- a/pkg/sql/stats/forecast.go +++ b/pkg/sql/stats/forecast.go @@ -38,7 +38,7 @@ var UseStatisticsForecasts = settings.RegisterBoolSetting( "sql.stats.forecasts.enabled", "when true, enables generation of statistics forecasts by default for all tables", true, -).WithPublic() + settings.WithPublic) // minObservationsForForecast is the minimum number of observed statistics // required to produce a statistics forecast. Forecasts based on 1 or 2 diff --git a/pkg/sql/stats/histogram.go b/pkg/sql/stats/histogram.go index 220ba2b84ab4..180b75d5464e 100644 --- a/pkg/sql/stats/histogram.go +++ b/pkg/sql/stats/histogram.go @@ -34,7 +34,7 @@ var DefaultHistogramBuckets = settings.RegisterIntSetting( "maximum number of histogram buckets to build during table statistics collection", 200, settings.NonNegativeIntWithMaximum(math.MaxUint32), -).WithPublic() + settings.WithPublic) // HistogramClusterMode controls the cluster setting for enabling // histogram collection. @@ -43,7 +43,7 @@ var HistogramClusterMode = settings.RegisterBoolSetting( "sql.stats.histogram_collection.enabled", "histogram collection mode", true, -).WithPublic() + settings.WithPublic) // HistogramVersion identifies histogram versions. type HistogramVersion uint32 diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics.go b/pkg/sql/stmtdiagnostics/statement_diagnostics.go index 1c5683bf1800..53fa814f66eb 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics.go @@ -45,12 +45,7 @@ var bundleChunkSize = settings.RegisterByteSizeSetting( "sql.stmt_diagnostics.bundle_chunk_size", "chunk size for statement diagnostic bundles", 1024*1024, - func(val int64) error { - if val < 16 { - return errors.Errorf("chunk size must be at least 16 bytes") - } - return nil - }, + settings.ByteSizeWithMinimum(16), ) // collectUntilExpiration enables continuous collection of statement bundles for diff --git a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel index f99dc1ad8202..29ba669ec7b7 100644 --- a/pkg/sql/storageparam/tablestorageparam/BUILD.bazel +++ b/pkg/sql/storageparam/tablestorageparam/BUILD.bazel @@ -6,7 +6,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam", visibility = ["//visibility:public"], deps = [ - "//pkg/settings", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/paramparse", diff --git a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go index be7f32bd8ba7..8c3e6647f2f7 100644 --- a/pkg/sql/storageparam/tablestorageparam/table_storage_param.go +++ b/pkg/sql/storageparam/tablestorageparam/table_storage_param.go @@ -17,7 +17,6 @@ import ( "math" "strings" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" @@ -459,11 +458,21 @@ var tableParams = map[string]tableParam{ onReset: autoStatsTableSettingResetFunc, }, catpb.AutoStatsMinStaleTableSettingName: { - onSet: autoStatsMinStaleRowsSettingFunc(settings.NonNegativeInt), + onSet: autoStatsMinStaleRowsSettingFunc(func(intVal int64) error { + if intVal < 0 { + return errors.New("cannot be negative") + } + return nil + }), onReset: autoStatsTableSettingResetFunc, }, catpb.AutoStatsFractionStaleTableSettingName: { - onSet: autoStatsFractionStaleRowsSettingFunc(settings.NonNegativeFloat), + onSet: autoStatsFractionStaleRowsSettingFunc(func(floatVal float64) error { + if floatVal < 0 { + return errors.New("cannot be negative") + } + return nil + }), onReset: autoStatsTableSettingResetFunc, }, `sql_stats_forecasts_enabled`: { @@ -490,7 +499,7 @@ var tableParams = map[string]tableParam{ if err != nil { return err } - if err = settings.NonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { + if err := nonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { return errors.Wrapf(err, "invalid integer value for %s", key) } uint32Val := uint32(intVal) @@ -510,7 +519,7 @@ var tableParams = map[string]tableParam{ if err != nil { return err } - if err = settings.NonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { + if err = nonNegativeIntWithMaximum(math.MaxUint32)(intVal); err != nil { return errors.Wrapf(err, "invalid integer value for %s", key) } uint32Val := uint32(intVal) @@ -538,6 +547,18 @@ var tableParams = map[string]tableParam{ }, } +func nonNegativeIntWithMaximum(max int64) func(int64) error { + return func(intVal int64) error { + if intVal < 0 { + return errors.Newf("cannot be set to a negative interger: %d", intVal) + } + if intVal > max { + return errors.Newf("cannot be set to an integer larger than %d", max) + } + return nil + } +} + func init() { for _, param := range []string{ `toast_tuple_target`, diff --git a/pkg/sql/telemetry_logging.go b/pkg/sql/telemetry_logging.go index aea08a89cd1f..0b0e9185520b 100644 --- a/pkg/sql/telemetry_logging.go +++ b/pkg/sql/telemetry_logging.go @@ -40,7 +40,7 @@ var telemetryInternalQueriesEnabled = settings.RegisterBoolSetting( "sql.telemetry.query_sampling.internal.enabled", "when set to true, internal queries will be sampled in telemetry logging", false, -).WithPublic() + settings.WithPublic) var telemetryInternalConsoleQueriesEnabled = settings.RegisterBoolSetting( settings.TenantWritable, diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index f609453eec59..d69e156cb904 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -51,7 +51,7 @@ var TempObjectCleanupInterval = settings.RegisterDurationSetting( "sql.temp_object_cleaner.cleanup_interval", "how often to clean up orphaned temporary objects", 30*time.Minute, -).WithPublic() + settings.WithPublic) // TempObjectWaitInterval is a ClusterSetting controlling how long // after a creation a temporary object will be cleaned up. @@ -60,7 +60,7 @@ var TempObjectWaitInterval = settings.RegisterDurationSetting( "sql.temp_object_cleaner.wait_interval", "how long after creation a temporary object will be cleaned up", 30*time.Minute, -).WithPublic() + settings.WithPublic) var ( temporaryObjectCleanerActiveCleanersMetric = metric.Metadata{ diff --git a/pkg/sql/tenant_accessors.go b/pkg/sql/tenant_accessors.go index 29ca183265c8..910f5255ba30 100644 --- a/pkg/sql/tenant_accessors.go +++ b/pkg/sql/tenant_accessors.go @@ -196,18 +196,15 @@ func GetExtendedTenantInfo( return res, nil } -var defaultTenantConfigTemplate = func() *settings.StringSetting { - s := settings.RegisterStringSetting( - settings.SystemOnly, - "sql.create_tenant.default_template", - "tenant to use as configuration template when LIKE is not specified in CREATE VIRTUAL CLUSTER", - // We use the empty string so that no template is used by default - // (i.e. empty proto, no setting overrides). - "", - ) - s.SetReportable(true) - return s -}() +var defaultTenantConfigTemplate = settings.RegisterStringSetting( + settings.SystemOnly, + "sql.create_tenant.default_template", + "tenant to use as configuration template when LIKE is not specified in CREATE VIRTUAL CLUSTER", + // We use the empty string so that no template is used by default + // (i.e. empty proto, no setting overrides). + "", + settings.WithReportable(true), +) // GetTenantTemplate loads the tenant template corresponding to the // provided origin tenant. If info is nil, likeTenantID is zero and diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index 8153940a77f9..27b5afa84bc9 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -47,28 +47,28 @@ var ( "default amount of rows to select in a single query during a TTL job", 500, settings.PositiveInt, - ).WithPublic() + settings.WithPublic) defaultDeleteBatchSize = settings.RegisterIntSetting( settings.TenantWritable, "sql.ttl.default_delete_batch_size", "default amount of rows to delete in a single query during a TTL job", 100, settings.PositiveInt, - ).WithPublic() + settings.WithPublic) defaultDeleteRateLimit = settings.RegisterIntSetting( settings.TenantWritable, "sql.ttl.default_delete_rate_limit", "default delete rate limit for all TTL jobs. Use 0 to signify no rate limit.", 0, settings.NonNegativeInt, - ).WithPublic() + settings.WithPublic) jobEnabled = settings.RegisterBoolSetting( settings.TenantWritable, "sql.ttl.job.enabled", "whether the TTL job is enabled", true, - ).WithPublic() + settings.WithPublic) ) type rowLevelTTLResumer struct { diff --git a/pkg/sql/txn_fingerprint_id_cache.go b/pkg/sql/txn_fingerprint_id_cache.go index a2d0ab821d31..7d7751a0fbb0 100644 --- a/pkg/sql/txn_fingerprint_id_cache.go +++ b/pkg/sql/txn_fingerprint_id_cache.go @@ -30,7 +30,7 @@ var TxnFingerprintIDCacheCapacity = settings.RegisterIntSetting( "the maximum number of txn fingerprint IDs stored", 100, settings.NonNegativeInt, -).WithPublic() + settings.WithPublic) // TxnFingerprintIDCache is a thread-safe cache tracking transaction // fingerprint IDs at the session level. diff --git a/pkg/sql/user.go b/pkg/sql/user.go index 8dcde8faa0d5..f8bc9b81b44f 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -447,7 +447,7 @@ var userLoginTimeout = settings.RegisterDurationSetting( "timeout after which client authentication times out if some system range is unavailable (0 = no timeout)", 10*time.Second, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // GetAllRoles returns a "set" (map) of Roles -> true. func (p *planner) GetAllRoles(ctx context.Context) (map[username.SQLUsername]bool, error) { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index fb937ce9d855..bdc0b8e5f9f2 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -72,7 +72,7 @@ var MaxSyncDuration = settings.RegisterDurationSetting( "maximum duration for disk operations; any operations that take longer"+ " than this setting trigger a warning log entry or process crash", maxSyncDurationDefault, -).WithPublic() + settings.WithPublic) // MaxSyncDurationFatalOnExceeded governs whether disk stalls longer than // MaxSyncDuration fatal the Cockroach process. Defaults to true. @@ -81,7 +81,7 @@ var MaxSyncDurationFatalOnExceeded = settings.RegisterBoolSetting( "storage.max_sync_duration.fatal.enabled", "if true, fatal the process when a disk operation exceeds storage.max_sync_duration", true, -).WithPublic() + settings.WithPublic) // ValueBlocksEnabled controls whether older versions of MVCC keys in the same // sstable will have their values written to value blocks. This only affects @@ -93,7 +93,8 @@ var ValueBlocksEnabled = settings.RegisterBoolSetting( "storage.value_blocks.enabled", "set to true to enable writing of value blocks in sstables", util.ConstantWithMetamorphicTestBool( - "storage.value_blocks.enabled", true)).WithPublic() + "storage.value_blocks.enabled", true), + settings.WithPublic) // IngestAsFlushable controls whether ingested sstables that overlap the // memtable may be lazily ingested: written to the WAL and enqueued in the list diff --git a/pkg/ts/db.go b/pkg/ts/db.go index dd5ceed48a7b..a111b9111ecc 100644 --- a/pkg/ts/db.go +++ b/pkg/ts/db.go @@ -46,7 +46,7 @@ var TimeseriesStorageEnabled = settings.RegisterBoolSetting( "if set, periodic timeseries data is stored within the cluster; disabling is not recommended "+ "unless you are storing the data elsewhere", true, -).WithPublic() + settings.WithPublic) // Resolution10sStorageTTL defines the maximum age of data that will be retained // at he 10 second resolution. Data older than this is subject to being "rolled @@ -57,7 +57,7 @@ var Resolution10sStorageTTL = settings.RegisterDurationSetting( "the maximum age of time series data stored at the 10 second resolution. Data older than this "+ "is subject to rollup and deletion.", resolution10sDefaultRollupThreshold, -).WithPublic() + settings.WithPublic) // Resolution30mStorageTTL defines the maximum age of data that will be // retained at he 30 minute resolution. Data older than this is subject to @@ -68,7 +68,7 @@ var Resolution30mStorageTTL = settings.RegisterDurationSetting( "the maximum age of time series data stored at the 30 minute resolution. Data older than this "+ "is subject to deletion.", resolution30mDefaultPruneThreshold, -).WithPublic() + settings.WithPublic) // DB provides Cockroach's Time Series API. type DB struct { diff --git a/pkg/ui/ui.go b/pkg/ui/ui.go index 790f6c073b3c..bdb063a53864 100644 --- a/pkg/ui/ui.go +++ b/pkg/ui/ui.go @@ -51,7 +51,7 @@ var _ = settings.RegisterEnumSetting( // See pkg/ui/workspaces/cluster-ui/webpack.config.js // and pkg/ui/workspaces/db-console/webpack.config.js. }, -).WithPublic() + settings.WithPublic) // Assets is used for embedded JS assets required for UI. // In case the binary is built without UI, it provides single index.html file with diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go index 55f15624fceb..3e2b66b0457d 100644 --- a/pkg/util/admission/io_load_listener.go +++ b/pkg/util/admission/io_load_listener.go @@ -65,7 +65,8 @@ var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting( "admission.disk_bandwidth_tokens.elastic.enabled", "when true, and provisioned bandwidth for the disk corresponding to a store is configured, "+ "tokens for elastic work will be limited if disk bandwidth becomes a bottleneck", - true).WithPublic() + true, + settings.WithPublic) // L0FileCountOverloadThreshold sets a file count threshold that signals an // overloaded store. diff --git a/pkg/util/admission/scheduler_latency_listener.go b/pkg/util/admission/scheduler_latency_listener.go index 990c76b3c5af..983a52688bec 100644 --- a/pkg/util/admission/scheduler_latency_listener.go +++ b/pkg/util/admission/scheduler_latency_listener.go @@ -12,7 +12,6 @@ package admission import ( "context" - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/settings" @@ -213,12 +212,7 @@ var ( // cluster settings to control how elastic CPU % is adjusted "admission.elastic_cpu.max_utilization", "sets the ceiling on per-node elastic work CPU % utilization", 0.75, // 75% - func(f float64) error { - if f < 0.05 || f > 1.0 { - return fmt.Errorf("expected max_utilization to be between [0.0, 1.0], got %0.2f", f) - } - return nil - }, + settings.FloatInRange(0.05, 1.0), ) elasticCPUMinUtilization = settings.RegisterFloatSetting( @@ -226,12 +220,7 @@ var ( // cluster settings to control how elastic CPU % is adjusted "admission.elastic_cpu.min_utilization", "sets the floor on per-node elastic work CPU % utilization", 0.05, // 5% - func(f float64) error { - if f < 0.01 || f > 1.0 { - return fmt.Errorf("expected min_utilization to be between [0.01, 1.0], got %0.2f", f) - } - return nil - }, + settings.FloatInRange(0.05, 1.0), ) elasticCPUInactivePoint = settings.RegisterFloatSetting( @@ -239,12 +228,7 @@ var ( // cluster settings to control how elastic CPU % is adjusted "admission.elastic_cpu.inactive_point", "the point between {min,max}_utilization the CPU % decreases to when there's no elastic work", 0.10, // 10% of the way between {min,max} utilization -- 12% if [min,max] = [5%,75%] - func(f float64) error { - if f < 0.0 || f > 1.0 { - return fmt.Errorf("expected inactive_point to be between [0.0, 1.0], got %0.2f", f) - } - return nil - }, + settings.Fraction, ) elasticCPUAdjustmentDeltaPerSecond = settings.RegisterFloatSetting( @@ -252,12 +236,7 @@ var ( // cluster settings to control how elastic CPU % is adjusted "admission.elastic_cpu.adjustment_delta_per_second", "sets the per-second % adjustment used when when adapting elastic work CPU %s", 0.001, // 0.1%, takes 10s to add 1% to elastic CPU limit - func(f float64) error { - if f < 0.0001 || f > 1.0 { - return fmt.Errorf("expected additive_delta_per_second to be between [0.0001, 1.0], got %0.2f", f) - } - return nil - }, + settings.FloatInRange(0.0001, 1.0), ) elasticCPUMultiplicativeFactorOnDecrease = settings.RegisterFloatSetting( @@ -279,12 +258,7 @@ var ( // cluster settings to control how elastic CPU % is adjusted "admission.elastic_cpu.scheduler_latency_target", "sets the p99 scheduling latency the elastic CPU controller aims for", time.Millisecond, - func(duration time.Duration) error { - if duration < 50*time.Microsecond || duration > time.Second { - return fmt.Errorf("expected scheduler_latency_target to be between [50μs, 1s], got %s", duration) - } - return nil - }, + settings.DurationInRange(50*time.Microsecond, time.Second), ) ) diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index d5526b11f08a..40be2cc05f13 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -50,7 +50,8 @@ var KVAdmissionControlEnabled = settings.RegisterBoolSetting( settings.SystemOnly, "admission.kv.enabled", "when true, work performed by the KV layer is subject to admission control", - true).WithPublic() + true, + settings.WithPublic) // KVBulkOnlyAdmissionControlEnabled controls whether user (normal and above // priority) work is subject to admission control. If it is set to true, then @@ -73,7 +74,8 @@ var SQLKVResponseAdmissionControlEnabled = settings.RegisterBoolSetting( "admission.sql_kv_response.enabled", "when true, work performed by the SQL layer when receiving a KV response is subject to "+ "admission control", - true).WithPublic() + true, + settings.WithPublic) // SQLSQLResponseAdmissionControlEnabled controls whether response processing // in SQL, for DistSQL requests, is enabled. @@ -82,7 +84,8 @@ var SQLSQLResponseAdmissionControlEnabled = settings.RegisterBoolSetting( "admission.sql_sql_response.enabled", "when true, work performed by the SQL layer when receiving a DistSQL response is subject "+ "to admission control", - true).WithPublic() + true, + settings.WithPublic) var admissionControlEnabledSettings = [numWorkKinds]*settings.BoolSetting{ KVWork: KVAdmissionControlEnabled, @@ -97,7 +100,8 @@ var KVTenantWeightsEnabled = settings.RegisterBoolSetting( settings.SystemOnly, "admission.kv.tenant_weights.enabled", "when true, tenant weights are enabled for KV admission control", - false).WithPublic() + false, + settings.WithPublic) // KVStoresTenantWeightsEnabled controls whether tenant weights are enabled // for KV-stores admission control. This setting has no effect if @@ -106,7 +110,8 @@ var KVStoresTenantWeightsEnabled = settings.RegisterBoolSetting( settings.SystemOnly, "admission.kv.stores.tenant_weights.enabled", "when true, tenant weights are enabled for KV-stores admission control", - false).WithPublic() + false, + settings.WithPublic) // EpochLIFOEnabled controls whether the adaptive epoch-LIFO scheme is enabled // for admission control. Is only relevant when the above admission control @@ -119,43 +124,44 @@ var EpochLIFOEnabled = settings.RegisterBoolSetting( settings.TenantWritable, "admission.epoch_lifo.enabled", "when true, epoch-LIFO behavior is enabled when there is significant delay in admission", - false).WithPublic() + false, + settings.WithPublic) var epochLIFOEpochDuration = settings.RegisterDurationSetting( settings.TenantWritable, "admission.epoch_lifo.epoch_duration", "the duration of an epoch, for epoch-LIFO admission control ordering", epochLength, - func(v time.Duration) error { + settings.WithValidateDuration(func(v time.Duration) error { if v < time.Millisecond { return errors.Errorf("epoch-LIFO: epoch duration is too small") } return nil - }).WithPublic() + }), settings.WithPublic) var epochLIFOEpochClosingDeltaDuration = settings.RegisterDurationSetting( settings.TenantWritable, "admission.epoch_lifo.epoch_closing_delta_duration", "the delta duration before closing an epoch, for epoch-LIFO admission control ordering", epochClosingDelta, - func(v time.Duration) error { + settings.WithValidateDuration(func(v time.Duration) error { if v < time.Millisecond { return errors.Errorf("epoch-LIFO: epoch closing delta is too small") } return nil - }).WithPublic() + }), settings.WithPublic) var epochLIFOQueueDelayThresholdToSwitchToLIFO = settings.RegisterDurationSetting( settings.TenantWritable, "admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo", "the queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering", maxQueueDelayToSwitchToLifo, - func(v time.Duration) error { + settings.WithValidateDuration(func(v time.Duration) error { if v < time.Millisecond { return errors.Errorf("epoch-LIFO: queue delay threshold is too small") } return nil - }).WithPublic() + }), settings.WithPublic) var rangeSequencerGCThreshold = settings.RegisterDurationSetting( settings.TenantWritable, diff --git a/pkg/util/log/logcrash/crash_reporting.go b/pkg/util/log/logcrash/crash_reporting.go index b316c664b8b1..ce2db47916ef 100644 --- a/pkg/util/log/logcrash/crash_reporting.go +++ b/pkg/util/log/logcrash/crash_reporting.go @@ -57,7 +57,7 @@ var ( "diagnostics.reporting.enabled", "enable reporting diagnostic metrics to cockroach labs", false, - ).WithPublic() + settings.WithPublic) // CrashReports wraps "diagnostics.reporting.send_crash_reports". CrashReports = settings.RegisterBoolSetting( diff --git a/pkg/util/schedulerlatency/sampler.go b/pkg/util/schedulerlatency/sampler.go index 5ee0d36e786c..fc9cccf8528d 100644 --- a/pkg/util/schedulerlatency/sampler.go +++ b/pkg/util/schedulerlatency/sampler.go @@ -38,12 +38,12 @@ var samplePeriod = settings.RegisterDurationSetting( "scheduler_latency.sample_period", "controls the duration between consecutive scheduler latency samples", 100*time.Millisecond, - func(period time.Duration) error { + settings.WithValidateDuration(func(period time.Duration) error { if period < time.Millisecond { return fmt.Errorf("minimum sample period is %s, got %s", time.Millisecond, period) } return nil - }, + }), ) var sampleDuration = settings.RegisterDurationSetting( @@ -51,12 +51,12 @@ var sampleDuration = settings.RegisterDurationSetting( "scheduler_latency.sample_duration", "controls the duration over which each scheduler latency sample is a measurement over", 2500*time.Millisecond, - func(duration time.Duration) error { + settings.WithValidateDuration(func(duration time.Duration) error { if duration < 100*time.Millisecond { return fmt.Errorf("minimum sample duration is %s, got %s", 100*time.Millisecond, duration) } return nil - }, + }), ) var schedulerLatency = metric.Metadata{ diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 208d457163f0..53ff32b082e3 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -117,56 +117,59 @@ var enableNetTrace = settings.RegisterBoolSetting( "trace.debug.enable", "if set, traces for recent requests can be seen at https:///debug/requests", false, -).WithPublic() + settings.WithPublic) -var openTelemetryCollector = settings.RegisterValidatedStringSetting( +var openTelemetryCollector = settings.RegisterStringSetting( settings.TenantWritable, "trace.opentelemetry.collector", "address of an OpenTelemetry trace collector to receive "+ "traces using the otel gRPC protocol, as :. "+ "If no port is specified, 4317 will be used.", envutil.EnvOrDefaultString("COCKROACH_OTLP_COLLECTOR", ""), - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { if s == "" { return nil } _, _, err := addr.SplitHostPort(s, "4317") return err - }, -).WithPublic() + }), + settings.WithPublic, +) -var jaegerAgent = settings.RegisterValidatedStringSetting( +var jaegerAgent = settings.RegisterStringSetting( settings.TenantWritable, "trace.jaeger.agent", "the address of a Jaeger agent to receive traces using the "+ "Jaeger UDP Thrift protocol, as :. "+ "If no port is specified, 6381 will be used.", envutil.EnvOrDefaultString("COCKROACH_JAEGER", ""), - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { if s == "" { return nil } _, _, err := addr.SplitHostPort(s, "6381") return err - }, -).WithPublic() + }), + settings.WithPublic, +) // ZipkinCollector is the cluster setting that specifies the Zipkin instance // to send traces to, if any. -var ZipkinCollector = settings.RegisterValidatedStringSetting( +var ZipkinCollector = settings.RegisterStringSetting( settings.TenantWritable, "trace.zipkin.collector", "the address of a Zipkin instance to receive traces, as :. "+ "If no port is specified, 9411 will be used.", envutil.EnvOrDefaultString("COCKROACH_ZIPKIN", ""), - func(_ *settings.Values, s string) error { + settings.WithValidateString(func(_ *settings.Values, s string) error { if s == "" { return nil } _, _, err := addr.SplitHostPort(s, "9411") return err - }, -).WithPublic() + }), + settings.WithPublic, +) // EnableActiveSpansRegistry controls Tracers configured as // WithTracingMode(TracingModeFromEnv) (which is the default). When enabled, @@ -178,7 +181,7 @@ var EnableActiveSpansRegistry = settings.RegisterBoolSetting( "trace.span_registry.enabled", "if set, ongoing traces can be seen at https:///#/debug/tracez", envutil.EnvOrDefaultBool("COCKROACH_REAL_SPANS", true), -).WithPublic() + settings.WithPublic) var periodicSnapshotInterval = settings.RegisterDurationSetting( settings.TenantWritable, @@ -186,7 +189,7 @@ var periodicSnapshotInterval = settings.RegisterDurationSetting( "if non-zero, interval at which background trace snapshots are captured", 0, settings.NonNegativeDuration, -).WithPublic() + settings.WithPublic) // panicOnUseAfterFinish, if set, causes use of a span after Finish() to panic // if detected.