Skip to content

Commit

Permalink
settings,*: use the option pattern with the setting register functions
Browse files Browse the repository at this point in the history
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 DurationWithMinimum(minValue time.Duration) SettingOption // RENAMED
func DurationWithMinimumOrZeroDisable(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 FloatWithMinimum(minValue float64) SettingOption // NEW
func FloatWithMinimumOrZeroDisable(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 IntWithMinimum(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
  • Loading branch information
knz committed Aug 19, 2023
1 parent 9fcc5c8 commit 094740b
Show file tree
Hide file tree
Showing 161 changed files with 1,295 additions and 1,501 deletions.
10 changes: 6 additions & 4 deletions pkg/ccl/auditloggingccl/audit_log_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,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",
Expand All @@ -71,13 +71,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,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backupdest/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
51 changes: 21 additions & 30 deletions pkg/ccl/changefeedccl/changefeedbase/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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.
//
Expand All @@ -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.
Expand All @@ -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 == "" {
Expand All @@ -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.
//
Expand All @@ -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.FloatWithMinimum(1),
)

// ProtectTimestampInterval controls the frequency of protected timestamp record updates
Expand All @@ -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(
Expand All @@ -218,15 +209,15 @@ 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(
settings.TenantWritable,
"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(
Expand All @@ -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(
Expand All @@ -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.
Expand Down Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/compression.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ var useFastGzip = settings.RegisterBoolSetting(
util.ConstantWithMetamorphicTestBool(
"changefeed.fast_gzip.enabled", true,
),
).WithPublic()
settings.WithPublic)

type compressionAlgo string

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/schemafeed/schema_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -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].
Expand Down
87 changes: 36 additions & 51 deletions pkg/ccl/jwtauthccl/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Loading

0 comments on commit 094740b

Please sign in to comment.