From a7c5cf255257e866828e8e5b979903134fccfc7b Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 27 Sep 2023 22:58:50 +0200 Subject: [PATCH] settings: rename and better document the setting classes **TLDR: "TenantReadOnly" -> "SystemVisible"; "TenantWritable" -> "ApplicationLevel".** See the new/extended explanatory comments in `pkg/settings/setting.go` for details. Review guidance: the only manual change is in `pkg/settings/setting.go`. Everything else was automated search-replace. No release note because the classes are not user-visible. Release note: None --- docs/RFCS/20211207_graceful_draining.md | 2 +- .../settings/settings-for-tenants.txt | 480 +++++++++--------- pkg/ccl/auditloggingccl/audit_log_config.go | 4 +- pkg/ccl/backupccl/backup_job.go | 2 +- pkg/ccl/backupccl/backup_planning.go | 2 +- pkg/ccl/backupccl/backup_processor.go | 12 +- pkg/ccl/backupccl/backup_tenant_test.go | 6 +- pkg/ccl/backupccl/backupbase/settings.go | 2 +- .../backupdest/backup_destination.go | 2 +- .../backupccl/backupinfo/manifest_handling.go | 4 +- pkg/ccl/backupccl/create_scheduled_backup.go | 2 +- pkg/ccl/backupccl/restore_data_processor.go | 6 +- pkg/ccl/backupccl/restore_job.go | 4 +- pkg/ccl/backupccl/restore_planning.go | 2 +- .../backupccl/restore_processor_planning.go | 6 +- pkg/ccl/backupccl/restore_progress.go | 2 +- pkg/ccl/backupccl/restore_span_covering.go | 2 +- pkg/ccl/backupccl/utils_test.go | 2 +- pkg/ccl/changefeedccl/changefeed_dist.go | 4 +- .../changefeedccl/changefeed_processors.go | 2 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 2 +- .../changefeedccl/changefeedbase/settings.go | 46 +- pkg/ccl/changefeedccl/compression.go | 2 +- .../changefeedccl/schemafeed/schema_feed.go | 2 +- pkg/ccl/changefeedccl/sink.go | 4 +- pkg/ccl/changefeedccl/sink_cloudstorage.go | 2 +- pkg/ccl/changefeedccl/telemetry.go | 2 +- pkg/ccl/jwtauthccl/settings.go | 10 +- .../kvccl/kvfollowerreadsccl/followerreads.go | 2 +- .../testdata/logic_test/tenant_settings | 4 +- .../tenantcostclient/tenant_side.go | 6 +- .../multitenantccl/tenantcostserver/server.go | 2 +- pkg/ccl/oidcccl/settings.go | 28 +- pkg/ccl/storageccl/external_sst_reader.go | 4 +- pkg/ccl/streamingccl/settings.go | 6 +- .../streamingest/stream_ingestion_dist.go | 4 +- .../stream_ingestion_frontier_processor.go | 4 +- .../stream_ingestion_processor.go | 10 +- pkg/ccl/utilccl/license_check.go | 2 +- pkg/cli/gen.go | 6 +- pkg/cloud/amazon/aws_kms.go | 2 +- pkg/cloud/amazon/s3_storage.go | 4 +- pkg/cloud/azure/azure_storage.go | 2 +- pkg/cloud/cloud_io.go | 8 +- pkg/cloud/gcp/gcs_storage.go | 4 +- pkg/cloud/impl_registry.go | 8 +- pkg/clusterversion/clusterversion_test.go | 2 +- pkg/clusterversion/setting.go | 4 +- pkg/jobs/config.go | 24 +- pkg/jobs/job_scheduler.go | 8 +- pkg/kv/bulk/setting.go | 2 +- pkg/kv/bulk/sst_batcher.go | 6 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 4 +- .../kvclient/kvcoord/dist_sender_rangefeed.go | 6 +- .../kvcoord/txn_interceptor_committer.go | 2 +- .../kvcoord/txn_interceptor_pipeliner.go | 8 +- .../kvcoord/txn_interceptor_span_refresher.go | 2 +- .../kvstreamer/avg_response_estimator.go | 2 +- pkg/kv/kvclient/kvstreamer/streamer.go | 2 +- pkg/kv/kvclient/rangefeed/db_adapter.go | 2 +- pkg/kv/kvprober/settings.go | 24 +- pkg/kv/kvserver/batcheval/cmd_export.go | 2 +- pkg/kv/kvserver/closedts/setting.go | 8 +- pkg/kv/kvserver/kvserverbase/base.go | 4 +- pkg/kv/kvserver/liveness/liveness.go | 2 +- .../protectedts/ptreconcile/reconciler.go | 2 +- pkg/kv/kvserver/protectedts/settings.go | 6 +- pkg/kv/kvserver/queue.go | 2 +- pkg/kv/kvserver/replica.go | 2 +- pkg/kv/kvserver/replica_follower_read.go | 2 +- pkg/kv/kvserver/replica_rangefeed.go | 4 +- pkg/kv/kvserver/store.go | 2 +- .../multitenantio/cost_controlling_io.go | 2 +- pkg/multitenant/tenantcostmodel/settings.go | 22 +- pkg/roachpb/span_stats.go | 8 +- pkg/rpc/settings.go | 4 +- pkg/security/cert_expiry_cache.go | 2 +- pkg/security/password.go | 16 +- pkg/security/tls_settings.go | 4 +- pkg/server/admin.go | 2 +- pkg/server/authserver/authentication.go | 2 +- pkg/server/clock_monotonicity.go | 4 +- pkg/server/cluster_settings.go | 8 +- pkg/server/debug/server.go | 2 +- pkg/server/diagnostics/reporter.go | 2 +- pkg/server/drain.go | 8 +- pkg/server/goroutinedumper/goroutinedumper.go | 4 +- pkg/server/hot_ranges.go | 2 +- pkg/server/node.go | 4 +- pkg/server/profiler/cpuprofiler.go | 8 +- pkg/server/profiler/profilestore.go | 8 +- pkg/server/server.go | 2 +- pkg/server/server_http.go | 2 +- pkg/server/server_sql.go | 8 +- pkg/server/server_systemlog_gc.go | 10 +- .../settingswatcher/settings_watcher.go | 36 +- .../settings_watcher_external_test.go | 20 +- pkg/server/status/recorder.go | 2 +- pkg/server/structlogging/hot_ranges_log.go | 6 +- .../tenantsettingswatcher/overrides_store.go | 6 +- pkg/server/tenantsettingswatcher/watcher.go | 2 +- pkg/server/tracedumper/tracedumper.go | 2 +- pkg/settings/doc.go | 2 +- pkg/settings/integration_tests/propagation.go | 4 +- .../read_only_1/propagation_test.go | 2 +- .../read_only_2/propagation_test.go | 2 +- .../read_write_1/propagation_test.go | 2 +- .../read_write_2/propagation_test.go | 2 +- .../integration_tests/settings_test.go | 10 +- pkg/settings/registry.go | 16 +- pkg/settings/setting.go | 113 ++++- pkg/settings/settings_test.go | 30 +- pkg/settings/updater.go | 8 +- pkg/settings/values.go | 8 +- pkg/spanconfig/spanconfigjob/job.go | 2 +- pkg/spanconfig/spanconfiglimiter/limiter.go | 2 +- pkg/spanconfig/spanconfigmanager/manager.go | 4 +- pkg/sql/alter_database.go | 2 +- pkg/sql/alter_role.go | 2 +- pkg/sql/authorization.go | 2 +- pkg/sql/backfill.go | 6 +- pkg/sql/backfill/backfill.go | 2 +- pkg/sql/backfill/mvcc_index_merger.go | 6 +- .../hydrateddesccache/hydratedcache.go | 2 +- pkg/sql/catalog/lease/lease.go | 6 +- pkg/sql/catalog/lease/storage.go | 4 +- pkg/sql/catalog/randgen/randgen.go | 4 +- .../schematelemetrycontroller/controller.go | 2 +- pkg/sql/catalog/tabledesc/table.go | 2 +- pkg/sql/closed_session_cache.go | 4 +- pkg/sql/colexec/colbuilder/execplan.go | 4 +- pkg/sql/colexec/hash_aggregator.go | 2 +- pkg/sql/colfetcher/cfetcher_wrapper.go | 2 +- pkg/sql/colfetcher/index_join.go | 2 +- pkg/sql/colflow/vectorized_flow.go | 2 +- pkg/sql/conn_executor.go | 10 +- pkg/sql/contention/cluster_settings.go | 6 +- .../contention/txnidcache/cluster_settings.go | 2 +- pkg/sql/crdb_internal.go | 2 +- pkg/sql/create_stats.go | 4 +- pkg/sql/delegate/show_tables.go | 2 +- pkg/sql/deprecatedshowranges/condition.go | 2 +- pkg/sql/descriptor.go | 6 +- pkg/sql/distsql_physical_planner.go | 2 +- pkg/sql/distsql_plan_stats.go | 4 +- pkg/sql/distsql_running.go | 8 +- pkg/sql/event_log.go | 2 +- pkg/sql/exec_log.go | 12 +- pkg/sql/exec_util.go | 112 ++-- pkg/sql/execinfra/readerbase.go | 4 +- pkg/sql/execinfra/utils.go | 2 +- pkg/sql/export.go | 2 +- pkg/sql/flowinfra/flow_registry.go | 2 +- pkg/sql/gcjob/gc_job.go | 4 +- pkg/sql/idxusage/cluster_settings.go | 2 +- pkg/sql/idxusage/index_usage_stats_rec.go | 2 +- pkg/sql/importer/export_base.go | 2 +- pkg/sql/importer/import_job.go | 2 +- pkg/sql/importer/import_planning.go | 2 +- pkg/sql/importer/import_processor.go | 10 +- pkg/sql/importer/import_processor_planning.go | 4 +- pkg/sql/instrumentation.go | 6 +- pkg/sql/join_token.go | 2 +- pkg/sql/logictest/logic.go | 6 +- 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 | 2 +- pkg/sql/pgwire/ident_map_conf.go | 2 +- pkg/sql/pgwire/pgwirebase/encoding.go | 2 +- pkg/sql/pgwire/server.go | 6 +- pkg/sql/plan_opt.go | 2 +- pkg/sql/resolver.go | 2 +- pkg/sql/revert.go | 6 +- pkg/sql/row/helper.go | 4 +- pkg/sql/rowexec/columnbackfiller.go | 2 +- pkg/sql/rowexec/indexbackfiller.go | 4 +- pkg/sql/rowexec/joinreader.go | 2 +- pkg/sql/rowexec/joinreader_strategies.go | 4 +- .../captured_index_usage_stats.go | 8 +- pkg/sql/schema_change_cluster_setting.go | 2 +- pkg/sql/schema_changer.go | 2 +- .../internal/scbuildstmt/statement_control.go | 2 +- pkg/sql/schemachanger/scrun/scrun.go | 2 +- pkg/sql/sem/eval/settings.go | 2 +- pkg/sql/serial.go | 2 +- pkg/sql/session_revival_token.go | 2 +- pkg/sql/session_state.go | 2 +- pkg/sql/sessioninit/cache.go | 2 +- pkg/sql/set_cluster_setting.go | 2 +- pkg/sql/set_zone_config.go | 2 +- pkg/sql/sql_activity_update_job.go | 6 +- .../instancestorage/instancestorage.go | 4 +- pkg/sql/sqlliveness/slinstance/slinstance.go | 4 +- pkg/sql/sqlliveness/slstorage/slstorage.go | 6 +- pkg/sql/sqlstats/cluster_settings.go | 32 +- pkg/sql/sqlstats/insights/insights.go | 12 +- .../persistedsqlstats/cluster_settings.go | 22 +- pkg/sql/sqlstats/sslocal/cluster_settings.go | 2 +- .../sqlstats/ssmemstorage/ss_mem_counter.go | 2 +- pkg/sql/stats/automatic_stats.go | 18 +- pkg/sql/stats/delete_stats.go | 2 +- pkg/sql/stats/forecast.go | 2 +- pkg/sql/stats/histogram.go | 4 +- .../stmtdiagnostics/statement_diagnostics.go | 6 +- pkg/sql/tablewriter.go | 2 +- pkg/sql/telemetry_logging.go | 10 +- pkg/sql/temporary_schema.go | 4 +- pkg/sql/tenant_accessors.go | 2 +- pkg/sql/truncate.go | 2 +- pkg/sql/ttl/ttlbase/ttl_helpers.go | 8 +- pkg/sql/txn_fingerprint_id_cache.go | 2 +- pkg/sql/user.go | 2 +- pkg/storage/engine.go | 4 +- pkg/storage/mvcc.go | 6 +- pkg/storage/pebble.go | 8 +- pkg/ts/db.go | 6 +- pkg/ui/ui.go | 2 +- pkg/util/admission/io_load_listener.go | 4 +- pkg/util/admission/kv_slot_adjuster.go | 2 +- pkg/util/admission/work_queue.go | 16 +- pkg/util/log/logcrash/crash_reporting.go | 6 +- pkg/util/schedulerlatency/sampler.go | 4 +- pkg/util/tracing/tracer.go | 14 +- 225 files changed, 959 insertions(+), 898 deletions(-) diff --git a/docs/RFCS/20211207_graceful_draining.md b/docs/RFCS/20211207_graceful_draining.md index 5b7284d7bcb4..89f982a8bdf2 100644 --- a/docs/RFCS/20211207_graceful_draining.md +++ b/docs/RFCS/20211207_graceful_draining.md @@ -157,7 +157,7 @@ declared as a `*settings.DurationSetting` as follows: ```go connectionWait = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "server.shutdown.connection_wait", "the amount of time a server waits for clients to close existing SQL connections. After the start of connection_wait, no new SQL connections are diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 0da64253c148..63373deded6d 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -1,323 +1,323 @@ Setting Type Default Description Class -admission.epoch_lifo.enabled boolean false when true, epoch-LIFO behavior is enabled when there is significant delay in admission tenant-rw -admission.epoch_lifo.epoch_closing_delta_duration duration 5ms the delta duration before closing an epoch, for epoch-LIFO admission control ordering tenant-rw -admission.epoch_lifo.epoch_duration duration 100ms the duration of an epoch, for epoch-LIFO admission control ordering tenant-rw -admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo duration 105ms the queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering tenant-rw -admission.sql_kv_response.enabled boolean true when true, work performed by the SQL layer when receiving a KV response is subject to admission control tenant-rw -admission.sql_sql_response.enabled boolean true when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control tenant-rw -bulkio.backup.deprecated_full_backup_with_subdir.enabled boolean false 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 tenant-rw -bulkio.backup.file_size byte size 128 MiB target size for individual data files produced during BACKUP tenant-rw -bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail tenant-rw -bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads tenant-rw -bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up tenant-rw -changefeed.backfill.concurrent_scan_requests integer 0 number of concurrent scan requests per node issued during a backfill tenant-rw -changefeed.backfill.scan_request_size integer 524288 the maximum number of bytes returned by each scan request tenant-rw -changefeed.balance_range_distribution.enabled boolean false if enabled, the ranges are balanced equally among all nodes tenant-rw -changefeed.batch_reduction_retry.enabled boolean false if true, kafka changefeeds upon erroring on an oversized batch will attempt to resend the messages with progressively lower batch sizes tenant-rw -changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can buffer tenant-rw -changefeed.event_consumer_workers integer 0 the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value. for experimental/core changefeeds and changefeeds using parquet format, this is disabled tenant-rw -changefeed.fast_gzip.enabled boolean true use fast gzip implementation tenant-rw -changefeed.frontier_highwater_lag_checkpoint_threshold duration 10m0s 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 tenant-rw -changefeed.memory.per_changefeed_limit byte size 512 MiB controls amount of data that can be buffered per changefeed tenant-rw -changefeed.min_highwater_advance duration 0s minimum amount of time the changefeed high water mark must advance for it to be eligible for checkpointing; Default of 0 will checkpoint every time frontier advances, as long as the rate of checkpointing keeps up with the rate of frontier changes tenant-rw -changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds tenant-rw -changefeed.protect_timestamp.max_age duration 96h0m0s fail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expiration tenant-rw -changefeed.protect_timestamp_interval duration 10m0s controls how often the changefeed forwards its protected timestamp to the resolved timestamp tenant-rw -changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables tenant-rw -changefeed.sink_io_workers integer 0 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 tenant-rw -cloudstorage.azure.concurrent_upload_buffers integer 1 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 tenant-rw -cloudstorage.http.custom_ca string custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage tenant-rw -cloudstorage.timeout duration 10m0s the timeout for import/export storage operations tenant-rw -cluster.organization string organization name tenant-rw -cluster.preserve_downgrade_option string disable (automatic or manual) cluster version upgrade from the specified version until reset tenant-rw -diagnostics.forced_sql_stat_reset.interval duration 2h0m0s interval after which the reported SQL Stats are reset even if not collected by telemetry reporter. It has a max value of 24H. tenant-rw -diagnostics.reporting.enabled boolean true enable reporting diagnostic metrics to cockroach labs tenant-rw -diagnostics.reporting.interval duration 1h0m0s interval at which diagnostics data should be reported tenant-rw -enterprise.license string the encoded cluster license tenant-rw -external.graphite.endpoint string if nonempty, push server metrics to the Graphite or Carbon server at the specified host:port tenant-rw -external.graphite.interval duration 10s the interval at which metrics are pushed to Graphite (if enabled) tenant-rw -feature.backup.enabled boolean true set to true to enable backups, false to disable; default is true tenant-rw -feature.changefeed.enabled boolean true set to true to enable changefeeds, false to disable; default is true tenant-rw -feature.export.enabled boolean true set to true to enable exports, false to disable; default is true tenant-rw -feature.import.enabled boolean true set to true to enable imports, false to disable; default is true tenant-rw -feature.restore.enabled boolean true set to true to enable restore, false to disable; default is true tenant-rw -feature.schema_change.enabled boolean true set to true to enable schema changes, false to disable; default is true tenant-rw -feature.stats.enabled boolean true set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true tenant-rw -jobs.retention_time duration 336h0m0s the amount of time for which records for completed jobs are retained tenant-rw -kv.bulk_sst.target_size byte size 16 MiB target size for SSTs emitted from export requests; export requests (i.e. BACKUP) may buffer up to the sum of kv.bulk_sst.target_size and kv.bulk_sst.max_allowed_overage in memory tenant-ro -kv.closed_timestamp.follower_reads.enabled boolean true allow (all) replicas to serve consistent historical reads based on closed timestamp information tenant-ro -kv.closed_timestamp.lead_for_global_reads_override duration 0s if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps tenant-ro -kv.closed_timestamp.side_transport_interval duration 200ms the interval at which the closed timestamp side-transport attempts to advance each range's closed timestamp; set to 0 to disable the side-transport tenant-ro -kv.closed_timestamp.target_duration duration 3s if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration tenant-ro -kv.protectedts.reconciliation.interval duration 5m0s the frequency for reconciling jobs with protected timestamp records tenant-ro -kv.rangefeed.client.stream_startup_rate integer 100 controls the rate per second the client will initiate new rangefeed stream for a single range; 0 implies unlimited tenant-rw -kv.rangefeed.closed_timestamp_refresh_interval duration 3s the interval at which closed-timestamp updatesare delivered to rangefeeds; set to 0 to use kv.closed_timestamp.side_transport_interval tenant-ro -kv.rangefeed.enabled boolean false if set, rangefeed registration is enabled tenant-ro -kv.rangefeed.range_stuck_threshold duration 1m0s restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.rangefeed.closed_timestamp_refresh_interval takes precedence) tenant-rw -kv.transaction.max_intents_bytes integer 4194304 maximum number of bytes used to track locks in transactions tenant-rw -kv.transaction.max_refresh_spans_bytes integer 4194304 maximum number of bytes used to track refresh spans in serializable transactions tenant-rw -kv.transaction.reject_over_max_intents_budget.enabled boolean false if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed tenant-rw -schedules.backup.gc_protection.enabled boolean true enable chaining of GC protection across backups run as part of a schedule tenant-rw -security.ocsp.mode enumeration off 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 = 0, lax = 1, strict = 2] tenant-rw -security.ocsp.timeout duration 3s timeout before considering the OCSP server unreachable tenant-rw -server.auth_log.sql_connections.enabled boolean false if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes) tenant-rw -server.auth_log.sql_sessions.enabled boolean false if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes) tenant-rw -server.authentication_cache.enabled boolean true enables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information tenant-rw -server.child_metrics.enabled boolean false enables the exporting of child metrics, additional prometheus time series with extra labels tenant-rw -server.client_cert_expiration_cache.capacity integer 1000 the maximum number of client cert expirations stored tenant-rw -server.clock.forward_jump_check.enabled boolean false if enabled, forward clock jumps > max_offset/2 will cause a panic tenant-rw -server.clock.persist_upper_bound_interval duration 0s the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature. tenant-rw -server.eventlog.enabled boolean true if set, logged notable events are also stored in the table system.eventlog tenant-rw -server.eventlog.ttl duration 2160h0m0s if nonzero, entries in system.eventlog older than this duration are periodically purged tenant-rw -server.host_based_authentication.configuration string host-based authentication configuration to use during connection authentication tenant-rw -server.hot_ranges_request.node.timeout duration 5m0s 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 tenant-rw -server.hsts.enabled boolean false if true, HSTS headers will be sent along with all HTTP requests. The headers will contain a max-age setting of one year. Browsers honoring the header will always use HTTPS to access the DB Console. Ensure that TLS is correctly configured prior to enabling. tenant-rw -server.identity_map.configuration string system-identity to database-username mappings tenant-rw -server.log_gc.max_deletions_per_cycle integer 1000 the maximum number of entries to delete on each purge of log-like system tables tenant-rw -server.log_gc.period duration 1h0m0s the period at which log-like system tables are checked for old entries tenant-rw -server.max_connections_per_gateway integer -1 the maximum number of SQL connections per gateway allowed at a given time (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. tenant-rw -server.oidc_authentication.autologin.enabled boolean false if true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint tenant-rw -server.oidc_authentication.button_text string Log in with your OIDC provider text to show on button on DB Console login page to login with your OIDC provider (only shown if OIDC is enabled) tenant-rw -server.oidc_authentication.claim_json_key string sets JSON key of principal to extract from payload after OIDC authentication completes (usually email or sid) tenant-rw -server.oidc_authentication.client_id string sets OIDC client id tenant-rw -server.oidc_authentication.client_secret string sets OIDC client secret tenant-rw -server.oidc_authentication.enabled boolean false enables or disabled OIDC login for the DB Console tenant-rw -server.oidc_authentication.principal_regex string (.+) 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) tenant-rw -server.oidc_authentication.provider_url string sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve) tenant-rw -server.oidc_authentication.redirect_url string https://localhost:8080/oidc/v1/callback 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) tenant-rw -server.oidc_authentication.scopes string openid sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`) tenant-rw -server.rangelog.ttl duration 720h0m0s if nonzero, entries in system.rangelog older than this duration are periodically purged tenant-rw -server.shutdown.connection_wait duration 0s the maximum amount of time a server waits for all SQL connections to be closed before proceeding with a drain. (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) tenant-rw -server.shutdown.drain_wait duration 0s the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting. --drain-wait is to specify the duration of the whole draining process, while server.shutdown.drain_wait is to set the wait time for health probes to notice that the node is not ready.) tenant-rw -server.shutdown.jobs_wait duration 10s the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown tenant-rw -server.shutdown.query_wait duration 10s the timeout for waiting for active queries to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) tenant-rw -server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead tenant-rw -server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM tenant-rw -server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled boolean true if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt tenant-rw -server.user_login.min_password_length integer 1 the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case. tenant-rw -server.user_login.password_encryption enumeration scram-sha-256 which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3] tenant-rw -server.user_login.password_hashes.default_cost.crdb_bcrypt integer 10 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31) tenant-rw -server.user_login.password_hashes.default_cost.scram_sha_256 integer 10610 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000) tenant-rw -server.user_login.rehash_scram_stored_passwords_on_cost_change.enabled boolean true if server.user_login.password_hashes.default_cost.scram_sha_256 differs from, the cost in a stored hash, this controls whether to automatically re-encode stored passwords using scram-sha-256 with the new default cost tenant-rw -server.user_login.timeout duration 10s timeout after which client authentication times out if some system range is unavailable (0 = no timeout) tenant-rw -server.user_login.upgrade_bcrypt_stored_passwords_to_scram.enabled boolean true 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 tenant-rw -server.web_session.purge.ttl duration 1h0m0s if nonzero, entries in system.web_sessions older than this duration are periodically purged tenant-rw -server.web_session.timeout duration 168h0m0s the duration that a newly created web session will be valid tenant-rw -sql.auth.change_own_password.enabled boolean false controls whether a user is allowed to change their own password, even if they have no other privileges tenant-rw -sql.auth.public_schema_create_privilege.enabled boolean true determines whether to grant all users the CREATE privileges on the public schema when it is created tenant-rw -sql.auth.resolve_membership_single_scan.enabled boolean true determines whether to populate the role membership cache with a single scan tenant-rw -sql.closed_session_cache.capacity integer 1000 the maximum number of sessions in the cache tenant-rw -sql.closed_session_cache.time_to_live integer 3600 the maximum time to live, in seconds tenant-rw -sql.contention.event_store.capacity byte size 64 MiB the in-memory storage capacity per-node of contention event store tenant-rw -sql.contention.event_store.duration_threshold duration 0s minimum contention duration to cause the contention events to be collected into crdb_internal.transaction_contention_events tenant-rw -sql.contention.txn_id_cache.max_size byte size 64 MiB the maximum byte size TxnID cache will use (set to 0 to disable) tenant-rw -sql.cross_db_fks.enabled boolean false if true, creating foreign key references across databases is allowed tenant-rw -sql.cross_db_sequence_owners.enabled boolean false if true, creating sequences owned by tables from other databases is allowed tenant-rw -sql.cross_db_sequence_references.enabled boolean false if true, sequences referenced by tables from other databases are allowed tenant-rw -sql.cross_db_views.enabled boolean false if true, creating views that refer to other databases is allowed tenant-rw +admission.epoch_lifo.enabled boolean false when true, epoch-LIFO behavior is enabled when there is significant delay in admission application +admission.epoch_lifo.epoch_closing_delta_duration duration 5ms the delta duration before closing an epoch, for epoch-LIFO admission control ordering application +admission.epoch_lifo.epoch_duration duration 100ms the duration of an epoch, for epoch-LIFO admission control ordering application +admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo duration 105ms the queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering application +admission.sql_kv_response.enabled boolean true when true, work performed by the SQL layer when receiving a KV response is subject to admission control application +admission.sql_sql_response.enabled boolean true when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control application +bulkio.backup.deprecated_full_backup_with_subdir.enabled boolean false 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 application +bulkio.backup.file_size byte size 128 MiB target size for individual data files produced during BACKUP application +bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail application +bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads application +bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up application +changefeed.backfill.concurrent_scan_requests integer 0 number of concurrent scan requests per node issued during a backfill application +changefeed.backfill.scan_request_size integer 524288 the maximum number of bytes returned by each scan request application +changefeed.balance_range_distribution.enabled boolean false if enabled, the ranges are balanced equally among all nodes application +changefeed.batch_reduction_retry.enabled boolean false if true, kafka changefeeds upon erroring on an oversized batch will attempt to resend the messages with progressively lower batch sizes application +changefeed.event_consumer_worker_queue_size integer 16 if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can buffer application +changefeed.event_consumer_workers integer 0 the number of workers to use when processing events: <0 disables, 0 assigns a reasonable default, >0 assigns the setting value. for experimental/core changefeeds and changefeeds using parquet format, this is disabled application +changefeed.fast_gzip.enabled boolean true use fast gzip implementation application +changefeed.frontier_highwater_lag_checkpoint_threshold duration 10m0s 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 application +changefeed.memory.per_changefeed_limit byte size 512 MiB controls amount of data that can be buffered per changefeed application +changefeed.min_highwater_advance duration 0s minimum amount of time the changefeed high water mark must advance for it to be eligible for checkpointing; Default of 0 will checkpoint every time frontier advances, as long as the rate of checkpointing keeps up with the rate of frontier changes application +changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds application +changefeed.protect_timestamp.max_age duration 96h0m0s fail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expiration application +changefeed.protect_timestamp_interval duration 10m0s controls how often the changefeed forwards its protected timestamp to the resolved timestamp application +changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables application +changefeed.sink_io_workers integer 0 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 application +cloudstorage.azure.concurrent_upload_buffers integer 1 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 application +cloudstorage.http.custom_ca string custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage application +cloudstorage.timeout duration 10m0s the timeout for import/export storage operations application +cluster.organization string organization name application +cluster.preserve_downgrade_option string disable (automatic or manual) cluster version upgrade from the specified version until reset application +diagnostics.forced_sql_stat_reset.interval duration 2h0m0s interval after which the reported SQL Stats are reset even if not collected by telemetry reporter. It has a max value of 24H. application +diagnostics.reporting.enabled boolean true enable reporting diagnostic metrics to cockroach labs application +diagnostics.reporting.interval duration 1h0m0s interval at which diagnostics data should be reported application +enterprise.license string the encoded cluster license application +external.graphite.endpoint string if nonempty, push server metrics to the Graphite or Carbon server at the specified host:port application +external.graphite.interval duration 10s the interval at which metrics are pushed to Graphite (if enabled) application +feature.backup.enabled boolean true set to true to enable backups, false to disable; default is true application +feature.changefeed.enabled boolean true set to true to enable changefeeds, false to disable; default is true application +feature.export.enabled boolean true set to true to enable exports, false to disable; default is true application +feature.import.enabled boolean true set to true to enable imports, false to disable; default is true application +feature.restore.enabled boolean true set to true to enable restore, false to disable; default is true application +feature.schema_change.enabled boolean true set to true to enable schema changes, false to disable; default is true application +feature.stats.enabled boolean true set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true application +jobs.retention_time duration 336h0m0s the amount of time for which records for completed jobs are retained application +kv.bulk_sst.target_size byte size 16 MiB target size for SSTs emitted from export requests; export requests (i.e. BACKUP) may buffer up to the sum of kv.bulk_sst.target_size and kv.bulk_sst.max_allowed_overage in memory system-visible +kv.closed_timestamp.follower_reads.enabled boolean true allow (all) replicas to serve consistent historical reads based on closed timestamp information system-visible +kv.closed_timestamp.lead_for_global_reads_override duration 0s if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps system-visible +kv.closed_timestamp.side_transport_interval duration 200ms the interval at which the closed timestamp side-transport attempts to advance each range's closed timestamp; set to 0 to disable the side-transport system-visible +kv.closed_timestamp.target_duration duration 3s if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration system-visible +kv.protectedts.reconciliation.interval duration 5m0s the frequency for reconciling jobs with protected timestamp records system-visible +kv.rangefeed.client.stream_startup_rate integer 100 controls the rate per second the client will initiate new rangefeed stream for a single range; 0 implies unlimited application +kv.rangefeed.closed_timestamp_refresh_interval duration 3s the interval at which closed-timestamp updatesare delivered to rangefeeds; set to 0 to use kv.closed_timestamp.side_transport_interval system-visible +kv.rangefeed.enabled boolean false if set, rangefeed registration is enabled system-visible +kv.rangefeed.range_stuck_threshold duration 1m0s restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.rangefeed.closed_timestamp_refresh_interval takes precedence) application +kv.transaction.max_intents_bytes integer 4194304 maximum number of bytes used to track locks in transactions application +kv.transaction.max_refresh_spans_bytes integer 4194304 maximum number of bytes used to track refresh spans in serializable transactions application +kv.transaction.reject_over_max_intents_budget.enabled boolean false if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed application +schedules.backup.gc_protection.enabled boolean true enable chaining of GC protection across backups run as part of a schedule application +security.ocsp.mode enumeration off 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 = 0, lax = 1, strict = 2] application +security.ocsp.timeout duration 3s timeout before considering the OCSP server unreachable application +server.auth_log.sql_connections.enabled boolean false if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes) application +server.auth_log.sql_sessions.enabled boolean false if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes) application +server.authentication_cache.enabled boolean true enables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information application +server.child_metrics.enabled boolean false enables the exporting of child metrics, additional prometheus time series with extra labels application +server.client_cert_expiration_cache.capacity integer 1000 the maximum number of client cert expirations stored application +server.clock.forward_jump_check.enabled boolean false if enabled, forward clock jumps > max_offset/2 will cause a panic application +server.clock.persist_upper_bound_interval duration 0s the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature. application +server.eventlog.enabled boolean true if set, logged notable events are also stored in the table system.eventlog application +server.eventlog.ttl duration 2160h0m0s if nonzero, entries in system.eventlog older than this duration are periodically purged application +server.host_based_authentication.configuration string host-based authentication configuration to use during connection authentication application +server.hot_ranges_request.node.timeout duration 5m0s 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 application +server.hsts.enabled boolean false if true, HSTS headers will be sent along with all HTTP requests. The headers will contain a max-age setting of one year. Browsers honoring the header will always use HTTPS to access the DB Console. Ensure that TLS is correctly configured prior to enabling. application +server.identity_map.configuration string system-identity to database-username mappings application +server.log_gc.max_deletions_per_cycle integer 1000 the maximum number of entries to delete on each purge of log-like system tables application +server.log_gc.period duration 1h0m0s the period at which log-like system tables are checked for old entries application +server.max_connections_per_gateway integer -1 the maximum number of SQL connections per gateway allowed at a given time (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. application +server.oidc_authentication.autologin.enabled boolean false if true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint application +server.oidc_authentication.button_text string Log in with your OIDC provider text to show on button on DB Console login page to login with your OIDC provider (only shown if OIDC is enabled) application +server.oidc_authentication.claim_json_key string sets JSON key of principal to extract from payload after OIDC authentication completes (usually email or sid) application +server.oidc_authentication.client_id string sets OIDC client id application +server.oidc_authentication.client_secret string sets OIDC client secret application +server.oidc_authentication.enabled boolean false enables or disabled OIDC login for the DB Console application +server.oidc_authentication.principal_regex string (.+) 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) application +server.oidc_authentication.provider_url string sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve) application +server.oidc_authentication.redirect_url string https://localhost:8080/oidc/v1/callback 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) application +server.oidc_authentication.scopes string openid sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`) application +server.rangelog.ttl duration 720h0m0s if nonzero, entries in system.rangelog older than this duration are periodically purged application +server.shutdown.connection_wait duration 0s the maximum amount of time a server waits for all SQL connections to be closed before proceeding with a drain. (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) application +server.shutdown.drain_wait duration 0s the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting. --drain-wait is to specify the duration of the whole draining process, while server.shutdown.drain_wait is to set the wait time for health probes to notice that the node is not ready.) application +server.shutdown.jobs_wait duration 10s the maximum amount of time a server waits for all currently executing jobs to notice drain request and to perform orderly shutdown application +server.shutdown.query_wait duration 10s the timeout for waiting for active queries to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) application +server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead application +server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM application +server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled boolean true if server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt application +server.user_login.min_password_length integer 1 the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case. application +server.user_login.password_encryption enumeration scram-sha-256 which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3] application +server.user_login.password_hashes.default_cost.crdb_bcrypt integer 10 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31) application +server.user_login.password_hashes.default_cost.scram_sha_256 integer 10610 the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000) application +server.user_login.rehash_scram_stored_passwords_on_cost_change.enabled boolean true if server.user_login.password_hashes.default_cost.scram_sha_256 differs from, the cost in a stored hash, this controls whether to automatically re-encode stored passwords using scram-sha-256 with the new default cost application +server.user_login.timeout duration 10s timeout after which client authentication times out if some system range is unavailable (0 = no timeout) application +server.user_login.upgrade_bcrypt_stored_passwords_to_scram.enabled boolean true 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 application +server.web_session.purge.ttl duration 1h0m0s if nonzero, entries in system.web_sessions older than this duration are periodically purged application +server.web_session.timeout duration 168h0m0s the duration that a newly created web session will be valid application +sql.auth.change_own_password.enabled boolean false controls whether a user is allowed to change their own password, even if they have no other privileges application +sql.auth.public_schema_create_privilege.enabled boolean true determines whether to grant all users the CREATE privileges on the public schema when it is created application +sql.auth.resolve_membership_single_scan.enabled boolean true determines whether to populate the role membership cache with a single scan application +sql.closed_session_cache.capacity integer 1000 the maximum number of sessions in the cache application +sql.closed_session_cache.time_to_live integer 3600 the maximum time to live, in seconds application +sql.contention.event_store.capacity byte size 64 MiB the in-memory storage capacity per-node of contention event store application +sql.contention.event_store.duration_threshold duration 0s minimum contention duration to cause the contention events to be collected into crdb_internal.transaction_contention_events application +sql.contention.txn_id_cache.max_size byte size 64 MiB the maximum byte size TxnID cache will use (set to 0 to disable) application +sql.cross_db_fks.enabled boolean false if true, creating foreign key references across databases is allowed application +sql.cross_db_sequence_owners.enabled boolean false if true, creating sequences owned by tables from other databases is allowed application +sql.cross_db_sequence_references.enabled boolean false if true, sequences referenced by tables from other databases are allowed application +sql.cross_db_views.enabled boolean false if true, creating views that refer to other databases is allowed application sql.defaults.cost_scans_with_default_col_size.enabled boolean false "setting to true uses the same size for all columns to compute scan cost This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.datestyle enumeration iso, mdy "default value for DateStyle session setting [iso, mdy = 0, iso, dmy = 1, iso, ymd = 2] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.default_hash_sharded_index_bucket_count integer 16 "used as bucket count if bucket count is not specified in hash sharded index definition This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.default_int_size integer 8 "the size, in bytes, of an INT type This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.disallow_full_table_scans.enabled boolean false "setting to true rejects queries that have planned a full table scan This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.distsql enumeration auto "default distributed SQL execution mode [off = 0, auto = 1, on = 2, always = 3] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.experimental_alter_column_type.enabled boolean false "default value for experimental_alter_column_type session setting; enables the use of ALTER COLUMN TYPE for general conversions This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.experimental_distsql_planning enumeration off "default experimental_distsql_planning mode; enables experimental opt-driven DistSQL planning [off = 0, on = 1] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.experimental_enable_unique_without_index_constraints.enabled boolean false "default value for experimental_enable_unique_without_index_constraints session setting;disables unique without index constraints by default This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.experimental_implicit_column_partitioning.enabled boolean false "default value for experimental_enable_temp_tables; allows for the use of implicit column partitioning This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.experimental_temporary_tables.enabled boolean false "default value for experimental_enable_temp_tables; allows for use of temporary tables by default This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.foreign_key_cascades_limit integer 10000 "default value for foreign_key_cascades_limit session setting; limits the number of cascading operations that run as part of a single query This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.idle_in_session_timeout duration 0s "default value for the idle_in_session_timeout; default value for the idle_in_session_timeout session setting; controls the duration a session is permitted to idle before the session is terminated; if set to 0, there is no timeout This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.idle_in_transaction_session_timeout duration 0s "default value for the idle_in_transaction_session_timeout; controls the duration a session is permitted to idle in a transaction before the session is terminated; if set to 0, there is no timeout This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.implicit_select_for_update.enabled boolean true "default value for enable_implicit_select_for_update session setting; enables FOR UPDATE locking during the row-fetch phase of mutation statements This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.insert_fast_path.enabled boolean true "default value for enable_insert_fast_path session setting; enables a specialized insert path This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.intervalstyle enumeration postgres "default value for IntervalStyle session setting [postgres = 0, iso_8601 = 1, sql_standard = 2] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.large_full_scan_rows float 1000 "default value for large_full_scan_rows session setting which determines the maximum table size allowed for a full scan when disallow_full_table_scans is set to true This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.locality_optimized_partitioned_index_scan.enabled boolean true "default value for locality_optimized_partitioned_index_scan session setting; enables searching for rows in the current region before searching remote regions This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.lock_timeout duration 0s "default value for the lock_timeout; default value for the lock_timeout session setting; controls the duration a query is permitted to wait while attempting to acquire a lock on a key or while blocking on an existing lock in order to perform a non-locking read on a key; if set to 0, there is no timeout This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.on_update_rehome_row.enabled boolean true "default value for on_update_rehome_row; enables ON UPDATE rehome_row() expressions to trigger on updates This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.optimizer_use_histograms.enabled boolean true "default value for optimizer_use_histograms session setting; enables usage of histograms in the optimizer by default This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.optimizer_use_multicol_stats.enabled boolean true "default value for optimizer_use_multicol_stats session setting; enables usage of multi-column stats in the optimizer by default This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.override_alter_primary_region_in_super_region.enabled boolean false "default value for override_alter_primary_region_in_super_region; allows for altering the primary region even if the primary region is a member of a super region This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.override_multi_region_zone_config.enabled boolean false "default value for override_multi_region_zone_config; allows for overriding the zone configs of a multi-region table or database This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.prefer_lookup_joins_for_fks.enabled boolean false "default value for prefer_lookup_joins_for_fks session setting; causes foreign key operations to use lookup joins when possible This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.primary_region string "if not empty, all databases created without a PRIMARY REGION will implicitly have the given PRIMARY REGION This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.reorder_joins_limit integer 8 "default number of joins to reorder This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.require_explicit_primary_keys.enabled boolean false "default value for requiring explicit primary keys in CREATE TABLE statements This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.results_buffer.size byte size 16 KiB "default size of the buffer that accumulates results for a statement or a batch of statements before they are sent to the client. This can be overridden on an individual connection with the 'results_buffer_size' parameter. Note that auto-retries generally only happen while no results have been delivered to the client, so reducing this size can increase the number of retriable errors a client receives. On the other hand, increasing the buffer size can increase the delay until the client receives the first result row. Updating the setting only affects new connections. Setting to 0 disables any buffering. This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.serial_normalization enumeration rowid "default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2, sql_sequence_cached = 3, unordered_rowid = 4] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.statement_timeout duration 0s "default value for the statement_timeout; default value for the statement_timeout session setting; controls the duration a query is permitted to run before it is canceled; if set to 0, there is no timeout This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.stub_catalog_tables.enabled boolean true "default value for stub_catalog_tables session setting This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.super_regions.enabled boolean false "default value for enable_super_regions; allows for the usage of super regions This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.transaction_rows_read_err integer 0 "the limit for the number of rows read by a SQL transaction which - once exceeded - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.transaction_rows_read_log integer 0 "the threshold for the number of rows read by a SQL transaction which - once exceeded - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.transaction_rows_written_err integer 0 "the limit for the number of rows written by a SQL transaction which - once exceeded - will fail the transaction (or will trigger a logging event to SQL_INTERNAL_PERF for internal transactions); use 0 to disable This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.transaction_rows_written_log integer 0 "the threshold for the number of rows written by a SQL transaction which - once exceeded - will trigger a logging event to SQL_PERF (or SQL_INTERNAL_PERF for internal transactions); use 0 to disable This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.use_declarative_schema_changer enumeration on "default value for use_declarative_schema_changer session setting;disables new schema changer by default [off = 0, on = 1, unsafe = 2, unsafe_always = 3] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.vectorize enumeration on "default vectorize mode [on = 0, on = 2, experimental_always = 3, off = 4] This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application sql.defaults.zigzag_join.enabled boolean false "default value for enable_zigzag_join session setting; disallows use of zig-zag join by default This cluster setting is being kept to preserve backwards-compatibility. -This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw -sql.distsql.temp_storage.workmem byte size 64 MiB maximum amount of memory in bytes a processor can use before falling back to temp storage tenant-rw -sql.guardrails.max_row_size_err byte size 512 MiB 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 tenant-rw -sql.guardrails.max_row_size_log byte size 64 MiB maximum size of row (or column family if multiple column families are in use) that SQL can 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 tenant-rw -sql.hash_sharded_range_pre_split.max integer 16 max pre-split ranges to have when adding hash sharded index to an existing table tenant-rw -sql.insights.anomaly_detection.enabled boolean true enable per-fingerprint latency recording and anomaly detection tenant-rw -sql.insights.anomaly_detection.latency_threshold duration 50ms statements must surpass this threshold to trigger anomaly detection and identification tenant-rw -sql.insights.anomaly_detection.memory_limit byte size 1.0 MiB the maximum amount of memory allowed for tracking statement latencies tenant-rw -sql.insights.execution_insights_capacity integer 1000 the size of the per-node store of execution insights tenant-rw -sql.insights.high_retry_count.threshold integer 10 the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem tenant-rw -sql.insights.latency_threshold duration 100ms amount of time after which an executing statement is considered slow. Use 0 to disable. tenant-rw -sql.log.slow_query.experimental_full_table_scans.enabled boolean false when set to true, statements that perform a full table/index scan will be logged to the 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. tenant-rw -sql.log.slow_query.internal_queries.enabled boolean false when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. tenant-rw -sql.log.slow_query.latency_threshold duration 0s when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node tenant-rw -sql.log.user_audit string user/role-based audit logging configuration. An enterprise license is required for this cluster setting to take effect. tenant-rw -sql.log.user_audit.reduced_config.enabled boolean false enables logic to compute a reduced audit configuration, computing the audit configuration only once at session start instead of at each SQL event. The tradeoff with the increase in performance (~5%), is that changes to the audit configuration (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. tenant-rw -sql.metrics.index_usage_stats.enabled boolean true collect per index usage statistics tenant-rw -sql.metrics.max_mem_reported_stmt_fingerprints integer 100000 the maximum number of reported statement fingerprints stored in memory tenant-rw -sql.metrics.max_mem_reported_txn_fingerprints integer 100000 the maximum number of reported transaction fingerprints stored in memory tenant-rw -sql.metrics.max_mem_stmt_fingerprints integer 100000 the maximum number of statement fingerprints stored in memory tenant-rw -sql.metrics.max_mem_txn_fingerprints integer 100000 the maximum number of transaction fingerprints stored in memory tenant-rw -sql.metrics.statement_details.dump_to_logs.enabled boolean false dump collected statement statistics to node logs when periodically cleared tenant-rw -sql.metrics.statement_details.enabled boolean true collect per-statement query statistics tenant-rw -sql.metrics.statement_details.gateway_node.enabled boolean false save the gateway node for each statement fingerprint. If false, the value will be stored as 0. tenant-rw -sql.metrics.statement_details.index_recommendation_collection.enabled boolean true generate an index recommendation for each fingerprint ID tenant-rw -sql.metrics.statement_details.max_mem_reported_idx_recommendations integer 5000 the maximum number of reported index recommendation info stored in memory tenant-rw -sql.metrics.statement_details.plan_collection.enabled boolean false periodically save a logical plan for each fingerprint tenant-rw -sql.metrics.statement_details.plan_collection.period duration 5m0s the time until a new logical plan is collected tenant-rw -sql.metrics.statement_details.threshold duration 0s minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected. tenant-rw -sql.metrics.transaction_details.enabled boolean true collect per-application transaction statistics tenant-rw -sql.multiple_modifications_of_table.enabled boolean false if true, allow statements containing multiple INSERT ON CONFLICT, UPSERT, UPDATE, or DELETE subqueries modifying the same table, at the risk of data corruption if the same row is modified multiple times by a single statement (multiple INSERT subqueries without ON CONFLICT cannot cause corruption and are always allowed) tenant-rw -sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region tenant-rw -sql.notices.enabled boolean true enable notices in the server/client protocol being sent tenant-rw -sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false 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 tenant-rw -sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job tenant-ro -sql.show_ranges_deprecated_behavior.enabled boolean false if set, SHOW RANGES and crdb_internal.ranges{_no_leases} behave with deprecated pre-v23.1 semantics. NB: the new SHOW RANGES interface has richer WITH options than pre-v23.1 SHOW RANGES. tenant-rw -sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators tenant-rw -sql.stats.activity.persisted_rows.max integer 200000 maximum number of rows of statement and transaction activity that will be persisted in the system tables tenant-rw -sql.stats.automatic_collection.enabled boolean true automatic statistics collection mode tenant-rw -sql.stats.automatic_collection.fraction_stale_rows float 0.2 target fraction of stale rows per table that will trigger a statistics refresh tenant-rw -sql.stats.automatic_collection.min_stale_rows integer 500 target minimum number of stale rows per table that will trigger a statistics refresh tenant-rw -sql.stats.cleanup.recurrence string @hourly cron-tab recurrence for SQL Stats cleanup job tenant-rw -sql.stats.flush.enabled boolean true if set, SQL execution statistics are periodically flushed to disk tenant-rw -sql.stats.flush.interval duration 10m0s the interval at which SQL execution statistics are flushed to disk, this value must be less than or equal to 1 hour tenant-rw -sql.stats.forecasts.enabled boolean true when true, enables generation of statistics forecasts by default for all tables tenant-rw -sql.stats.histogram_buckets.count integer 200 maximum number of histogram buckets to build during table statistics collection tenant-rw -sql.stats.histogram_collection.enabled boolean true histogram collection mode tenant-rw -sql.stats.histogram_samples.count integer 10000 number of rows sampled for histogram construction during table statistics collection tenant-rw -sql.stats.multi_column_collection.enabled boolean true multi-column statistics collection mode tenant-rw -sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum retention period for table statistics collected on non-default columns tenant-rw -sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables before compaction begins tenant-rw -sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job tenant-rw -sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request tenant-rw -sql.stats.response.show_internal.enabled boolean false controls if statistics for internal executions should be returned by the CombinedStatements and if internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display statistics on the SQL Activity pages tenant-rw -sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer tenant-rw -sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables tenant-rw -sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel tenant-rw -sql.telemetry.query_sampling.internal.enabled boolean false when set to true, internal queries will be sampled in telemetry logging tenant-rw -sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up orphaned temporary objects tenant-rw -sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up tenant-rw -sql.log.all_statements.enabled boolean false set to true to enable logging of all executed statements tenant-rw -sql.trace.session_eventlog.enabled boolean false set to true to enable session tracing; note that enabling this may have a negative performance impact tenant-rw -sql.trace.stmt.enable_threshold duration 0s enables tracing on all statements; statements executing for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; this setting applies to individual statements within a transaction and is therefore finer-grained than sql.trace.txn.enable_threshold tenant-rw -sql.trace.txn.enable_threshold duration 0s enables tracing on all transactions; transactions open for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; 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) tenant-rw -sql.ttl.default_delete_batch_size integer 100 default amount of rows to delete in a single query during a TTL job tenant-rw -sql.ttl.default_delete_rate_limit integer 0 default delete rate limit (rows per second) per node for each TTL job. Use 0 to signify no rate limit. tenant-rw -sql.ttl.default_select_batch_size integer 500 default amount of rows to select in a single query during a TTL job tenant-rw -sql.ttl.job.enabled boolean true whether the TTL job is enabled tenant-rw -sql.txn.read_committed_syntax.enabled boolean false set to true to allow transactions to use the READ COMMITTED isolation level if specified by BEGIN/SET commands tenant-rw -sql.txn_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored tenant-rw -storage.max_sync_duration duration 20s maximum duration for disk operations; any operations that take longer than this setting trigger a warning log entry or process crash tenant-ro -storage.max_sync_duration.fatal.enabled boolean true if true, fatal the process when a disk operation exceeds storage.max_sync_duration tenant-rw -storage.value_blocks.enabled boolean true set to true to enable writing of value blocks in sstables tenant-rw -timeseries.storage.enabled boolean true if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere tenant-rw -timeseries.storage.resolution_10s.ttl duration 240h0m0s the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion. tenant-rw -timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion. tenant-rw -trace.debug_http_endpoint.enabled boolean false if set, traces for recent requests can be seen at https:///debug/requests tenant-rw -trace.jaeger.agent string 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. tenant-rw -trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. tenant-rw -trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured tenant-rw -trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez tenant-rw -trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. tenant-rw -ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] tenant-rw -version version 1000023.1-26 set the active cluster version in the format '.' tenant-rw +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" application +sql.distsql.temp_storage.workmem byte size 64 MiB maximum amount of memory in bytes a processor can use before falling back to temp storage application +sql.guardrails.max_row_size_err byte size 512 MiB 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 application +sql.guardrails.max_row_size_log byte size 64 MiB maximum size of row (or column family if multiple column families are in use) that SQL can 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 application +sql.hash_sharded_range_pre_split.max integer 16 max pre-split ranges to have when adding hash sharded index to an existing table application +sql.insights.anomaly_detection.enabled boolean true enable per-fingerprint latency recording and anomaly detection application +sql.insights.anomaly_detection.latency_threshold duration 50ms statements must surpass this threshold to trigger anomaly detection and identification application +sql.insights.anomaly_detection.memory_limit byte size 1.0 MiB the maximum amount of memory allowed for tracking statement latencies application +sql.insights.execution_insights_capacity integer 1000 the size of the per-node store of execution insights application +sql.insights.high_retry_count.threshold integer 10 the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem application +sql.insights.latency_threshold duration 100ms amount of time after which an executing statement is considered slow. Use 0 to disable. application +sql.log.slow_query.experimental_full_table_scans.enabled boolean false when set to true, statements that perform a full table/index scan will be logged to the 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. application +sql.log.slow_query.internal_queries.enabled boolean false when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. application +sql.log.slow_query.latency_threshold duration 0s when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node application +sql.log.user_audit string user/role-based audit logging configuration. An enterprise license is required for this cluster setting to take effect. application +sql.log.user_audit.reduced_config.enabled boolean false enables logic to compute a reduced audit configuration, computing the audit configuration only once at session start instead of at each SQL event. The tradeoff with the increase in performance (~5%), is that changes to the audit configuration (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. application +sql.metrics.index_usage_stats.enabled boolean true collect per index usage statistics application +sql.metrics.max_mem_reported_stmt_fingerprints integer 100000 the maximum number of reported statement fingerprints stored in memory application +sql.metrics.max_mem_reported_txn_fingerprints integer 100000 the maximum number of reported transaction fingerprints stored in memory application +sql.metrics.max_mem_stmt_fingerprints integer 100000 the maximum number of statement fingerprints stored in memory application +sql.metrics.max_mem_txn_fingerprints integer 100000 the maximum number of transaction fingerprints stored in memory application +sql.metrics.statement_details.dump_to_logs.enabled boolean false dump collected statement statistics to node logs when periodically cleared application +sql.metrics.statement_details.enabled boolean true collect per-statement query statistics application +sql.metrics.statement_details.gateway_node.enabled boolean false save the gateway node for each statement fingerprint. If false, the value will be stored as 0. application +sql.metrics.statement_details.index_recommendation_collection.enabled boolean true generate an index recommendation for each fingerprint ID application +sql.metrics.statement_details.max_mem_reported_idx_recommendations integer 5000 the maximum number of reported index recommendation info stored in memory application +sql.metrics.statement_details.plan_collection.enabled boolean false periodically save a logical plan for each fingerprint application +sql.metrics.statement_details.plan_collection.period duration 5m0s the time until a new logical plan is collected application +sql.metrics.statement_details.threshold duration 0s minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected. application +sql.metrics.transaction_details.enabled boolean true collect per-application transaction statistics application +sql.multiple_modifications_of_table.enabled boolean false if true, allow statements containing multiple INSERT ON CONFLICT, UPSERT, UPDATE, or DELETE subqueries modifying the same table, at the risk of data corruption if the same row is modified multiple times by a single statement (multiple INSERT subqueries without ON CONFLICT cannot cause corruption and are always allowed) application +sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region application +sql.notices.enabled boolean true enable notices in the server/client protocol being sent application +sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false 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 application +sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job system-visible +sql.show_ranges_deprecated_behavior.enabled boolean false if set, SHOW RANGES and crdb_internal.ranges{_no_leases} behave with deprecated pre-v23.1 semantics. NB: the new SHOW RANGES interface has richer WITH options than pre-v23.1 SHOW RANGES. application +sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators application +sql.stats.activity.persisted_rows.max integer 200000 maximum number of rows of statement and transaction activity that will be persisted in the system tables application +sql.stats.automatic_collection.enabled boolean true automatic statistics collection mode application +sql.stats.automatic_collection.fraction_stale_rows float 0.2 target fraction of stale rows per table that will trigger a statistics refresh application +sql.stats.automatic_collection.min_stale_rows integer 500 target minimum number of stale rows per table that will trigger a statistics refresh application +sql.stats.cleanup.recurrence string @hourly cron-tab recurrence for SQL Stats cleanup job application +sql.stats.flush.enabled boolean true if set, SQL execution statistics are periodically flushed to disk application +sql.stats.flush.interval duration 10m0s the interval at which SQL execution statistics are flushed to disk, this value must be less than or equal to 1 hour application +sql.stats.forecasts.enabled boolean true when true, enables generation of statistics forecasts by default for all tables application +sql.stats.histogram_buckets.count integer 200 maximum number of histogram buckets to build during table statistics collection application +sql.stats.histogram_collection.enabled boolean true histogram collection mode application +sql.stats.histogram_samples.count integer 10000 number of rows sampled for histogram construction during table statistics collection application +sql.stats.multi_column_collection.enabled boolean true multi-column statistics collection mode application +sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum retention period for table statistics collected on non-default columns application +sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables before compaction begins application +sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job application +sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request application +sql.stats.response.show_internal.enabled boolean false controls if statistics for internal executions should be returned by the CombinedStatements and if internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display statistics on the SQL Activity pages application +sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer application +sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables application +sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel application +sql.telemetry.query_sampling.internal.enabled boolean false when set to true, internal queries will be sampled in telemetry logging application +sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up orphaned temporary objects application +sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up application +sql.log.all_statements.enabled boolean false set to true to enable logging of all executed statements application +sql.trace.session_eventlog.enabled boolean false set to true to enable session tracing; note that enabling this may have a negative performance impact application +sql.trace.stmt.enable_threshold duration 0s enables tracing on all statements; statements executing for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; this setting applies to individual statements within a transaction and is therefore finer-grained than sql.trace.txn.enable_threshold application +sql.trace.txn.enable_threshold duration 0s enables tracing on all transactions; transactions open for longer than this duration will have their trace logged (set to 0 to disable); note that enabling this may have a negative performance impact; 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) application +sql.ttl.default_delete_batch_size integer 100 default amount of rows to delete in a single query during a TTL job application +sql.ttl.default_delete_rate_limit integer 0 default delete rate limit (rows per second) per node for each TTL job. Use 0 to signify no rate limit. application +sql.ttl.default_select_batch_size integer 500 default amount of rows to select in a single query during a TTL job application +sql.ttl.job.enabled boolean true whether the TTL job is enabled application +sql.txn.read_committed_syntax.enabled boolean false set to true to allow transactions to use the READ COMMITTED isolation level if specified by BEGIN/SET commands application +sql.txn_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored application +storage.max_sync_duration duration 20s maximum duration for disk operations; any operations that take longer than this setting trigger a warning log entry or process crash system-visible +storage.max_sync_duration.fatal.enabled boolean true if true, fatal the process when a disk operation exceeds storage.max_sync_duration application +storage.value_blocks.enabled boolean true set to true to enable writing of value blocks in sstables application +timeseries.storage.enabled boolean true if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere application +timeseries.storage.resolution_10s.ttl duration 240h0m0s the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion. application +timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion. application +trace.debug_http_endpoint.enabled boolean false if set, traces for recent requests can be seen at https:///debug/requests application +trace.jaeger.agent string 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. application +trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. application +trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured application +trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez application +trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. application +ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] application +version version 1000023.1-26 set the active cluster version in the format '.' application diff --git a/pkg/ccl/auditloggingccl/audit_log_config.go b/pkg/ccl/auditloggingccl/audit_log_config.go index 732e6e6a271b..f6b83936806b 100644 --- a/pkg/ccl/auditloggingccl/audit_log_config.go +++ b/pkg/ccl/auditloggingccl/audit_log_config.go @@ -24,7 +24,7 @@ const auditConfigDefaultValue = "" // UserAuditLogConfig is a cluster setting that takes a user/role-based audit configuration. var UserAuditLogConfig = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.log.user_audit", "user/role-based audit logging configuration. An enterprise license is required for this cluster setting to take effect.", auditConfigDefaultValue, @@ -39,7 +39,7 @@ var UserAuditLogConfig = settings.RegisterStringSetting( // reflected within session. Users will need to start a new session to see these changes in their // auditing behaviour. var UserAuditEnableReducedConfig = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.log.user_audit.reduced_config.enabled", "enables logic to compute a reduced audit configuration, computing the audit "+ "configuration only once at session start instead of at each SQL event. The tradeoff "+ diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index a61662830368..87beb1f5748f 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -71,7 +71,7 @@ import ( // BackupCheckpointInterval is the interval at which backup progress is saved // to durable storage. var BackupCheckpointInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.checkpoint_interval", "the minimum time between writing progress checkpoints during a backup", time.Minute) diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 9e2d1576b078..76d3f222c39c 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -83,7 +83,7 @@ type tableAndIndex struct { // featureBackupEnabled is used to enable and disable the BACKUP feature. var featureBackupEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.backup.enabled", "set to true to enable backups, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index 402a29b25576..e7770499c6d6 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -52,35 +52,35 @@ var backupOutputTypes = []*types.T{} var ( priorityAfter = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.read_with_priority_after", "amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads", time.Minute, settings.NonNegativeDuration, settings.WithPublic) delayPerAttempt = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.read_retry_delay", "amount of time since the read-as-of time, per-prior attempt, to wait before making another attempt", time.Second*5, settings.NonNegativeDuration, ) timeoutPerAttempt = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.read_timeout", "amount of time after which a read attempt is considered timed out, which causes the backup to fail", time.Minute*5, settings.NonNegativeDuration, settings.WithPublic) targetFileSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.file_size", "target size for individual data files produced during BACKUP", 128<<20, settings.WithPublic) splitKeysOnTimestamps = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.split_keys_on_timestamps", "split backup data on timestamps when writing revision history", true, @@ -88,7 +88,7 @@ var ( ) sendExportRequestWithVerboseTracing = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.export_request_verbose_tracing", "send each export request with a verbose tracing span", util.ConstantWithMetamorphicTestBool("export_request_verbose_tracing", false), diff --git a/pkg/ccl/backupccl/backup_tenant_test.go b/pkg/ccl/backupccl/backup_tenant_test.go index 17f11b528221..8fb2469af678 100644 --- a/pkg/ccl/backupccl/backup_tenant_test.go +++ b/pkg/ccl/backupccl/backup_tenant_test.go @@ -160,7 +160,7 @@ func TestTenantBackupMultiRegionDatabases(t *testing.T) { { // Flip the tenant-read only cluster setting; ensure database can be restored // on the system tenant but not on the secondary tenant. - setAndWaitForTenantReadOnlyClusterSetting( + setAndWaitForSystemVisibleClusterSetting( t, sql.SecondaryTenantsMultiRegionAbstractionsEnabledSettingName, sqlDB, @@ -184,7 +184,7 @@ func TestTenantBackupMultiRegionDatabases(t *testing.T) { { // Flip the tenant-read only cluster setting back to true and ensure the // restore succeeds. - setAndWaitForTenantReadOnlyClusterSetting( + setAndWaitForSystemVisibleClusterSetting( t, sql.SecondaryTenantsMultiRegionAbstractionsEnabledSettingName, sqlDB, @@ -205,7 +205,7 @@ func TestTenantBackupMultiRegionDatabases(t *testing.T) { "SET CLUSTER SETTING %s = 'us-east1'", sql.DefaultPrimaryRegionClusterSettingName, ), ) - setAndWaitForTenantReadOnlyClusterSetting( + setAndWaitForSystemVisibleClusterSetting( t, sql.SecondaryTenantsMultiRegionAbstractionsEnabledSettingName, sqlDB, diff --git a/pkg/ccl/backupccl/backupbase/settings.go b/pkg/ccl/backupccl/backupbase/settings.go index 922ac3e2e79a..6ee44bc1c51a 100644 --- a/pkg/ccl/backupccl/backupbase/settings.go +++ b/pkg/ccl/backupccl/backupbase/settings.go @@ -24,7 +24,7 @@ var ( // SmallFileBuffer is the size limit used when buffering backup files before // merging them. SmallFileBuffer = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.merge_file_buffer_size", "size limit used when buffering backup files before merging them", int64(defaultSmallFileBuffer), diff --git a/pkg/ccl/backupccl/backupdest/backup_destination.go b/pkg/ccl/backupccl/backupdest/backup_destination.go index 707610869a7f..3e2ffc48ed0e 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination.go @@ -59,7 +59,7 @@ var backupPathRE = regexp.MustCompile("^/?[^\\/]+/[^\\/]+/[^\\/]+/" + backupbase // specified subdirectory if no backup already exists at that subdirectory. As // of 22.1, this feature is default disabled, and will be totally disabled by 22.2. var featureFullBackupUserSubdir = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.backup.deprecated_full_backup_with_subdir.enabled", "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", diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index 1a289bb43fc3..b35604bcd571 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -91,7 +91,7 @@ const ( // WriteMetadataSST controls if we write the experimental new format BACKUP // metadata file. var WriteMetadataSST = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulkio.write_metadata_sst.enabled", "write experimental new format BACKUP metadata file", util.ConstantWithMetamorphicTestBool("write-metadata-sst", false), @@ -102,7 +102,7 @@ var WriteMetadataSST = settings.RegisterBoolSetting( // descriptors. This new format of metadata is written in addition to the // `BACKUP_MANIFEST` file, and is expected to be its replacement in the future. var WriteMetadataWithExternalSSTsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "backup.write_metadata_with_external_ssts.enabled", "write BACKUP metadata along with supporting SST files", util.ConstantWithMetamorphicTestBool("backup.write_metadata_with_external_ssts.enabled", true), diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 19cb12c67be4..c66abeb7cc7a 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -58,7 +58,7 @@ var scheduledBackupOptionExpectValues = map[string]exprutil.KVStringOptValidate{ // scheduledBackupGCProtectionEnabled is used to enable and disable the chaining // of protected timestamps amongst scheduled backups. var scheduledBackupGCProtectionEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "schedules.backup.gc_protection.enabled", "enable chaining of GC protection across backups run as part of a schedule", true, /* defaultValue */ diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index fa56c1e27e53..c61b2f7d13ad 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -140,7 +140,7 @@ var defaultNumWorkers = util.ConstantWithMetamorphicTestRange( // The maximum is not enforced since if the maximum is reduced in the future that // may cause the cluster setting to fail. var numRestoreWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_io_write.restore_node_concurrency", fmt.Sprintf("the number of workers processing a restore per job per node; maximum %d", maxConcurrentRestoreWorkers), @@ -153,7 +153,7 @@ var numRestoreWorkers = settings.RegisterIntSetting( // and the limit determined by restorePerProcessorMemoryLimitSQLFraction // and --max-sql-memory. var restorePerProcessorMemoryLimit = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.per_processor_memory_limit", "limit on the amount of memory that can be used by a restore processor", 1<<30, // 1 GiB @@ -162,7 +162,7 @@ var restorePerProcessorMemoryLimit = settings.RegisterByteSizeSetting( // restorePerProcessorMemoryLimitSQLFraction is the maximum percentage of the // SQL memory pool that could be used by a restoreDataProcessor. var restorePerProcessorMemoryLimitSQLFraction = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.per_processor_memory_limit_sql_fraction", "limit on the amount of memory that can be used by a restore processor as a fraction of max SQL memory", 0.5, diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 8405f6a596ca..3f579631f67c 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -86,7 +86,7 @@ import ( const restoreStatsInsertBatchSize = 10 var useSimpleImportSpans = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.use_simple_import_spans", "if set to true, restore will generate its import spans using the makeSimpleImportSpans algorithm", false, @@ -94,7 +94,7 @@ var useSimpleImportSpans = settings.RegisterBoolSetting( ) var restoreStatsInsertionConcurrency = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.insert_stats_workers", "number of concurrent workers that will restore backed up table statistics", 5, diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 95b58edf0634..a50dd0d54769 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -93,7 +93,7 @@ var allowedDebugPauseOnValues = map[string]struct{}{ // featureRestoreEnabled is used to enable and disable the RESTORE feature. var featureRestoreEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.restore.enabled", "set to true to enable restore, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index cbb7cbcb63ae..b36fb8ecdda3 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -36,14 +36,14 @@ import ( ) var replanRestoreThreshold = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.replan_flow_threshold", "fraction of initial flow instances that would be added or updated above which a RESTORE execution plan is restarted from the last checkpoint (0=disabled)", 0.0, ) var replanRestoreFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.replan_flow_frequency", "frequency at which RESTORE checks to see if restarting would change its updates its physical execution plan", time.Minute*2, @@ -51,7 +51,7 @@ var replanRestoreFrequency = settings.RegisterDurationSetting( ) var memoryMonitorSSTs = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.restore.memory_monitor_ssts", "if true, restore will limit number of simultaneously open SSTs to keep memory usage under the configured memory fraction", false, diff --git a/pkg/ccl/backupccl/restore_progress.go b/pkg/ccl/backupccl/restore_progress.go index 9f65e58c112b..bf929a6bff28 100644 --- a/pkg/ccl/backupccl/restore_progress.go +++ b/pkg/ccl/backupccl/restore_progress.go @@ -28,7 +28,7 @@ import ( // to the checkpoint record. The default is set using the same reasoning as // changefeed.frontier_checkpoint_max_bytes. var restoreCheckpointMaxBytes = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "restore.frontier_checkpoint_max_bytes", "controls the maximum size of the restore checkpoint frontier as a the sum of the (span,"+ "timestamp) tuples", diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index ba92f472b892..4644f3d1eaf6 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -51,7 +51,7 @@ func (ie intervalSpan) Range() interval.Range { // tracked within restore spans, this could become dynamic and much larger (e.g. // totalSize/numNodes*someConstant). var targetRestoreSpanSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "backup.restore_span.target_size", "target size to which base spans of a restore are merged to produce a restore span (0 disables)", 384<<20, diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 00327d259ded..556679419006 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -390,7 +390,7 @@ func thresholdFromTrace(t *testing.T, traceString string) hlc.Timestamp { return thresh } -func setAndWaitForTenantReadOnlyClusterSetting( +func setAndWaitForSystemVisibleClusterSetting( t *testing.T, settingName settings.SettingName, systemTenantRunner *sqlutils.SQLRunner, diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index 9de3de7abfa6..3f918fabc3d2 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -316,7 +316,7 @@ func startDistChangefeed( } var enableBalancedRangeDistribution = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.balance_range_distribution.enable", "if enabled, the ranges are balanced equally among all nodes", util.ConstantWithMetamorphicTestBool( @@ -508,7 +508,7 @@ func (w *changefeedResultWriter) Err() error { } var rebalanceThreshold = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.balance_range_distribution.sensitivity", "rebalance if the number of ranges on a node exceeds the average by this fraction", 0.05, diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 109ed63aa8d8..4e794198c3a5 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -578,7 +578,7 @@ func (ca *changeAggregator) close() { } var aggregatorHeartbeatFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.aggregator.heartbeat", "changefeed aggregator will emit a heartbeat message to the coordinator with this frequency; 0 disables. "+ "The setting value should be <=1/2 of server.shutdown.jobs_wait period", diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index 12a9f3d5e4b6..12f6adf3f966 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -63,7 +63,7 @@ import ( // featureChangefeedEnabled is used to enable and disable the CHANGEFEED feature. var featureChangefeedEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.changefeed.enabled", "set to true to enable changefeeds, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index 63aa89b04003..ec9d9586c7ab 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -22,7 +22,7 @@ import ( // NB: The more generic name of this setting precedes its current // interpretation. It used to control additional polling rates. var TableDescriptorPollInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.experimental_poll_interval", "polling interval for the table descriptors", 1*time.Second, @@ -44,7 +44,7 @@ func TestingSetDefaultMinCheckpointFrequency(f time.Duration) func() { // PerChangefeedMemLimit controls how much data can be buffered by // a single changefeed. var PerChangefeedMemLimit = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.memory.per_changefeed_limit", "controls amount of data that can be buffered per changefeed", 1<<29, // 512MiB @@ -52,7 +52,7 @@ var PerChangefeedMemLimit = settings.RegisterByteSizeSetting( // SlowSpanLogThreshold controls when we will log slow spans. var SlowSpanLogThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.slow_span_log_threshold", "a changefeed will log spans with resolved timestamps this far behind the current wall-clock time; if 0, a default value is calculated based on other cluster settings", 0, @@ -62,7 +62,7 @@ var SlowSpanLogThreshold = settings.RegisterDurationSetting( // IdleTimeout controls how long the changefeed will wait for a new KV being // emitted before marking itself as idle. var IdleTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.idle_timeout", "a changefeed will mark itself idle if no changes have been emitted for greater than this duration; if 0, the changefeed will never be marked idle", 10*time.Minute, @@ -72,7 +72,7 @@ var IdleTimeout = settings.RegisterDurationSetting( // FrontierCheckpointFrequency controls the frequency of frontier checkpoints. var FrontierCheckpointFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.frontier_checkpoint_frequency", "controls the frequency with which span level checkpoints will be written; if 0, disabled", 10*time.Minute, @@ -83,7 +83,7 @@ var FrontierCheckpointFrequency = settings.RegisterDurationSetting( // mark is allowed to lag behind the leading edge of the frontier before we // begin to attempt checkpointing spans above the high-water mark var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.frontier_highwater_lag_checkpoint_threshold", "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, @@ -105,7 +105,7 @@ var FrontierHighwaterLagCheckpointThreshold = settings.RegisterDurationSetting( // Therefore, we should write at most 6 MB of checkpoint/hour; OR, based on the default // FrontierCheckpointFrequency setting, 1 MB per checkpoint. var FrontierCheckpointMaxBytes = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.frontier_checkpoint_max_bytes", "controls the maximum size of the checkpoint as a total size of key bytes", 1<<20, // 1 MiB @@ -115,7 +115,7 @@ var FrontierCheckpointMaxBytes = settings.RegisterByteSizeSetting( // Scan requests are issued when changefeed performs the backfill. // If set to 0, a reasonable default will be chosen. var ScanRequestLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.backfill.concurrent_scan_requests", "number of concurrent scan requests per node issued during a backfill", 0, @@ -126,7 +126,7 @@ var ScanRequestLimit = settings.RegisterIntSetting( // TODO(cdc,yevgeniy,irfansharif): 16 MiB is too large for "elastic" work such // as this; reduce the default. Evaluate this as part of #90089. var ScanRequestSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.backfill.scan_request_size", "the maximum number of bytes returned by each scan request", 1<<19, // 1/2 MiB @@ -151,7 +151,7 @@ type SinkThrottleConfig struct { // NodeSinkThrottleConfig is the node wide throttling configuration for changefeeds. var NodeSinkThrottleConfig = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.node_throttle_config", "specifies node level throttling configuration for all changefeeeds", "", @@ -171,7 +171,7 @@ func validateSinkThrottleConfig(values *settings.Values, configStr string) error // MinHighWaterMarkCheckpointAdvance specifies the minimum amount of time the // changefeed high water mark must advance for it to be eligible for checkpointing. var MinHighWaterMarkCheckpointAdvance = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.min_highwater_advance", "minimum amount of time the changefeed high water mark must advance "+ "for it to be eligible for checkpointing; Default of 0 will checkpoint every time frontier "+ @@ -187,7 +187,7 @@ var MinHighWaterMarkCheckpointAdvance = settings.RegisterDurationSetting( // with complex schemes to accurately measure and adjust current memory usage, // we'll request the amount of memory multiplied by this fudge factor. var EventMemoryMultiplier = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.event_memory_multiplier", "the amount of memory required to process an event is multiplied by this factor", 3, @@ -196,7 +196,7 @@ var EventMemoryMultiplier = settings.RegisterFloatSetting( // ProtectTimestampInterval controls the frequency of protected timestamp record updates var ProtectTimestampInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.protect_timestamp_interval", "controls how often the changefeed forwards its protected timestamp to the resolved timestamp", 10*time.Minute, @@ -205,7 +205,7 @@ var ProtectTimestampInterval = settings.RegisterDurationSetting( // MaxProtectedTimestampAge controls the frequency of protected timestamp record updates var MaxProtectedTimestampAge = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.protect_timestamp.max_age", "fail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expiration", 4*24*time.Hour, @@ -214,7 +214,7 @@ var MaxProtectedTimestampAge = settings.RegisterDurationSetting( // BatchReductionRetryEnabled enables the temporary reduction of batch sizes upon kafka message too large errors var BatchReductionRetryEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, @@ -223,7 +223,7 @@ var BatchReductionRetryEnabled = settings.RegisterBoolSetting( // UseMuxRangeFeed enables the use of MuxRangeFeed RPC. var UseMuxRangeFeed = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.mux_rangefeed.enabled", "if true, changefeed uses multiplexing rangefeed RPC", util.ConstantWithMetamorphicTestBool("changefeed.mux_rangefeed.enabled", false), @@ -232,7 +232,7 @@ var UseMuxRangeFeed = settings.RegisterBoolSetting( // EventConsumerWorkers specifies the maximum number of workers to use when // processing events. var EventConsumerWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.event_consumer_workers", "the number of workers to use when processing events: <0 disables, "+ "0 assigns a reasonable default, >0 assigns the setting value. for experimental/core "+ @@ -242,7 +242,7 @@ var EventConsumerWorkers = settings.RegisterIntSetting( // EventConsumerWorkerQueueSize specifies the maximum number of events a worker buffer. var EventConsumerWorkerQueueSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.event_consumer_worker_queue_size", "if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events "+ "which a worker can buffer", @@ -255,7 +255,7 @@ var EventConsumerWorkerQueueSize = settings.RegisterIntSetting( // For example, every N milliseconds of CPU work, request N more // milliseconds of CPU time. var EventConsumerPacerRequestSize = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.cpu.per_event_consumer_worker_allocation", "an event consumer worker will perform a blocking request for CPU time "+ "before consuming events. after fully utilizing this CPU time, it will "+ @@ -267,7 +267,7 @@ var EventConsumerPacerRequestSize = settings.RegisterDurationSetting( // PerEventElasticCPUControlEnabled determines whether changefeed event // processing integrates with elastic CPU control. var PerEventElasticCPUControlEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.cpu.per_event_elastic_control.enabled", "determines whether changefeed event processing integrates with elastic CPU control", true, @@ -276,7 +276,7 @@ var PerEventElasticCPUControlEnabled = settings.RegisterBoolSetting( // RequireExternalConnectionSink is used to restrict non-admins with the CHANGEFEED privilege // to create changefeeds to external connections only. var RequireExternalConnectionSink = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.permissions.require_external_connection_sink", "if enabled, this settings restricts users with the CHANGEFEED privilege"+ " to create changefeeds with external connection sinks only."+ @@ -288,7 +288,7 @@ var RequireExternalConnectionSink = settings.RegisterBoolSetting( // SinkIOWorkers controls the number of IO workers used by sinks that use // parallelIO to be able to send multiple requests in parallel. var SinkIOWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.sink_io_workers", "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", @@ -300,7 +300,7 @@ var SinkIOWorkers = settings.RegisterIntSetting( // example, every N milliseconds of CPU work, request N more milliseconds of CPU // time. var SinkPacerRequestSize = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.cpu.sink_encoding_allocation", "an event consumer worker will perform a blocking request for CPU time "+ "before consuming events. after fully utilizing this CPU time, it will "+ diff --git a/pkg/ccl/changefeedccl/compression.go b/pkg/ccl/changefeedccl/compression.go index a2be6f197948..1e30f010ea19 100644 --- a/pkg/ccl/changefeedccl/compression.go +++ b/pkg/ccl/changefeedccl/compression.go @@ -21,7 +21,7 @@ import ( ) var useFastGzip = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.fast_gzip.enabled", "use fast gzip implementation", util.ConstantWithMetamorphicTestBool( diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index b738cb1b0726..3c4ee3d7ab6e 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -727,7 +727,7 @@ func (tf *schemaFeed) validateDescriptor( } var highPriorityAfter = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go index a567f835fe92..b3cb36d30012 100644 --- a/pkg/ccl/changefeedccl/sink.go +++ b/pkg/ccl/changefeedccl/sink.go @@ -159,7 +159,7 @@ func getAndDialSink( // WebhookV2Enabled determines whether or not the refactored Webhook sink // or the deprecated sink should be used. var WebhookV2Enabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.new_webhook_sink_enabled", "if enabled, this setting enables a new implementation of the webhook sink"+ " that allows for a much higher throughput", @@ -171,7 +171,7 @@ var WebhookV2Enabled = settings.RegisterBoolSetting( // PubsubV2Enabled determines whether or not the refactored Webhook sink // or the deprecated sink should be used. var PubsubV2Enabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.new_pubsub_sink_enabled", "if enabled, this setting enables a new implementation of the pubsub sink"+ " that allows for a higher throughput", diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go index 50ada755ef39..f152f59f9058 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go @@ -697,7 +697,7 @@ func (s *cloudStorageSink) setDataFileTimestamp() { // enableAsyncFlush controls async flushing behavior for this sink. var enableAsyncFlush = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.cloudstorage.async_flush.enabled", "enable async flushing", true, diff --git a/pkg/ccl/changefeedccl/telemetry.go b/pkg/ccl/changefeedccl/telemetry.go index 77aca8496e15..e8df88f882dd 100644 --- a/pkg/ccl/changefeedccl/telemetry.go +++ b/pkg/ccl/changefeedccl/telemetry.go @@ -201,7 +201,7 @@ func (r *telemetryMetricsRecorder) recordSinkIOInflightChange(delta int64) { // ContinuousTelemetryInterval determines the interval at which each node emits telemetry events // during the lifespan of each enterprise changefeed. var ContinuousTelemetryInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "changefeed.telemetry.continuous_logging.interval", "determines the interval at which each node emits continuous telemetry events"+ " during the lifespan of every enterprise changefeed; setting a zero value disables", diff --git a/pkg/ccl/jwtauthccl/settings.go b/pkg/ccl/jwtauthccl/settings.go index fa90a50c6436..c4a31498fc0c 100644 --- a/pkg/ccl/jwtauthccl/settings.go +++ b/pkg/ccl/jwtauthccl/settings.go @@ -29,7 +29,7 @@ const ( // JWTAuthClaim sets the JWT claim that is parsed to get the username. var JWTAuthClaim = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, JWTAuthClaimSettingName, "sets the JWT claim that is parsed to get the username", "", @@ -38,7 +38,7 @@ var JWTAuthClaim = settings.RegisterStringSetting( // JWTAuthAudience sets accepted audience values for JWT logins over the SQL interface. var JWTAuthAudience = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, JWTAuthAudienceSettingName, "sets accepted audience values for JWT logins over the SQL interface", "", @@ -47,7 +47,7 @@ var JWTAuthAudience = settings.RegisterStringSetting( // JWTAuthEnabled enables or disabled JWT login over the SQL interface. var JWTAuthEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, JWTAuthEnabledSettingName, "enables or disabled JWT login for the SQL interface", false, @@ -56,7 +56,7 @@ var JWTAuthEnabled = settings.RegisterBoolSetting( // JWTAuthJWKS is the public key set for JWT logins over the SQL interface. var JWTAuthJWKS = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, JWTAuthJWKSSettingName, "sets the public key set for JWT logins over the SQL interface (JWKS format)", "{\"keys\":[]}", @@ -65,7 +65,7 @@ var JWTAuthJWKS = settings.RegisterStringSetting( // JWTAuthIssuers is the list of "issuer" values that are accepted for JWT logins over the SQL interface. var JWTAuthIssuers = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, 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", diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go index 31f899953e00..2d6236128c0d 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go @@ -36,7 +36,7 @@ import ( // measure of how long closed timestamp updates are supposed to take from the // leaseholder to the followers. var ClosedTimestampPropagationSlack = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.closed_timestamp.propagation_slack", "a conservative estimate of the amount of time expect for closed timestamps to "+ "propagate from a leaseholder to followers. This is taken into account by "+ diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_settings b/pkg/ccl/logictestccl/testdata/logic_test/tenant_settings index 8aab7bd3bcce..364cb21f5bcf 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/tenant_settings +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_settings @@ -108,7 +108,7 @@ SHOW CLUSTER SETTING sql.notices.enabled ---- true -# Verify that we disallow setting a TenantWritable setting that is overridden. +# Verify that we disallow setting a ApplicationLevel setting that is overridden. statement error cluster setting 'sql.notices.enabled' is currently overridden by the operator SET CLUSTER SETTING sql.notices.enabled = false @@ -138,7 +138,7 @@ SHOW CLUSTER SETTING sql.notices.enabled ---- true -# Verify that the tenant cannot modify TenantReadOnly settings. +# Verify that the tenant cannot modify SystemVisible settings. query T SHOW CLUSTER SETTING kv.protectedts.reconciliation.interval ---- diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go index 4ca59456b3f6..5d58d784b4d3 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go @@ -36,7 +36,7 @@ import ( // TargetPeriodSetting is exported for testing purposes. var TargetPeriodSetting = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_control_period", "target duration between token bucket requests (requires restart)", 10*time.Second, @@ -46,7 +46,7 @@ var TargetPeriodSetting = settings.RegisterDurationSetting( // CPUUsageAllowance is exported for testing purposes. var CPUUsageAllowance = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cpu_usage_allowance", "this much CPU usage per second is considered background usage and "+ "doesn't contribute to consumption; for example, if it is set to 10ms, "+ @@ -59,7 +59,7 @@ var CPUUsageAllowance = settings.RegisterDurationSetting( // ExternalIORUAccountingMode controls whether external ingress and // egress bytes are included in RU calculations. var ExternalIORUAccountingMode = *settings.RegisterStringSetting( - settings.TenantReadOnly, + settings.SystemVisible, "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), "+ diff --git a/pkg/ccl/multitenantccl/tenantcostserver/server.go b/pkg/ccl/multitenantccl/tenantcostserver/server.go index d53f110ac87c..63bd2098110b 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/server.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/server.go @@ -33,7 +33,7 @@ type instance struct { // // tenantcostclient.extendedReportingPeriodFactor. var instanceInactivity = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "tenant_usage_instance_inactivity", "server instances that have not reported consumption for longer than this value are cleaned up; "+ "should be at least four times higher than the tenant_cost_control.token_request_period of any tenant", diff --git a/pkg/ccl/oidcccl/settings.go b/pkg/ccl/oidcccl/settings.go index 918e04bbba89..9ca8a1b4f438 100644 --- a/pkg/ccl/oidcccl/settings.go +++ b/pkg/ccl/oidcccl/settings.go @@ -42,7 +42,7 @@ const ( // OIDCEnabled enables or disabled OIDC login for the DB Console. var OIDCEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCEnabledSettingName, "enables or disabled OIDC login for the DB Console", false, @@ -52,7 +52,7 @@ var OIDCEnabled = settings.RegisterBoolSetting( // OIDCClientID is the OIDC client id. var OIDCClientID = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCClientIDSettingName, "sets OIDC client id", "", @@ -62,7 +62,7 @@ var OIDCClientID = settings.RegisterStringSetting( // OIDCClientSecret is the OIDC client secret. var OIDCClientSecret = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCClientSecretSettingName, "sets OIDC client secret", "", @@ -171,7 +171,7 @@ func validateOIDCRedirectURL(values *settings.Values, s string) error { // where the cluster's region is not listed in the `redirect_urls` object, we // will use the required `default_url` callback URL. var OIDCRedirectURL = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, 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 "+ @@ -185,7 +185,7 @@ var OIDCRedirectURL = settings.RegisterStringSetting( // OIDCProviderURL is the location of the OIDC discovery document for the auth // provider. var OIDCProviderURL = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCProviderURLSettingName, "sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)", "", @@ -201,7 +201,7 @@ var OIDCProviderURL = settings.RegisterStringSetting( // OIDCScopes contains the list of scopes to request from the auth provider. var OIDCScopes = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCScopesSettingName, "sets OIDC scopes to include with authentication request "+ "(space delimited list of strings, required to start with `openid`)", @@ -218,7 +218,7 @@ var OIDCScopes = settings.RegisterStringSetting( // OIDCClaimJSONKey is the key of the claim to extract from the OIDC id_token. var OIDCClaimJSONKey = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCClaimJSONKeySettingName, "sets JSON key of principal to extract from payload after OIDC authentication completes "+ "(usually email or sid)", @@ -229,7 +229,7 @@ var OIDCClaimJSONKey = settings.RegisterStringSetting( // OIDCPrincipalRegex is a regular expression to apply to the OIDC id_token // claim value to conver it to a DB principal. var OIDCPrincipalRegex = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, 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)", @@ -247,7 +247,7 @@ var OIDCPrincipalRegex = settings.RegisterStringSetting( // OIDCButtonText is a string to display on the button in the DB Console to // login with OIDC. var OIDCButtonText = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCButtonTextSettingName, "text to show on button on DB Console login page to login with your OIDC provider "+ "(only shown if OIDC is enabled)", @@ -258,7 +258,7 @@ var OIDCButtonText = settings.RegisterStringSetting( // OIDCAutoLogin is a boolean that enables automatic redirection to OIDC auth in // the DB Console. var OIDCAutoLogin = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.oidc_authentication.autologin", "if true, logged-out visitors to the DB Console will be "+ "automatically redirected to the OIDC login endpoint", @@ -270,7 +270,7 @@ var OIDCAutoLogin = settings.RegisterBoolSetting( // OIDCGenerateClusterSSOTokenEnabled enables or disables generating JWT auth // tokens for cluster SSO with OIDC. var OIDCGenerateClusterSSOTokenEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCGenerateClusterSSOTokenEnabledSettingName, "enables or disables using OIDC to generate JWT auth tokens for cluster SSO", false, @@ -286,7 +286,7 @@ const ( // OIDCGenerateClusterSSOTokenUseToken selects which OIDC callback token to use // for cluster SSO. var OIDCGenerateClusterSSOTokenUseToken = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCGenerateClusterSSOTokenUseTokenSettingName, "selects which OIDC callback token to use for cluster SSO", "id_token", @@ -299,7 +299,7 @@ var OIDCGenerateClusterSSOTokenUseToken = settings.RegisterEnumSetting( // OIDCGenerateClusterSSOTokenSQLHost stores the host name or address to be used // for making SQL connections to the cluster, for display purposes only. var OIDCGenerateClusterSSOTokenSQLHost = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCGenerateClusterSSOTokenSQLHostSettingName, "stores the host name or address to be used for making SQL connections to the cluster, for display purposes only", "localhost", @@ -308,7 +308,7 @@ var OIDCGenerateClusterSSOTokenSQLHost = settings.RegisterStringSetting( // OIDCGenerateClusterSSOTokenSQLPort stores the port number to be used for making // SQL connections to the cluster, for display purposes only. var OIDCGenerateClusterSSOTokenSQLPort = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, OIDCGenerateClusterSSOTokenSQLPortSettingName, "stores the port number to be used for making SQL connections to the cluster, for display purposes only", 26257, diff --git a/pkg/ccl/storageccl/external_sst_reader.go b/pkg/ccl/storageccl/external_sst_reader.go index 04474b9f4c84..924283989bd1 100644 --- a/pkg/ccl/storageccl/external_sst_reader.go +++ b/pkg/ccl/storageccl/external_sst_reader.go @@ -28,14 +28,14 @@ import ( // RemoteSSTs lets external SSTables get iterated directly in some cases, // rather than being downloaded entirely first. var remoteSSTs = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.stream_external_ssts.enabled", "if enabled, external SSTables are iterated directly in some cases, rather than being downloaded entirely first", true, ) var remoteSSTSuffixCacheSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.stream_external_ssts.suffix_cache_size", "size of suffix of remote SSTs to download and cache before reading from remote stream", 64<<10, diff --git a/pkg/ccl/streamingccl/settings.go b/pkg/ccl/streamingccl/settings.go index 01b627effa4c..c4cbdcbbf005 100644 --- a/pkg/ccl/streamingccl/settings.go +++ b/pkg/ccl/streamingccl/settings.go @@ -17,7 +17,7 @@ import ( // CrossClusterReplicationEnabled enables the ability to setup and control a // cross cluster replication stream. var CrossClusterReplicationEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cross_cluster_replication.enabled", "enables the ability to setup and control a cross cluster replication stream", false, @@ -26,7 +26,7 @@ var CrossClusterReplicationEnabled = settings.RegisterBoolSetting( // StreamReplicationStreamLivenessTrackFrequency controls frequency to check // the liveness of a streaming replication producer job. var StreamReplicationStreamLivenessTrackFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.stream_liveness_track_frequency", "controls how frequent we check for the liveness of a replication stream producer job", time.Minute, @@ -35,7 +35,7 @@ var StreamReplicationStreamLivenessTrackFrequency = settings.RegisterDurationSet // StreamReplicationJobLivenessTimeout controls how long we wait for to kill // an inactive producer job. var StreamReplicationJobLivenessTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.job_liveness_timeout", "controls how long we wait for to kill an inactive producer job", 3*24*time.Hour, diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go index 5c79614db213..9f852d1b961d 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go @@ -39,7 +39,7 @@ import ( ) var replanThreshold = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.replan_flow_threshold", "fraction of nodes in the producer or consumer job that would need to change to refresh the"+ " physical execution plan. If set to 0, the physical plan will not automatically refresh.", @@ -48,7 +48,7 @@ var replanThreshold = settings.RegisterFloatSetting( ) var replanFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.replan_flow_frequency", "frequency at which the consumer job checks to refresh its physical execution plan", 10*time.Minute, diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go index d532a9ec4d36..26713605f04a 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go @@ -39,7 +39,7 @@ import ( // JobCheckpointFrequency controls the frequency of frontier checkpoints into // the jobs table. var JobCheckpointFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.job_checkpoint_frequency", "controls the frequency with which partitions update their progress; if 0, disabled", 10*time.Second, @@ -52,7 +52,7 @@ var JobCheckpointFrequency = settings.RegisterDurationSetting( // TODO(adityamaru): This timer should be removed once each job is aware of whether // it is profiling or not. var DumpFrontierEntries = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "stream_replication.dump_frontier_entries_frequency", "controls the frequency with which the frontier entries are persisted; if 0, disabled", 10*time.Minute, diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go index 2e50f6d00d9c..557c26e5ae0f 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go @@ -50,7 +50,7 @@ import ( ) var minimumFlushInterval = settings.RegisterDurationSettingWithExplicitUnit( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.stream_ingestion.minimum_flush_interval", "the minimum timestamp between flushes; flushes may still occur if internal buffers fill up", 5*time.Second, @@ -58,21 +58,21 @@ var minimumFlushInterval = settings.RegisterDurationSettingWithExplicitUnit( ) var maxKVBufferSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.stream_ingestion.kv_buffer_size", "the maximum size of the KV buffer allowed before a flush", 128<<20, // 128 MiB ) var maxRangeKeyBufferSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.stream_ingestion.range_key_buffer_size", "the maximum size of the range key buffer allowed before a flush", 32<<20, // 32 MiB ) var tooSmallRangeKeySize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.stream_ingestion.ingest_range_keys_as_writes", "size below which a range key SST will be ingested using normal writes", 400*1<<10, // 400 KiB @@ -82,7 +82,7 @@ var tooSmallRangeKeySize = settings.RegisterByteSizeSetting( // the system.jobs table to check whether the stream ingestion job has been // signaled to cutover. var cutoverSignalPollInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.stream_ingestion.cutover_signal_poll_interval", "the interval at which the stream ingestion job checks if it has been signaled to cutover", 10*time.Second, diff --git a/pkg/ccl/utilccl/license_check.go b/pkg/ccl/utilccl/license_check.go index 7061efbcf912..1977a7ac3484 100644 --- a/pkg/ccl/utilccl/license_check.go +++ b/pkg/ccl/utilccl/license_check.go @@ -32,7 +32,7 @@ import ( ) var enterpriseLicense = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "enterprise.license", "the encoded cluster license", "", diff --git a/pkg/cli/gen.go b/pkg/cli/gen.go index a4edf1d19088..fdc6e0a03a78 100644 --- a/pkg/cli/gen.go +++ b/pkg/cli/gen.go @@ -286,9 +286,9 @@ This session variable default should now be configured using %s`, switch setting.Class() { case settings.SystemOnly: class = classLabels[0] - case settings.TenantReadOnly: + case settings.SystemVisible: class = classLabels[1] - case settings.TenantWritable: + case settings.ApplicationLevel: class = classLabels[2] } row = append(row, class) @@ -461,7 +461,7 @@ func init() { f.StringVar(&classHeaderLabel, "class-header-label", "Class", "label to use in the output for the class column") f.StringSliceVar(&classLabels, "class-labels", - []string{"system-only", "tenant-ro", "tenant-rw"}, + []string{"system-only", "system-visible", "application"}, "label to use in the output for the various setting classes") genCmd.AddCommand(genCmds...) diff --git a/pkg/cloud/amazon/aws_kms.go b/pkg/cloud/amazon/aws_kms.go index 1735b2aad8e6..23fc955e6f8d 100644 --- a/pkg/cloud/amazon/aws_kms.go +++ b/pkg/cloud/amazon/aws_kms.go @@ -59,7 +59,7 @@ type kmsURIParams struct { } var reuseKMSSession = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.aws.reuse_kms_session.enabled", "persist the last opened AWS KMS session and reuse it when opening a new session with the same arguments", util.ConstantWithMetamorphicTestBool("aws-reuse-kms", true), diff --git a/pkg/cloud/amazon/s3_storage.go b/pkg/cloud/amazon/s3_storage.go index 879690da763b..e8e7027fd4fd 100644 --- a/pkg/cloud/amazon/s3_storage.go +++ b/pkg/cloud/amazon/s3_storage.go @@ -153,14 +153,14 @@ type s3Client struct { } var reuseSession = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.s3.session_reuse.enabled", "persist the last opened s3 session and re-use it when opening a new session with the same arguments", true, ) var usePutObject = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.s3.buffer_and_put_uploads.enabled", "construct files in memory before uploading via PutObject (may cause crashes due to memory usage)", false, diff --git a/pkg/cloud/azure/azure_storage.go b/pkg/cloud/azure/azure_storage.go index cf41013b6d8c..78ce45af5d4d 100644 --- a/pkg/cloud/azure/azure_storage.go +++ b/pkg/cloud/azure/azure_storage.go @@ -40,7 +40,7 @@ import ( ) var maxConcurrentUploadBuffers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.azure.concurrent_upload_buffers", "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", diff --git a/pkg/cloud/cloud_io.go b/pkg/cloud/cloud_io.go index 6f526747334e..80058088b41f 100644 --- a/pkg/cloud/cloud_io.go +++ b/pkg/cloud/cloud_io.go @@ -35,14 +35,14 @@ import ( // Timeout is a cluster setting used for cloud storage interactions. var Timeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.timeout", "the timeout for import/export storage operations", 10*time.Minute, settings.WithPublic) var httpCustomCA = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.http.custom_ca", "custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage", "", @@ -51,14 +51,14 @@ var httpCustomCA = settings.RegisterStringSetting( // WriteChunkSize is used to control the size of each chunk that is buffered and // uploaded by the cloud storage client. var WriteChunkSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.write_chunk.size", "controls the size of each file chunk uploaded by the cloud storage client", 8<<20, ) var retryConnectionTimedOut = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.connection_timed_out_retries.enabled", "retry generic connection timed out errors; use with extreme caution", false, diff --git a/pkg/cloud/gcp/gcs_storage.go b/pkg/cloud/gcp/gcs_storage.go index b57f9bce375f..750f501ee6ec 100644 --- a/pkg/cloud/gcp/gcs_storage.go +++ b/pkg/cloud/gcp/gcs_storage.go @@ -60,7 +60,7 @@ const ( // gcsChunkingEnabled is used to enable and disable chunking of file upload to // Google Cloud Storage. var gcsChunkingEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.gs.chunking.enabled", "enable chunking of file upload to Google Cloud Storage", true, /* default */ @@ -69,7 +69,7 @@ var gcsChunkingEnabled = settings.RegisterBoolSetting( // gcsChunkRetryTimeout is used to configure the per-chunk retry deadline when // uploading chunks to Google Cloud Storage. var gcsChunkRetryTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cloudstorage.gs.chunking.retry_timeout", "per-chunk retry deadline when chunking of file upload to Google Cloud Storage", 60*time.Second, diff --git a/pkg/cloud/impl_registry.go b/pkg/cloud/impl_registry.go index 48773c17be61..3726a63663d5 100644 --- a/pkg/cloud/impl_registry.go +++ b/pkg/cloud/impl_registry.go @@ -73,21 +73,21 @@ func registerLimiterSettings(providerType cloudpb.ExternalStorageProvider) { limiterSettings[providerType] = readAndWriteSettings{ read: rateAndBurstSettings{ - rate: settings.RegisterByteSizeSetting(settings.TenantWritable, readRateName, + rate: settings.RegisterByteSizeSetting(settings.ApplicationLevel, readRateName, "limit on number of bytes per second per node across operations writing to the designated cloud storage provider if non-zero", 0, ), - burst: settings.RegisterByteSizeSetting(settings.TenantWritable, readBurstName, + burst: settings.RegisterByteSizeSetting(settings.ApplicationLevel, readBurstName, "burst limit on number of bytes per second per node across operations writing to the designated cloud storage provider if non-zero", 0, ), }, write: rateAndBurstSettings{ - rate: settings.RegisterByteSizeSetting(settings.TenantWritable, writeRateName, + rate: settings.RegisterByteSizeSetting(settings.ApplicationLevel, writeRateName, "limit on number of bytes per second per node across operations writing to the designated cloud storage provider if non-zero", 0, ), - burst: settings.RegisterByteSizeSetting(settings.TenantWritable, writeBurstName, + burst: settings.RegisterByteSizeSetting(settings.ApplicationLevel, writeBurstName, "burst limit on number of bytes per second per node across operations writing to the designated cloud storage provider if non-zero", 0, ), diff --git a/pkg/clusterversion/clusterversion_test.go b/pkg/clusterversion/clusterversion_test.go index 839818acea3c..f3304c70afda 100644 --- a/pkg/clusterversion/clusterversion_test.go +++ b/pkg/clusterversion/clusterversion_test.go @@ -29,7 +29,7 @@ func TestClusterVersionOnChange(t *testing.T) { cvs := &clusterVersionSetting{} cvs.VersionSetting = settings.MakeVersionSetting(cvs) settings.RegisterVersionSetting( - settings.TenantWritable, + settings.ApplicationLevel, "dummy version key", "test description", &cvs.VersionSetting) diff --git a/pkg/clusterversion/setting.go b/pkg/clusterversion/setting.go index 32fcecd3a477..0a9415adb534 100644 --- a/pkg/clusterversion/setting.go +++ b/pkg/clusterversion/setting.go @@ -64,7 +64,7 @@ func registerClusterVersionSetting() *clusterVersionSetting { s := &clusterVersionSetting{} s.VersionSetting = settings.MakeVersionSetting(s) settings.RegisterVersionSetting( - settings.TenantWritable, + settings.ApplicationLevel, KeyVersionSetting, "set the active cluster version in the format '.'", // hide optional `-, &s.VersionSetting, @@ -241,7 +241,7 @@ func (cv *clusterVersionSetting) validateBinaryVersions( } var PreserveDowngradeVersion = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cluster.preserve_downgrade_option", "disable (automatic or manual) cluster version upgrade from the specified version until reset", "", diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index 58ae1e6dae43..fa37021f5887 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -79,7 +79,7 @@ const ( var ( intervalBaseSetting = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, intervalBaseSettingKey, "the base multiplier for other intervals such as adopt, cancel, and gc", defaultIntervalBase, @@ -87,7 +87,7 @@ var ( ) adoptIntervalSetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, adoptIntervalSettingKey, "the interval at which a node (a) claims some of the pending jobs and "+ "(b) restart its already claimed jobs that are in running or reverting "+ @@ -97,7 +97,7 @@ var ( ) cancelIntervalSetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, cancelIntervalSettingKey, "the interval at which a node cancels the jobs belonging to the known "+ "dead sessions", @@ -108,7 +108,7 @@ var ( // PollJobsMetricsInterval is the interval at which a tenant in the cluster // will poll the jobs table for metrics PollJobsMetricsInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, metricsPollingIntervalKey, "the interval at which a node in the cluster will poll the jobs table for metrics", defaultPollForMetricsInterval, @@ -116,7 +116,7 @@ var ( ) gcIntervalSetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, gcIntervalSettingKey, "the interval a node deletes expired job records that have exceeded their "+ "retention duration", @@ -126,7 +126,7 @@ var ( // RetentionTimeSetting wraps "jobs.retention_timehelpers_test.go". RetentionTimeSetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, retentionTimeSettingKey, "the amount of time for which records for completed jobs are retained", defaultRetentionTime, @@ -134,7 +134,7 @@ var ( settings.WithPublic) cancellationsUpdateLimitSetting = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, cancelUpdateLimitKey, "the number of jobs that can be updated when canceling jobs concurrently from dead sessions", defaultCancellationsUpdateLimit, @@ -142,7 +142,7 @@ var ( ) retryInitialDelaySetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, retryInitialDelaySettingKey, "the starting duration of exponential-backoff delay"+ " to retry a job which encountered a retryable error or had its coordinator"+ @@ -152,7 +152,7 @@ var ( ) retryMaxDelaySetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, retryMaxDelaySettingKey, "the maximum duration by which a job can be delayed to retry", defaultRetryMaxDelay, @@ -160,7 +160,7 @@ var ( ) executionErrorsMaxEntriesSetting = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, executionErrorsMaxEntriesKey, "the maximum number of retriable error entries which will be stored for introspection", defaultExecutionErrorsMaxEntries, @@ -168,7 +168,7 @@ var ( ) executionErrorsMaxEntrySize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, executionErrorsMaxEntrySizeKey, "the maximum byte size of individual error entries which will be stored"+ " for introspection", @@ -176,7 +176,7 @@ var ( ) debugPausepoints = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, debugPausePointsSettingKey, "the list, comma separated, of named pausepoints currently enabled for debugging", "", diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index 1efaa3cc3b47..f66a01aaa947 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -436,28 +436,28 @@ func (s *jobScheduler) runDaemon(ctx context.Context, stopper *stop.Stopper) { } var schedulerEnabledSetting = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "jobs.scheduler.enabled", "enable/disable job scheduler", true, ) var schedulerPaceSetting = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "jobs.scheduler.pace", "how often to scan system.scheduled_jobs table", time.Minute, ) var schedulerMaxJobsPerIterationSetting = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "jobs.scheduler.max_jobs_per_iteration", "how many schedules to start per iteration; setting to 0 turns off this limit", 5, ) var schedulerScheduleExecutionTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "jobs.scheduler.schedule_execution.timeout", "sets a timeout on for schedule execution; 0 disables timeout", 30*time.Second, diff --git a/pkg/kv/bulk/setting.go b/pkg/kv/bulk/setting.go index b6373fe8f76d..37c7d3b52917 100644 --- a/pkg/kv/bulk/setting.go +++ b/pkg/kv/bulk/setting.go @@ -19,7 +19,7 @@ import ( var ( // IngestBatchSize controls the size of ingest ssts. IngestBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.batch_size", "the maximum size of the payload in an AddSSTable request", 16<<20, diff --git a/pkg/kv/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index c0b98854b94d..62056e914c73 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -46,14 +46,14 @@ const maxScatterSize = 4 << 20 var ( tooSmallSSTSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_io_write.small_write_size", "size below which a 'bulk' write will be performed as a normal write instead", 400*1<<10, // 400 Kib ) ingestDelay = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.ingest.flush_delay", "amount of time to wait before sending a file to the KV/Storage layer to ingest", 0, @@ -61,7 +61,7 @@ var ( ) senderConcurrency = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.ingest.sender_concurrency_limit", "maximum number of concurrent bulk ingest requests sent by any one sender, such as a processor in an IMPORT, index creation or RESTORE, etc (0 = no limit)", 0, diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 0986e38d35f5..b457e3089819 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -250,7 +250,7 @@ const ( ) var rangeDescriptorCacheSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.range_descriptor_cache.size", "maximum number of entries in the range descriptor cache", 1e6, @@ -261,7 +261,7 @@ var rangeDescriptorCacheSize = settings.RegisterIntSetting( // senderConcurrencyLimit controls the maximum number of asynchronous send // requests. var senderConcurrencyLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.dist_sender.concurrency_limit", "maximum number of asynchronous send requests", max(defaultSenderConcurrency, int64(64*runtime.GOMAXPROCS(0))), diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index 0e455225fb11..706630225d76 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -56,7 +56,7 @@ type singleRangeInfo struct { } var useDedicatedRangefeedConnectionClass = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.rangefeed.use_dedicated_connection_class.enabled", "uses dedicated connection when running rangefeeds", util.ConstantWithMetamorphicTestBool( @@ -64,7 +64,7 @@ var useDedicatedRangefeedConnectionClass = settings.RegisterBoolSetting( ) var catchupStartupRate = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.rangefeed.client.stream_startup_rate", "controls the rate per second the client will initiate new rangefeed stream for a single range; 0 implies unlimited", 100, // e.g.: 200 seconds for 20k ranges. @@ -73,7 +73,7 @@ var catchupStartupRate = settings.RegisterIntSetting( ) var rangefeedRangeStuckThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.rangefeed.range_stuck_threshold", "restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.rangefeed.closed_timestamp_refresh_interval takes precedence)", time.Minute, diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go index 86f33f0c3a2b..53acaddc303c 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go @@ -26,7 +26,7 @@ import ( ) var parallelCommitsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.parallel_commits_enabled", "if enabled, transactional commits will be parallelized with transactional writes", true, diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go index 75d72b335beb..acb9878e43fc 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go @@ -33,14 +33,14 @@ const txnPipelinerBtreeDegree = 32 // PipelinedWritesEnabled is the kv.transaction.write_pipelining.enabled cluster setting. var PipelinedWritesEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.write_pipelining_enabled", "if enabled, transactional writes are pipelined through Raft consensus", true, settings.WithName("kv.transaction.write_pipelining.enabled"), ) var pipelinedWritesMaxBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.write_pipelining_max_batch_size", "if non-zero, defines that maximum size batch that will be pipelined through Raft consensus", // NB: there is a tradeoff between the overhead of synchronously waiting for @@ -80,7 +80,7 @@ var pipelinedWritesMaxBatchSize = settings.RegisterIntSetting( // find matching intents. // See #54029 for more details. var TrackedWritesMaxSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.max_intents_bytes", "maximum number of bytes used to track locks in transactions", 1<<22, /* 4 MB */ @@ -89,7 +89,7 @@ var TrackedWritesMaxSize = settings.RegisterIntSetting( // rejectTxnOverTrackedWritesBudget dictates what happens when a txn exceeds // kv.transaction.max_intents_bytes. var rejectTxnOverTrackedWritesBudget = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.reject_over_max_intents_budget.enabled", "if set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) "+ "are rejected instead of having their lock spans imprecisely compressed", diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index 41c525312e69..a380c9918f4d 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -38,7 +38,7 @@ const ( // on the coordinator during the lifetime of a transaction. Refresh spans // are used for SERIALIZABLE transactions to avoid client restarts. var MaxTxnRefreshSpansBytes = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.transaction.max_refresh_spans_bytes", "maximum number of bytes used to track refresh spans in serializable transactions", 1<<22, /* 4 MB */ diff --git a/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go b/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go index 926f24c02a5c..355191da59ee 100644 --- a/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go +++ b/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go @@ -41,7 +41,7 @@ const ( // streamerAvgResponseSizeMultiple determines the multiple used when calculating // the average response size. var streamerAvgResponseSizeMultiple = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.distsql.streamer.avg_response_size_multiple", "determines the multiple used when calculating the average response size by the streamer component", defaultAvgResponseSizeMultiple, diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go index 6424343190a3..ac97f0caacfd 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer.go +++ b/pkg/kv/kvclient/kvstreamer/streamer.go @@ -326,7 +326,7 @@ type streamerStatistics struct { // this setting is chosen arbitrarily as 1/8th of the default value for the // senderConcurrencyLimit. var streamerConcurrencyLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.streamer.concurrency_limit", "maximum number of asynchronous requests by a single streamer", max(128, int64(8*runtime.GOMAXPROCS(0))), diff --git a/pkg/kv/kvclient/rangefeed/db_adapter.go b/pkg/kv/kvclient/rangefeed/db_adapter.go index fe0456076399..54ddb9c1176d 100644 --- a/pkg/kv/kvclient/rangefeed/db_adapter.go +++ b/pkg/kv/kvclient/rangefeed/db_adapter.go @@ -40,7 +40,7 @@ type dbAdapter struct { var _ DB = (*dbAdapter)(nil) var maxScanParallelism = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.rangefeed.max_scan_parallelism", "maximum number of concurrent scan requests that can be issued during initial scan", 64, diff --git a/pkg/kv/kvprober/settings.go b/pkg/kv/kvprober/settings.go index 202368a630f6..a2ca905a128b 100644 --- a/pkg/kv/kvprober/settings.go +++ b/pkg/kv/kvprober/settings.go @@ -21,7 +21,7 @@ import ( // ensures that kvprober will not be significantly affected if the cluster is // overloaded. var bypassAdmissionControl = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.bypass_admission_control.enabled", "set to bypass admission control queue for kvprober requests; "+ "note that dedicated clusters should have this set as users own capacity planning "+ @@ -30,7 +30,7 @@ var bypassAdmissionControl = settings.RegisterBoolSetting( ) var readEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.read.enabled", "whether the KV read prober is enabled", false) @@ -38,7 +38,7 @@ var readEnabled = settings.RegisterBoolSetting( // TODO(josh): Another option is for the cluster setting to be a QPS target // for the cluster as a whole. var readInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.read.interval", "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; "+ @@ -47,7 +47,7 @@ var readInterval = settings.RegisterDurationSetting( settings.PositiveDuration) var readTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.read.timeout", // Slow enough response times are not different than errors from the // perspective of the user. @@ -59,13 +59,13 @@ var readTimeout = settings.RegisterDurationSetting( ) var writeEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.write.enabled", "whether the KV write prober is enabled", false) var writeInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.write.interval", "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; "+ @@ -75,7 +75,7 @@ var writeInterval = settings.RegisterDurationSetting( ) var writeTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.write.timeout", // Slow enough response times are not different than errors from the // perspective of the user. @@ -87,7 +87,7 @@ var writeTimeout = settings.RegisterDurationSetting( ) var scanMeta2Timeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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", @@ -96,7 +96,7 @@ var scanMeta2Timeout = settings.RegisterDurationSetting( ) var numStepsToPlanAtOnce = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.planner.num_steps_to_plan_at_once", "the number of Steps to plan at once, where a Step is a decision on "+ "what range to probe; the order of the Steps is randomized within "+ @@ -107,7 +107,7 @@ var numStepsToPlanAtOnce = settings.RegisterIntSetting( settings.PositiveInt) var quarantinePoolSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.quarantine_pool_size", "the maximum size of the kv prober quarantine pool, where the quarantine "+ "pool holds Steps for ranges that have been probed and timed out; If "+ @@ -117,7 +117,7 @@ var quarantinePoolSize = settings.RegisterIntSetting( settings.PositiveInt) var quarantineWriteEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.quarantine.write.enabled", "whether the KV write prober is enabled for the quarantine pool; The "+ "quarantine pool holds a separate group of ranges that have previously failed "+ @@ -126,7 +126,7 @@ var quarantineWriteEnabled = settings.RegisterBoolSetting( false) var quarantineWriteInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.prober.quarantine.write.interval", "how often each node sends a write probe for the quarantine pool to the KV layer "+ "on average (jitter is added); "+ diff --git a/pkg/kv/kvserver/batcheval/cmd_export.go b/pkg/kv/kvserver/batcheval/cmd_export.go index 3145dfb1468d..edc646698037 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export.go +++ b/pkg/kv/kvserver/batcheval/cmd_export.go @@ -38,7 +38,7 @@ const SSTTargetSizeSetting = "kv.bulk_sst.target_size" // ExportRequestTargetFileSize controls the target file size for SSTs created // during backups. var ExportRequestTargetFileSize = settings.RegisterByteSizeSetting( - settings.TenantReadOnly, // used by BACKUP + settings.SystemVisible, // used by BACKUP SSTTargetSizeSetting, fmt.Sprintf("target size for SSTs emitted from export requests; "+ "export requests (i.e. BACKUP) may buffer up to the sum of %s and %s in memory", diff --git a/pkg/kv/kvserver/closedts/setting.go b/pkg/kv/kvserver/closedts/setting.go index 2edd93dce2be..5b4d03dd9b36 100644 --- a/pkg/kv/kvserver/closedts/setting.go +++ b/pkg/kv/kvserver/closedts/setting.go @@ -16,14 +16,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" ) -// NB: These settings are TenantReadOnly because they need to be read by e.g. +// NB: These settings are SystemVisible because they need to be read by e.g. // rangefeed clients and follower_read_timestamp(). However, they really need // to see the host's setting, not the tenant's setting. See: // https://github.com/cockroachdb/cockroach/issues/108677 // TargetDuration is the follower reads closed timestamp update target duration. var TargetDuration = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.closed_timestamp.target_duration", "if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration", 3*time.Second, @@ -33,7 +33,7 @@ var TargetDuration = settings.RegisterDurationSetting( // SideTransportCloseInterval determines the ClosedTimestampSender's frequency. var SideTransportCloseInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.closed_timestamp.side_transport_interval", "the interval at which the closed timestamp side-transport attempts to "+ "advance each range's closed timestamp; set to 0 to disable the side-transport", @@ -47,7 +47,7 @@ var SideTransportCloseInterval = settings.RegisterDurationSetting( // (see TargetForPolicy), if it is set to a non-zero value. Meant as an escape // hatch. var LeadForGlobalReadsOverride = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.closed_timestamp.lead_for_global_reads_override", "if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps", 0, diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index 8fce50282cb6..0e0768ce8a74 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -279,7 +279,7 @@ func IntersectSpan( // SplitByLoadMergeDelay wraps "kv.range_split.by_load_merge_delay". var SplitByLoadMergeDelay = settings.RegisterDurationSetting( - settings.TenantReadOnly, // used by TRUNCATE in SQL + settings.SystemVisible, // used by TRUNCATE in SQL "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, @@ -298,7 +298,7 @@ const ( // MaxCommandSize wraps "kv.raft.command.max_size". var MaxCommandSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.raft.command.max_size", "maximum size of a raft command", MaxCommandSizeDefault, diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index e8069fa19a2f..ca70c8c90df9 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -56,7 +56,7 @@ const minTimeUntilNodeDead = gossip.StoresInterval + 15*time.Second // TimeUntilNodeDead wraps "server.time_until_store_dead". var TimeUntilNodeDead = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, timeUntilNodeDeadSettingName, "the time after which if there is no new gossiped information about a store, it is considered dead", 5*time.Minute, diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go index 0a007a26ff38..26c5449327c7 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go @@ -31,7 +31,7 @@ import ( // ReconcileInterval is the interval between two reconciliations of protected // timestamp records. var ReconcileInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.protectedts.reconciliation.interval", "the frequency for reconciling jobs with protected timestamp records", 5*time.Minute, diff --git a/pkg/kv/kvserver/protectedts/settings.go b/pkg/kv/kvserver/protectedts/settings.go index fe9f63555655..27ff9df9d9c7 100644 --- a/pkg/kv/kvserver/protectedts/settings.go +++ b/pkg/kv/kvserver/protectedts/settings.go @@ -22,7 +22,7 @@ import ( // MaxBytes controls the maximum number of bytes worth of spans and metadata // which can be protected by all protected timestamp records. var MaxBytes = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.protectedts.max_bytes", "if non-zero the limit of the number of bytes of spans and metadata which can be protected", 1<<20, // 1 MiB @@ -33,7 +33,7 @@ var MaxBytes = settings.RegisterIntSetting( // MaxSpans controls the maximum number of spans which can be protected // by all protected timestamp records. var MaxSpans = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.protectedts.max_spans", "if non-zero the limit of the number of spans which can be protected", 32768, @@ -44,7 +44,7 @@ var MaxSpans = settings.RegisterIntSetting( // PollInterval defines how frequently the protectedts state is polled by the // Tracker. var PollInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.protectedts.poll_interval", // TODO(ajwerner): better description. "the interval at which the protectedts subsystem state is polled", diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index a7b166fc0d66..f05caa2933ba 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -54,7 +54,7 @@ const ( // which the processing of a queue may time out. It is an escape hatch to raise // the timeout for queues. var queueGuaranteedProcessingTimeBudget = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.queue.process.guaranteed_time_budget", "the guaranteed duration before which the processing of a queue may "+ "time out", diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 22d96e1d614a..89e1e33006a6 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -88,7 +88,7 @@ const ( // threshold and the current GC TTL (true) or just based on the GC threshold // (false). var StrictGCEnforcement = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.gc_ttl.strict_enforcement.enabled", "if true, fail to serve requests at timestamps below the TTL even if the data still exists", true, diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index 7fac760c443d..49dbc9c88e0c 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -27,7 +27,7 @@ import ( // information is collected and passed around, regardless of the value of this // setting. var FollowerReadsEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, // needed for planning in SQL + settings.SystemVisible, // needed for planning in SQL "kv.closed_timestamp.follower_reads_enabled", "allow (all) replicas to serve consistent historical reads based on closed timestamp information", true, diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 814f0ebe2fb4..6dab2f5d5823 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -47,7 +47,7 @@ import ( // ranges and ranges covering tables in the system database); this setting // covers everything else. var RangefeedEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.rangefeed.enabled", "if set, rangefeed registration is enabled", false, @@ -56,7 +56,7 @@ var RangefeedEnabled = settings.RegisterBoolSetting( // RangeFeedRefreshInterval controls the frequency with which we deliver closed // timestamp updates to rangefeeds. var RangeFeedRefreshInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "kv.rangefeed.closed_timestamp_refresh_interval", "the interval at which closed-timestamp updates"+ "are delivered to rangefeeds; set to 0 to use kv.closed_timestamp.side_transport_interval", diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 6538915dfa6c..64e43611a85a 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -284,7 +284,7 @@ const leaseTransferWaitSettingName = "server.shutdown.lease_transfer_wait" // here since we check it in the caller to limit generated requests as well // to prevent excessive queuing. var ExportRequestsLimit = settings.RegisterIntSetting( - settings.TenantReadOnly, // used in backup processor + settings.SystemVisible, // used in backup processor "kv.bulk_io_write.concurrent_export_requests", "number of export requests a store will handle concurrently before queuing", 3, diff --git a/pkg/multitenant/multitenantio/cost_controlling_io.go b/pkg/multitenant/multitenantio/cost_controlling_io.go index 813d7c7a1e63..c6c1dd6cfa7f 100644 --- a/pkg/multitenant/multitenantio/cost_controlling_io.go +++ b/pkg/multitenant/multitenantio/cost_controlling_io.go @@ -25,7 +25,7 @@ import ( // the common case, but without allowing an unbounded number of bytes to be // read/written before accounting for them. var DefaultBytesAllowedBeforeAccounting = settings.RegisterIntSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_external_io_default_bytes_allowed_before_accounting", "controls how many bytes will be read/written before blocking for RUs when writing to external storage", 16<<20, // 16 MB diff --git a/pkg/multitenant/tenantcostmodel/settings.go b/pkg/multitenant/tenantcostmodel/settings.go index 790e20d7968d..4a9eb59a731b 100644 --- a/pkg/multitenant/tenantcostmodel/settings.go +++ b/pkg/multitenant/tenantcostmodel/settings.go @@ -30,7 +30,7 @@ import ( // from the host cluster. var ( ReadBatchCost = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.read_batch_cost", "base cost of a read batch in Request Units", 0.50, @@ -38,7 +38,7 @@ var ( ) ReadRequestCost = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.read_request_cost", "base cost of a read request in Request Units", 0.125, @@ -46,7 +46,7 @@ var ( ) ReadPayloadCostPerMiB = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.read_payload_cost_per_mebibyte", "cost of a read payload in Request Units per MiB", 16, @@ -54,7 +54,7 @@ var ( ) WriteBatchCost = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.write_batch_cost", "base cost of a write batch in Request Units", 1, @@ -62,7 +62,7 @@ var ( ) WriteRequestCost = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.write_request_cost", "base cost of a write request in Request Units", 1, @@ -70,7 +70,7 @@ var ( ) WritePayloadCostPerMiB = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.write_payload_cost_per_mebibyte", "cost of a write payload in Request Units per MiB", 1024, @@ -78,7 +78,7 @@ var ( ) SQLCPUSecondCost = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.sql_cpu_second_cost", "cost of a CPU-second in SQL pods in Request Units", 333.3333, @@ -86,7 +86,7 @@ var ( ) PgwireEgressCostPerMiB = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.pgwire_egress_cost_per_mebibyte", "cost of client <-> SQL ingress/egress per MiB", 1024, @@ -94,7 +94,7 @@ var ( ) ExternalIOEgressCostPerMiB = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.external_io_egress_per_mebibyte", "cost of a write to external storage in Request Units per MiB", 1024, @@ -102,7 +102,7 @@ var ( ) ExternalIOIngressCostPerMiB = settings.RegisterFloatSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.external_io_ingress_per_mebibyte", "cost of a read from external storage in Request Units per MiB", 0, @@ -110,7 +110,7 @@ var ( ) CrossRegionNetworkCostSetting = settings.RegisterStringSetting( - settings.TenantReadOnly, + settings.SystemVisible, "tenant_cost_model.cross_region_network_cost", "network cost table for cross-region traffic", "", diff --git a/pkg/roachpb/span_stats.go b/pkg/roachpb/span_stats.go index 4f1a9d0a33aa..40586a4f8347 100644 --- a/pkg/roachpb/span_stats.go +++ b/pkg/roachpb/span_stats.go @@ -23,7 +23,7 @@ const DefaultSpanStatsSpanLimit = 1000 // SpanStatsBatchLimit registers the maximum number of spans allowed in a // span stats request payload. var SpanStatsBatchLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.span_stats.span_batch_limit", "the maximum number of spans allowed in a request payload for span statistics", DefaultSpanStatsSpanLimit, @@ -31,7 +31,7 @@ var SpanStatsBatchLimit = settings.RegisterIntSetting( ) var SpanStatsNodeTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.span_stats.node.timeout", "the duration allowed for a single node to return span stats data before"+ " the request is cancelled; if set to 0, there is no timeout", @@ -44,7 +44,7 @@ const defaultRangeStatsBatchLimit = 100 // RangeStatsBatchLimit registers the maximum number of ranges to be batched // when fetching range stats for a span. var RangeStatsBatchLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.span_stats.range_batch_limit", "the maximum batch size when fetching ranges statistics for a span", defaultRangeStatsBatchLimit, @@ -54,7 +54,7 @@ var RangeStatsBatchLimit = settings.RegisterIntSetting( // RangeDescPageSize controls the page size when iterating through range // descriptors. var RangeDescPageSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.span_stats.range_desc_page_size", "the page size when iterating through range descriptors", 100, diff --git a/pkg/rpc/settings.go b/pkg/rpc/settings.go index 31ae22dfcd29..f3aa6eaaa22f 100644 --- a/pkg/rpc/settings.go +++ b/pkg/rpc/settings.go @@ -32,7 +32,7 @@ func init() { } var enableRPCCircuitBreakers = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "rpc.circuit_breaker.enabled", "enables stateful management of failed connections, including circuit breaking "+ "when in unhealthy state; only use in case of issues - logging may be suboptimal, "+ @@ -43,7 +43,7 @@ var enableRPCCircuitBreakers = settings.RegisterBoolSetting( // TODO(baptist): Remove in 23.2 (or 24.1) once validating dialback works for all scenarios. var useDialback = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "rpc.dialback.enabled", "if true, require bidirectional RPC connections between nodes to prevent one-way network unavailability", true, diff --git a/pkg/security/cert_expiry_cache.go b/pkg/security/cert_expiry_cache.go index 52418da771f2..991afb0799dc 100644 --- a/pkg/security/cert_expiry_cache.go +++ b/pkg/security/cert_expiry_cache.go @@ -33,7 +33,7 @@ const CacheCapacityMax = 65000 // ClientCertExpirationCacheCapacity is the cluster setting that controls the // maximum number of client cert expirations in the cache. var ClientCertExpirationCacheCapacity = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.client_cert_expiration_cache.capacity", "the maximum number of client cert expirations stored", 1000, diff --git a/pkg/security/password.go b/pkg/security/password.go index 956c4b5afdb1..0c727accbe76 100644 --- a/pkg/security/password.go +++ b/pkg/security/password.go @@ -35,7 +35,7 @@ import ( // For estimates, see: // http://security.stackexchange.com/questions/17207/recommended-of-rounds-for-bcrypt var BcryptCost = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, BcryptCostSettingName, fmt.Sprintf( "the hashing cost to use when storing passwords supplied as cleartext by SQL clients "+ @@ -59,7 +59,7 @@ const BcryptCostSettingName = "server.user_login.password_hashes.default_cost.cr // The value of 4096 is the minimum value recommended by RFC 5802. // It should be increased along with computation power. var SCRAMCost = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, SCRAMCostSettingName, fmt.Sprintf( "the hashing cost to use when storing passwords supplied as cleartext by SQL clients "+ @@ -97,7 +97,7 @@ var ErrUnknownHashMethod = errors.New("unknown hash method") // to read the current hash method. Instead use the // GetConfiguredHashMethod() function. var PasswordHashMethod = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.password_encryption", "which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD", // Note: the default is initially SCRAM, even in mixed-version clusters where @@ -136,7 +136,7 @@ func GetConfiguredPasswordHashMethod(sv *settings.Values) (method password.HashM // AutoDetectPasswordHashes is the cluster setting that configures whether // the server recognizes pre-hashed passwords. var AutoDetectPasswordHashes = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.store_client_pre_hashed_passwords.enabled", "whether the server accepts to store passwords pre-hashed by clients", true, @@ -145,7 +145,7 @@ var AutoDetectPasswordHashes = settings.RegisterBoolSetting( // MinPasswordLength is the cluster setting that configures the // minimum SQL password length. var MinPasswordLength = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.min_password_length", "the minimum length accepted for passwords set in cleartext via SQL. "+ "Note that a value lower than 1 is ignored: passwords cannot be empty in any case.", @@ -156,7 +156,7 @@ var MinPasswordLength = settings.RegisterIntSetting( // AutoUpgradePasswordHashes is the cluster setting that configures whether to // automatically re-encode stored passwords using crdb-bcrypt to scram-sha-256. var AutoUpgradePasswordHashes = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.upgrade_bcrypt_stored_passwords_to_scram.enabled", "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", @@ -166,7 +166,7 @@ var AutoUpgradePasswordHashes = settings.RegisterBoolSetting( // AutoDowngradePasswordHashes is the cluster setting that configures whether to // automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt. var AutoDowngradePasswordHashes = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled", "if server.user_login.password_encryption=crdb-bcrypt, this controls "+ "whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt", @@ -177,7 +177,7 @@ var AutoDowngradePasswordHashes = settings.RegisterBoolSetting( // automatically re-encode stored passwords using scram-sha-256 to use a new // default cost setting. var AutoRehashOnSCRAMCostChange = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.rehash_scram_stored_passwords_on_cost_change.enabled", "if server.user_login.password_hashes.default_cost.scram_sha_256 differs from, "+ "the cost in a stored hash, this controls whether to automatically re-encode "+ diff --git a/pkg/security/tls_settings.go b/pkg/security/tls_settings.go index 0015952d0e06..42a510f62517 100644 --- a/pkg/security/tls_settings.go +++ b/pkg/security/tls_settings.go @@ -41,7 +41,7 @@ type TLSSettings interface { } var ocspMode = settings.RegisterEnumSetting( - settings.TenantWritable, "security.ocsp.mode", + settings.ApplicationLevel, "security.ocsp.mode", "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.", @@ -49,7 +49,7 @@ var ocspMode = settings.RegisterEnumSetting( settings.WithPublic) var ocspTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, "security.ocsp.timeout", + settings.ApplicationLevel, "security.ocsp.timeout", "timeout before considering the OCSP server unreachable", 3*time.Second, settings.NonNegativeDuration, diff --git a/pkg/server/admin.go b/pkg/server/admin.go index a67fc7463a90..b52f35397194 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -141,7 +141,7 @@ type systemAdminServer struct { var noteworthyAdminMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEWORTHY_ADMIN_MEMORY_USAGE", 100*1024) var tableStatsMaxFetcherConcurrency = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.admin.table_stats.max_fetcher_concurrency", "maximum number of concurrent table stats fetches to run", 64, // arbitrary diff --git a/pkg/server/authserver/authentication.go b/pkg/server/authserver/authentication.go index 28eef76395b3..4428af559f77 100644 --- a/pkg/server/authserver/authentication.go +++ b/pkg/server/authserver/authentication.go @@ -95,7 +95,7 @@ var ConfigureOIDC = func( // WebSessionTimeout is the cluster setting for web session TTL. var WebSessionTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.web_session_timeout", "the duration that a newly created web session will be valid", 7*24*time.Hour, diff --git a/pkg/server/clock_monotonicity.go b/pkg/server/clock_monotonicity.go index 6b3fec3c8125..ae8426a2c36f 100644 --- a/pkg/server/clock_monotonicity.go +++ b/pkg/server/clock_monotonicity.go @@ -24,7 +24,7 @@ import ( var ( forwardClockJumpCheckEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.clock.forward_jump_check_enabled", "if enabled, forward clock jumps > max_offset/2 will cause a panic", false, @@ -32,7 +32,7 @@ var ( settings.WithPublic) persistHLCUpperBoundInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.clock.persist_upper_bound_interval", "the interval between persisting the wall time upper bound of the clock. The clock "+ "does not generate a wall time greater than the persisted timestamp and will panic if "+ diff --git a/pkg/server/cluster_settings.go b/pkg/server/cluster_settings.go index 84179db622b3..a91257a84cbb 100644 --- a/pkg/server/cluster_settings.go +++ b/pkg/server/cluster_settings.go @@ -15,7 +15,7 @@ import "github.com/cockroachdb/cockroach/pkg/settings" // SQLStatsResponseMax controls the maximum number of statements and transactions returned by the // CombinedStatements endpoint. var SQLStatsResponseMax = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.response.max", "the maximum number of statements and transaction stats returned in a CombinedStatements request", 20000, @@ -25,7 +25,7 @@ var SQLStatsResponseMax = settings.RegisterIntSetting( // SQLStatsShowInternal controls if statistics for internal executions should be returned in sql stats APIs, // including: CombinedStatementStats, ListSessions, and ListLocalSessions. var SQLStatsShowInternal = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.response.show_internal.enabled", "controls if statistics for internal executions should be returned by the CombinedStatements and if "+ "internal sessions should be returned by the ListSessions endpoints. These endpoints are used to display "+ @@ -38,7 +38,7 @@ var SQLStatsShowInternal = settings.RegisterBoolSetting( // acts as a cache storing the top queries from system.statement_statistics // and system.transaction_statistics tables. var StatsActivityUIEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.activity.ui.enabled", "enable the combined statistics endpoint to get data from the system activity tables", true) @@ -46,7 +46,7 @@ var StatsActivityUIEnabled = settings.RegisterBoolSetting( // PersistedInsightsUIEnabled controls if the insights endpoint uses // the persisted statement_execution_insights and transaction_execution_insights tables. var PersistedInsightsUIEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.persisted_insights.ui.enabled", "enable the insights endpoint to get data from the persisted insights tables", false) diff --git a/pkg/server/debug/server.go b/pkg/server/debug/server.go index ab247858787c..8e858fa8f00b 100644 --- a/pkg/server/debug/server.go +++ b/pkg/server/debug/server.go @@ -57,7 +57,7 @@ const Endpoint = "/debug/" // 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.ApplicationLevel, "server.remote_debugging.mode", "unused", "local", settings.Retired) // Server serves the /debug/* family of tools. diff --git a/pkg/server/diagnostics/reporter.go b/pkg/server/diagnostics/reporter.go index efadf2641515..4b8b07810357 100644 --- a/pkg/server/diagnostics/reporter.go +++ b/pkg/server/diagnostics/reporter.go @@ -58,7 +58,7 @@ type NodeStatusGenerator interface { } var reportFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "diagnostics.reporting.interval", "interval at which diagnostics data should be reported", time.Hour, diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 8e1623067333..706289d0053a 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -38,7 +38,7 @@ import ( var ( queryWait = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.shutdown.query_wait", "the timeout for waiting for active queries to finish during a drain "+ "(note that the --drain-wait parameter for cockroach node drain may need adjustment "+ @@ -48,7 +48,7 @@ var ( settings.WithPublic) drainWait = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.shutdown.drain_wait", "the amount of time a server waits in an unready state before proceeding with a drain "+ "(note that the --drain-wait parameter for cockroach node drain may need adjustment "+ @@ -60,7 +60,7 @@ var ( settings.WithPublic) connectionWait = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.shutdown.connection_wait", "the maximum amount of time a server waits for all SQL connections to "+ "be closed before proceeding with a drain. "+ @@ -71,7 +71,7 @@ var ( settings.WithPublic) jobRegistryWait = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.shutdown.jobs_wait", "the maximum amount of time a server waits for all currently executing jobs "+ "to notice drain request and to perform orderly shutdown", diff --git a/pkg/server/goroutinedumper/goroutinedumper.go b/pkg/server/goroutinedumper/goroutinedumper.go index 0f97e6ce1382..7f1ca375601b 100644 --- a/pkg/server/goroutinedumper/goroutinedumper.go +++ b/pkg/server/goroutinedumper/goroutinedumper.go @@ -34,14 +34,14 @@ const ( var ( numGoroutinesThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.goroutine_dump.num_goroutines_threshold", "a threshold beyond which if number of goroutines increases, "+ "then goroutine dump can be triggered", 1000, ) totalDumpSizeLimit = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.goroutine_dump.total_dump_size_limit", "total size of goroutine dumps to be kept. "+ "Dumps are GC'ed in the order of creation time. The latest dump is "+ diff --git a/pkg/server/hot_ranges.go b/pkg/server/hot_ranges.go index cd44f7380baa..4d0dcef0f62e 100644 --- a/pkg/server/hot_ranges.go +++ b/pkg/server/hot_ranges.go @@ -22,7 +22,7 @@ import ( // at the expense of possibly incomplete data, or raised for complete data, at the cost // of a possibly slower response. var HotRangesRequestNodeTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.hot_ranges_request.node.timeout", "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, diff --git a/pkg/server/node.go b/pkg/server/node.go index c1d9bd5fc0d5..b19f64bd0d71 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -218,7 +218,7 @@ This metric is thus not an indicator of KV health.`, var ( // graphiteEndpoint is host:port, if any, of Graphite metrics server. graphiteEndpoint = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "external.graphite.endpoint", "if nonempty, push server metrics to the Graphite or Carbon server at the specified host:port", "", @@ -226,7 +226,7 @@ var ( // graphiteInterval is how often metrics are pushed to Graphite, if enabled. graphiteInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, graphiteIntervalKey, "the interval at which metrics are pushed to Graphite (if enabled)", 10*time.Second, diff --git a/pkg/server/profiler/cpuprofiler.go b/pkg/server/profiler/cpuprofiler.go index 9d49bc4a723d..e9003b3a91e6 100644 --- a/pkg/server/profiler/cpuprofiler.go +++ b/pkg/server/profiler/cpuprofiler.go @@ -27,14 +27,14 @@ import ( ) var maxCombinedCPUProfFileSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cpu_profile.total_dump_size_limit", "maximum combined disk size of preserved CPU profiles", 128<<20, // 128MiB ) var cpuUsageCombined = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cpu_profile.cpu_usage_combined_threshold", "a threshold beyond which if the combined cpu usage is above, "+ "then a cpu profile can be triggered. If a value over 100 is set, "+ @@ -45,7 +45,7 @@ var cpuUsageCombined = settings.RegisterIntSetting( ) var cpuProfileInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cpu_profile.interval", // NB: this is not the entire explanation - it's when we stop taking into // account the high water mark seen. Without this, if CPU ever reaches 100%, @@ -55,7 +55,7 @@ var cpuProfileInterval = settings.RegisterDurationSetting( ) var cpuProfileDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cpu_profile.duration", "the duration for how long a cpu profile is taken", 10*time.Second, settings.PositiveDuration, diff --git a/pkg/server/profiler/profilestore.go b/pkg/server/profiler/profilestore.go index 97a641fe423d..ec3af3b1cf92 100644 --- a/pkg/server/profiler/profilestore.go +++ b/pkg/server/profiler/profilestore.go @@ -28,7 +28,7 @@ import ( var ( maxProfiles = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.mem_profile.max_profiles", "maximum number of profiles to be kept per ramp-up of memory usage. "+ "A ramp-up is defined as a sequence of profiles with increasing usage.", @@ -36,7 +36,7 @@ var ( ) maxCombinedFileSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.mem_profile.total_dump_size_limit", "maximum combined disk size of preserved memory profiles", 128<<20, // 128MiB @@ -45,12 +45,12 @@ var ( func init() { _ = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.heap_profile.max_profiles", "use server.mem_profile.max_profiles instead", 5, settings.Retired) _ = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.heap_profile.total_dump_size_limit", "use server.mem_profile.total_dump_size_limit instead", 128<<20, // 128MiB diff --git a/pkg/server/server.go b/pkg/server/server.go index 3863acdff18b..5d7532030fe4 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1111,7 +1111,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf kvStoresIterator: kvserver.MakeStoresIterator(node.stores), inspectzServer: inspectzServer, - notifyChangeToTenantReadOnlySettings: tenantSettingsWatcher.SetAlternateDefaults, + notifyChangeToSystemVisibleSettings: tenantSettingsWatcher.SetAlternateDefaults, }, SQLConfig: &cfg.SQLConfig, BaseConfig: &cfg.BaseConfig, diff --git a/pkg/server/server_http.go b/pkg/server/server_http.go index 6dcdf81d6c59..0beb34d7271f 100644 --- a/pkg/server/server_http.go +++ b/pkg/server/server_http.go @@ -67,7 +67,7 @@ func newHTTPServer( // server. These instruct a valid user agent to use HTTPS *only* // for all future connections to this host. var HSTSEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.hsts.enabled", "if true, HSTS headers will be sent along with all HTTP "+ "requests. The headers will contain a max-age setting of one "+ diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index c98278078b33..2f9ab30e73d2 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -259,12 +259,12 @@ type sqlServerOptionalKVArgs struct { // the equivalent of /inspectz but through SQL. inspectzServer inspectzpb.InspectzServer - // notifyChangeToTenantReadOnlySettings is called by the settings + // notifyChangeToSystemVisibleSettings is called by the settings // watcher when one or more TenandReadOnly setting is updated via // SET CLUSTER SETTING (i.e. updated in system.settings). // // The second argument must be sorted by setting key already. - notifyChangeToTenantReadOnlySettings func(context.Context, []kvpb.TenantSetting) + notifyChangeToSystemVisibleSettings func(context.Context, []kvpb.TenantSetting) } // sqlServerOptionalTenantArgs are the arguments supplied to newSQLServer which @@ -434,7 +434,7 @@ type monitorAndMetricsOptions struct { } var vmoduleSetting = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.debug.default_vmodule", "vmodule string (ignored by any server with an explicit one provided at start)", "", @@ -577,7 +577,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { var settingsWatcher *settingswatcher.SettingsWatcher if codec.ForSystemTenant() { settingsWatcher = settingswatcher.NewWithNotifier(ctx, - cfg.clock, codec, cfg.Settings, cfg.rangeFeedFactory, cfg.stopper, cfg.notifyChangeToTenantReadOnlySettings, cfg.settingsStorage, + cfg.clock, codec, cfg.Settings, cfg.rangeFeedFactory, cfg.stopper, cfg.notifyChangeToSystemVisibleSettings, cfg.settingsStorage, ) } else { // Create the tenant settings watcher, using the tenant connector as the diff --git a/pkg/server/server_systemlog_gc.go b/pkg/server/server_systemlog_gc.go index d7952392eae9..3627b04fcf43 100644 --- a/pkg/server/server_systemlog_gc.go +++ b/pkg/server/server_systemlog_gc.go @@ -29,7 +29,7 @@ import ( var ( systemLogGCPeriod = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.log_gc.period", "the period at which log-like system tables are checked for old entries", time.Hour, @@ -37,7 +37,7 @@ var ( settings.WithPublic) systemLogGCLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.log_gc.max_deletions_per_cycle", "the maximum number of entries to delete on each purge of log-like system tables", 1000, @@ -46,7 +46,7 @@ var ( // rangeLogTTL is the TTL for rows in system.rangelog. If non zero, range log // entries are periodically garbage collected. rangeLogTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.rangelog.ttl", "if nonzero, entries in system.rangelog older than this duration are periodically purged", 30*24*time.Hour, // 30 days @@ -55,14 +55,14 @@ var ( // eventLogTTL is the TTL for rows in system.eventlog. If non zero, event log // entries are periodically garbage collected. eventLogTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.eventlog.ttl", "if nonzero, entries in system.eventlog older than this duration are periodically purged", 90*24*time.Hour, // 90 days settings.WithPublic) webSessionPurgeTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.web_session.purge.ttl", "if nonzero, entries in system.web_sessions older than this duration are periodically purged", time.Hour, diff --git a/pkg/server/settingswatcher/settings_watcher.go b/pkg/server/settingswatcher/settings_watcher.go index 6740a1d77db2..16ade0ffee4c 100644 --- a/pkg/server/settingswatcher/settings_watcher.go +++ b/pkg/server/settingswatcher/settings_watcher.go @@ -70,15 +70,15 @@ type SettingsWatcher struct { updateWait chan struct{} } - // notifyTenantReadOnlyChange is called when one or more - // TenantReadOnly setting changes. It is only set when the + // notifySystemVisibleChange is called when one or more + // SystemVisible setting changes. It is only set when the // SettingsWatcher is created with NewWithNotifier. It is used by // the tenant setting override watcher to pick up defaults set via // system.settings in the system tenant. // // The callee function can assume that the slice in the second // argument is sorted by InternalKey. - notifyTenantReadOnlyChange func(context.Context, []kvpb.TenantSetting) + notifySystemVisibleChange func(context.Context, []kvpb.TenantSetting) // testingWatcherKnobs allows the client to inject testing knobs into // the underlying rangefeedcache.Watcher. @@ -117,7 +117,7 @@ func New( } // NewWithNotifier constructs a new SettingsWatcher which notifies -// an observer about changes to TenantReadOnly settings. +// an observer about changes to SystemVisible settings. func NewWithNotifier( ctx context.Context, clock *hlc.Clock, @@ -129,7 +129,7 @@ func NewWithNotifier( storage Storage, // optional ) *SettingsWatcher { w := New(clock, codec, settingsToUpdate, f, stopper, storage) - w.notifyTenantReadOnlyChange = notify + w.notifySystemVisibleChange = notify return w } @@ -155,7 +155,7 @@ func NewWithOverrides( // the stopper is stopped prior to the initial data being retrieved. func (s *SettingsWatcher) Start(ctx context.Context) error { // Ensure we inform the read-only default notify callback function - // of the build-time defaults for TenantReadOnly settings. + // of the build-time defaults for SystemVisible settings. // // Note: we cannot call this in the New() function above because // this can only be called after the in-RAM values have been loaded @@ -279,11 +279,11 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { } func (s *SettingsWatcher) loadInitialReadOnlyDefaults(ctx context.Context) { - if s.notifyTenantReadOnlyChange == nil { + if s.notifySystemVisibleChange == nil { return } - // When there is no explicit value in system.settings for a TenantReadOnly + // When there is no explicit value in system.settings for a SystemVisible // setting, we still want to propagate the system tenant's idea // of the default value as an override to secondary tenants. // @@ -292,7 +292,7 @@ func (s *SettingsWatcher) loadInitialReadOnlyDefaults(ctx context.Context) { // setting. We want to make sure that the secondary tenant's idea of // the default value is the same as the system tenant's. - tenantReadOnlyKeys := settings.TenantReadOnlyKeys() + tenantReadOnlyKeys := settings.SystemVisibleKeys() payloads := make([]kvpb.TenantSetting, 0, len(tenantReadOnlyKeys)) for _, key := range tenantReadOnlyKeys { knownSetting, payload := s.getSettingAndValue(key) @@ -304,7 +304,7 @@ func (s *SettingsWatcher) loadInitialReadOnlyDefaults(ctx context.Context) { // Make sure the payloads are sorted, as this is required by the // notify API. sort.Slice(payloads, func(i, j int) bool { return payloads[i].InternalKey < payloads[j].InternalKey }) - s.notifyTenantReadOnlyChange(ctx, payloads) + s.notifySystemVisibleChange(ctx, payloads) } // TestingRestart restarts the rangefeeds and waits for the initial @@ -343,7 +343,7 @@ func (s *SettingsWatcher) handleKV( return nil } if !s.codec.ForSystemTenant() { - if setting.Class() != settings.TenantWritable { + if setting.Class() != settings.ApplicationLevel { log.Warningf(ctx, "ignoring read-only setting %s", settingKey) return nil } @@ -404,10 +404,10 @@ func (s *SettingsWatcher) maybeSet( } } - if class == settings.TenantReadOnly { + if class == settings.SystemVisible { // Notify the tenant settings watcher there is a new fallback // default for this setting. - s.setTenantReadOnlyDefault(ctx, key) + s.setSystemVisibleDefault(ctx, key) } } @@ -602,11 +602,11 @@ func (s *SettingsWatcher) GetTenantClusterVersion() clusterversion.Handle { return s.settings.Version } -// setTenantReadOnlyDefault is called by the watcher above for any +// setSystemVisibleDefault is called by the watcher above for any // changes to system.settings made on a setting with class -// TenantReadOnly. -func (s *SettingsWatcher) setTenantReadOnlyDefault(ctx context.Context, key settings.InternalKey) { - if s.notifyTenantReadOnlyChange == nil { +// SystemVisible. +func (s *SettingsWatcher) setSystemVisibleDefault(ctx context.Context, key settings.InternalKey) { + if s.notifySystemVisibleChange == nil { return } @@ -621,7 +621,7 @@ func (s *SettingsWatcher) setTenantReadOnlyDefault(ctx context.Context, key sett log.VEventf(ctx, 1, "propagating read-only default %+v", payload) - s.notifyTenantReadOnlyChange(ctx, []kvpb.TenantSetting{payload}) + s.notifySystemVisibleChange(ctx, []kvpb.TenantSetting{payload}) } func (s *SettingsWatcher) getSettingAndValue(key settings.InternalKey) (bool, kvpb.TenantSetting) { diff --git a/pkg/server/settingswatcher/settings_watcher_external_test.go b/pkg/server/settingswatcher/settings_watcher_external_test.go index 389234d4373a..4749c246c9d0 100644 --- a/pkg/server/settingswatcher/settings_watcher_external_test.go +++ b/pkg/server/settingswatcher/settings_watcher_external_test.go @@ -239,10 +239,10 @@ func (f *fakeStorage) getNumWrites() int { return f.numWrites } -var _ = settings.RegisterStringSetting(settings.TenantWritable, "str.foo", "desc", "") -var _ = settings.RegisterStringSetting(settings.TenantWritable, "str.bar", "desc", "bar") -var _ = settings.RegisterIntSetting(settings.TenantWritable, "i0", "desc", 0) -var _ = settings.RegisterIntSetting(settings.TenantWritable, "i1", "desc", 1) +var _ = settings.RegisterStringSetting(settings.ApplicationLevel, "str.foo", "desc", "") +var _ = settings.RegisterStringSetting(settings.ApplicationLevel, "str.bar", "desc", "bar") +var _ = settings.RegisterIntSetting(settings.ApplicationLevel, "i0", "desc", 0) +var _ = settings.RegisterIntSetting(settings.ApplicationLevel, "i1", "desc", 1) func TestSettingsWatcherWithOverrides(t *testing.T) { defer leaktest.AfterTest(t)() @@ -529,7 +529,7 @@ func TestStaleRowsDoNotCauseSettingsToRegress(t *testing.T) { ) fakeSetting := settings.RegisterStringSetting( - settings.TenantWritable, fakeSettingName, "for testing", defaultFakeSettingValue, + settings.ApplicationLevel, fakeSettingName, "for testing", defaultFakeSettingValue, ) // Set a cluster setting in the real cluster and read its raw KV. @@ -627,11 +627,11 @@ func TestStaleRowsDoNotCauseSettingsToRegress(t *testing.T) { settingStillHasValueAfterAShortWhile(t, defaultFakeSettingValue) } -var _ = settings.RegisterStringSetting(settings.TenantReadOnly, "str.baz", "desc", "initial") +var _ = settings.RegisterStringSetting(settings.SystemVisible, "str.baz", "desc", "initial") var _ = settings.RegisterStringSetting(settings.SystemOnly, "str.yay", "desc", "") // TestNotifyCalledUponReadOnlySettingChanges verifies that the notify -// function callback is called when a TenantReadOnly setting is +// function callback is called when a SystemVisible setting is // updated in system.settings. func TestNotifyCalledUponReadOnlySettingChanges(t *testing.T) { defer leaktest.AfterTest(t)() @@ -683,9 +683,9 @@ func TestNotifyCalledUponReadOnlySettingChanges(t *testing.T) { t.Run("initial scan", func(t *testing.T) { // The notifier is called at least once for all the - // pre-existing TenantReadOnly settings. + // pre-existing SystemVisible settings. testutils.SucceedsSoon(t, func() error { - for _, k := range settings.TenantReadOnlyKeys() { + for _, k := range settings.SystemVisibleKeys() { seen, v := contains(k) if !seen { return errors.Newf("%s not seen yet", k) @@ -716,7 +716,7 @@ func TestNotifyCalledUponReadOnlySettingChanges(t *testing.T) { // The rangefeed event for str.baz was delivered after those for // str.foo and str.yay. If we had incorrectly notified an update - // for non-TenantReadOnly setting, they would show up in the + // for non-SystemVisible setting, they would show up in the // updated list. mu.Lock() defer mu.Unlock() diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index 03bfe7b363aa..bf42fd0a7942 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -89,7 +89,7 @@ type storeMetrics interface { // ChildMetricsEnabled enables exporting of additional prometheus time series with extra labels var ChildMetricsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, "server.child_metrics.enabled", + settings.ApplicationLevel, "server.child_metrics.enabled", "enables the exporting of child metrics, additional prometheus time series with extra labels", false, settings.WithPublic) diff --git a/pkg/server/structlogging/hot_ranges_log.go b/pkg/server/structlogging/hot_ranges_log.go index 6164bbd52def..58fa4cd8596d 100644 --- a/pkg/server/structlogging/hot_ranges_log.go +++ b/pkg/server/structlogging/hot_ranges_log.go @@ -31,7 +31,7 @@ import ( const ReportTopHottestRanges = 5 var TelemetryHotRangesStatsInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.telemetry.hot_ranges_stats.interval", "the time interval to log hot ranges stats", 4*time.Hour, @@ -39,14 +39,14 @@ var TelemetryHotRangesStatsInterval = settings.RegisterDurationSetting( ) var TelemetryHotRangesStatsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.telemetry.hot_ranges_stats.enabled", "enable/disable capturing hot ranges statistics to the telemetry logging channel", true, ) var TelemetryHotRangesStatsLoggingDelay = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.telemetry.hot_ranges_stats.logging_delay", "the time delay between emitting individual hot ranges stats logs", 1*time.Second, diff --git a/pkg/server/tenantsettingswatcher/overrides_store.go b/pkg/server/tenantsettingswatcher/overrides_store.go index 552f9e5dca45..c84914c8ddf6 100644 --- a/pkg/server/tenantsettingswatcher/overrides_store.go +++ b/pkg/server/tenantsettingswatcher/overrides_store.go @@ -53,7 +53,7 @@ type overridesStore struct { // // The slice is sorted by InternalKey. // - // At the time of this writing, this is used for TenantReadOnly + // At the time of this writing, this is used for SystemVisible // settings. alternateDefaults []kvpb.TenantSetting } @@ -141,7 +141,7 @@ func (s *overridesStore) setAll( providedKeys[v.InternalKey] = struct{}{} } // If we are setting the all-tenant overrides, ensure there is a - // pseudo-override for every TenantReadOnly setting with an + // pseudo-override for every SystemVisible setting with an // alternate default. if tenantID == allTenantOverridesID && len(s.mu.alternateDefaults) > 0 { // We can set copyOverrides==false because we took ownership @@ -256,7 +256,7 @@ func (s *overridesStore) setTenantOverride( // when there is no stored default in .tenant_settings. // // At the time of this writing, this is called when a change is made -// to a TenantReadOnly setting in the system tenant's system.settings +// to a SystemVisible setting in the system tenant's system.settings // table. Values set this way serve as default value if there is no // override in system.tenant_settings. // diff --git a/pkg/server/tenantsettingswatcher/watcher.go b/pkg/server/tenantsettingswatcher/watcher.go index 7568a5dfaca0..9aeb6684d44b 100644 --- a/pkg/server/tenantsettingswatcher/watcher.go +++ b/pkg/server/tenantsettingswatcher/watcher.go @@ -272,7 +272,7 @@ func (w *Watcher) GetAllTenantOverrides( // // The second argument must be sorted by setting key already. // -// At the time of this writing, this is used for TenantReadOnly +// At the time of this writing, this is used for SystemVisible // settings, so that the values from the system tenant's // system.settings table are used when there is no override // in .tenant_settings. diff --git a/pkg/server/tracedumper/tracedumper.go b/pkg/server/tracedumper/tracedumper.go index 238b4c580d5c..8d9910669679 100644 --- a/pkg/server/tracedumper/tracedumper.go +++ b/pkg/server/tracedumper/tracedumper.go @@ -34,7 +34,7 @@ const ( var ( totalDumpSizeLimit = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.job_trace.total_dump_size_limit", "total size of job trace dumps to be kept. "+ "Dumps are GC'ed in the order of creation time. The latest dump is "+ diff --git a/pkg/settings/doc.go b/pkg/settings/doc.go index a4cf1414aa3f..b0e943b05e53 100644 --- a/pkg/settings/doc.go +++ b/pkg/settings/doc.go @@ -24,7 +24,7 @@ setting is to be used. For example, to add an "enterprise" flag, adding into license_check.go: var enterpriseEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "enterprise.enabled", "some doc for the setting", false, diff --git a/pkg/settings/integration_tests/propagation.go b/pkg/settings/integration_tests/propagation.go index 058bc993848d..d10da01b4310 100644 --- a/pkg/settings/integration_tests/propagation.go +++ b/pkg/settings/integration_tests/propagation.go @@ -64,9 +64,9 @@ func RunSettingDefaultPropagationTest( return tenantAllOverride } // No tenant override. What is the default? - // For TenantReadOnly, if there is a custom value in the + // For SystemVisible, if there is a custom value in the // system interface, that becomes the default. - if setting.Class() == settings.TenantReadOnly && sysOverride != "" { + if setting.Class() == settings.SystemVisible && sysOverride != "" { return sysOverride } // Otherwise, fall back to the default. diff --git a/pkg/settings/integration_tests/read_only_1/propagation_test.go b/pkg/settings/integration_tests/read_only_1/propagation_test.go index 5e461830f1de..6a32aebdeb0b 100644 --- a/pkg/settings/integration_tests/read_only_1/propagation_test.go +++ b/pkg/settings/integration_tests/read_only_1/propagation_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -var roS = settings.RegisterStringSetting(settings.TenantReadOnly, "tenant.read.only", "desc", "initial") +var roS = settings.RegisterStringSetting(settings.SystemVisible, "tenant.read.only", "desc", "initial") // TestSettingDefaultPropagationReadOnly1 runs one of 4 invocations of // `RunSettingDefaultPropagationTest`. The test is split 4-ways across diff --git a/pkg/settings/integration_tests/read_only_2/propagation_test.go b/pkg/settings/integration_tests/read_only_2/propagation_test.go index a4d304237e1b..5d81a7fb11fe 100644 --- a/pkg/settings/integration_tests/read_only_2/propagation_test.go +++ b/pkg/settings/integration_tests/read_only_2/propagation_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -var roS = settings.RegisterStringSetting(settings.TenantReadOnly, "tenant.read.only", "desc", "initial") +var roS = settings.RegisterStringSetting(settings.SystemVisible, "tenant.read.only", "desc", "initial") // TestSettingDefaultPropagationReadOnly2 runs one of 4 invocations of // `RunSettingDefaultPropagationTest`. The test is split 4-ways across diff --git a/pkg/settings/integration_tests/read_write_1/propagation_test.go b/pkg/settings/integration_tests/read_write_1/propagation_test.go index a3b8baba2ffb..e8dd617c1661 100644 --- a/pkg/settings/integration_tests/read_write_1/propagation_test.go +++ b/pkg/settings/integration_tests/read_write_1/propagation_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -var rwS = settings.RegisterStringSetting(settings.TenantWritable, "tenant.writable", "desc", "initial") +var rwS = settings.RegisterStringSetting(settings.ApplicationLevel, "tenant.writable", "desc", "initial") // TestSettingDefaultPropagationReadWrite1 runs one of 4 invocations of // `RunSettingDefaultPropagationTest`. The test is split 4-ways across diff --git a/pkg/settings/integration_tests/read_write_2/propagation_test.go b/pkg/settings/integration_tests/read_write_2/propagation_test.go index 67ee54084fc7..b48f163fa260 100644 --- a/pkg/settings/integration_tests/read_write_2/propagation_test.go +++ b/pkg/settings/integration_tests/read_write_2/propagation_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" ) -var rwS = settings.RegisterStringSetting(settings.TenantWritable, "tenant.writable", "desc", "initial") +var rwS = settings.RegisterStringSetting(settings.ApplicationLevel, "tenant.writable", "desc", "initial") // TestSettingDefaultPropagationReadWrite2 runs one of 4 invocations of // `RunSettingDefaultPropagationTest`. The test is split 4-ways across diff --git a/pkg/settings/integration_tests/settings_test.go b/pkg/settings/integration_tests/settings_test.go index 878cd7eb642b..3aaef91d1bb3 100644 --- a/pkg/settings/integration_tests/settings_test.go +++ b/pkg/settings/integration_tests/settings_test.go @@ -35,7 +35,7 @@ const byteSizeKey = "testing.bytesize" const enumKey = "testing.enum" var strA = settings.RegisterStringSetting( - settings.TenantWritable, strKey, "desc", "", + settings.ApplicationLevel, 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) @@ -43,7 +43,7 @@ var strA = settings.RegisterStringSetting( return nil })) var intA = settings.RegisterIntSetting( - settings.TenantWritable, intKey, "desc", 1, + settings.ApplicationLevel, 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) @@ -51,7 +51,7 @@ var intA = settings.RegisterIntSetting( return nil })) var durationA = settings.RegisterDurationSetting( - settings.TenantWritable, durationKey, "desc", time.Minute, + settings.ApplicationLevel, 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) @@ -59,10 +59,10 @@ var durationA = settings.RegisterDurationSetting( return nil })) var byteSizeA = settings.RegisterByteSizeSetting( - settings.TenantWritable, byteSizeKey, "desc", 1024*1024, + settings.ApplicationLevel, byteSizeKey, "desc", 1024*1024, ) var enumA = settings.RegisterEnumSetting( - settings.TenantWritable, enumKey, "desc", "foo", map[int64]string{1: "foo", 2: "bar"}) + settings.ApplicationLevel, enumKey, "desc", "foo", map[int64]string{1: "foo", 2: "bar"}) func TestSettingsRefresh(t *testing.T) { defer leaktest.AfterTest(t)() diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index df7941da027d..374bc525ddc9 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -27,7 +27,7 @@ import ( var registry = make(map[InternalKey]internalSetting) // tenantReadOnlyKeys contains the keys of settings that have the -// class TenantReadOnly. This is used to initialize defaults in the +// class SystemVisible. This is used to initialize defaults in the // tenant settings watcher. var tenantReadOnlyKeys []InternalKey @@ -57,12 +57,12 @@ func TestingSaveRegistry() func() { for k, v := range aliasRegistry { origAliases[k] = v } - var origTenantReadOnlyKeys = make([]InternalKey, len(tenantReadOnlyKeys)) - copy(origTenantReadOnlyKeys, tenantReadOnlyKeys) + var origSystemVisibleKeys = make([]InternalKey, len(tenantReadOnlyKeys)) + copy(origSystemVisibleKeys, tenantReadOnlyKeys) return func() { registry = origRegistry aliasRegistry = origAliases - tenantReadOnlyKeys = origTenantReadOnlyKeys + tenantReadOnlyKeys = origSystemVisibleKeys } } @@ -285,7 +285,7 @@ func register(class Class, key InternalKey, desc string, s internalSetting) { s.init(class, key, desc, slot) registry[key] = s slotTable[slot] = s - if class == TenantReadOnly { + if class == SystemVisible { tenantReadOnlyKeys = append(tenantReadOnlyKeys, key) } } @@ -314,10 +314,10 @@ func Keys(forSystemTenant bool) (res []InternalKey) { return res } -// TenantReadOnlyKeys returns a array with all the known keys that -// have the class TenantReadOnly. It might not be sorted. +// SystemVisibleKeys returns a array with all the known keys that +// have the class SystemVisible. It might not be sorted. // The caller must refrain from modifying the return value. -func TenantReadOnlyKeys() []InternalKey { +func SystemVisibleKeys() []InternalKey { return tenantReadOnlyKeys } diff --git a/pkg/settings/setting.go b/pkg/settings/setting.go index 4b12abb64bde..651ba6b001a1 100644 --- a/pkg/settings/setting.go +++ b/pkg/settings/setting.go @@ -117,45 +117,106 @@ type NonMaskedSetting interface { ValueOrigin(ctx context.Context, sv *Values) ValueOrigin } -// Class describes the scope of a setting in multi-tenant scenarios. While all -// settings can be used on the system tenant, the classes restrict use on -// non-system tenants. -// -// Settings can only be registered via the Class, e.g. -// SystemOnly.RegisterIntSetting(). +// Class describes the scope of a setting under cluster +// virtualization. // // Guidelines for choosing a class: // -// - Make sure to read the descriptions below carefully to understand the -// differences in semantics. +// - Make sure to read the descriptions below carefully to +// understand the differences in semantics. +// +// - Rules of thumb: // -// - If the setting controls a user-visible aspect of SQL, it should be a -// TenantWritable setting. +// 1. if an end-user should be able to modify the setting in +// CockroachCloud Serverless, AND different end-users should be +// able to use different values, the setting should have class +// ApplicationLevel. // -// - Control settings relevant to tenant-specific internal implementation -// should be TenantReadOnly. +// 2. if a setting should only be controlled by SREs in +// CockroachCloud Serverless, OR a single value must apply to +// multiple tenants (virtual clusters) simultaneously, the setting +// must not use class ApplicationLevel. // -// - When in doubt, the first choice to consider should be TenantReadOnly. +// 3. if and only if a setting relevant to the KV/storage layer +// and whose value is shared across all virtual cluster is ever +// accessed by code in the application layer (SQL execution or +// HTTP handlers), use SystemVisible. // -// - SystemOnly should be used with caution: even internal tenant code is -// disallowed from using these settings at all. +// 4. in other cases, use SystemOnly. type Class int8 const ( - // SystemOnly settings are associated with single-tenant clusters and host - // clusters. Settings with this class do not exist on non-system tenants and - // can only be used by the system tenant. + // SystemOnly settings are specific to the KV/storage layer and + // cannot be accessed from application layer code (in particular not + // from SQL layer nor HTTP handlers). + // + // As a rule of thumb, use this class if: + // + // - the setting may be accessed from the shared KV/storage layer; + // + // - AND, the setting should only be controllable by SREs (not + // end-users) in CockroachCloud Serverless, AND a single value + // must apply to all virtual clusters simultaneously; + // + // (If this part of the condition does not hold, consider + // ApplicationLevel instead.) + // + // - AND, its value is never needed in the application layer (i.e. + // it is not read from SQL code, HTTP handlers or other code + // that would run in SQL pods in CC Serverless). + // + // (If this part of the condition does not hold, consider + // SystemVisible instead.) SystemOnly Class = iota - // TenantReadOnly settings are visible to non-system tenants but cannot be - // modified by the tenant. Values for these settings are set from the system - // tenant and propagated from the host cluster. - TenantReadOnly + // SystemVisible settings are specific to the KV/storage layer and + // are also visible to virtual clusters. + // + // As a rule of thumb, use this class if: + // + // - the setting may be accessed from the shared KV/storage layer; + // + // - AND the setting should only be controllable by SREs (not + // end-users) in CockroachCloud Serverless, AND a single value + // must apply to all virtual clusters simultaneously; + // + // (If this part of the condition does not hold, consider + // ApplicationLevel instead.) + // + // - AND, its value is sometimes needed in the application layer + // (i.e. it may be read from SQL code, HTTP handlers or other + // code that could run in SQL pods in CC Serverless). + // + // (If this part of the condition does not hold, consider + // SystemOnly instead.) + SystemVisible - // TenantWritable settings are visible to and can be modified by non-system - // tenants. The system can still override these settings; the overrides are - // propagated from the host cluster. - TenantWritable + // ApplicationLevel settings are readable and can optionally be + // modified by virtual clusters. + // + // As a rule of thumb, use this class if: + // + // - the setting is never accessed by the shared KV/storage layer; + // + // - AND, any of the following holds: + // + // - end-users should legitimately be able to modify + // the setting in CockroachCloud Serverless; + // + // (If this part of the condition does not hold, but the other + // part of the condition below does hold, consider still using + // ApplicationLevel and force an override using ALTER VIRTUAL + // CLUSTER SET CLUSTER SETTING. This makes the + // ApplicationLevel setting effectively read-only from the + // virtual cluster's perspective, because system overrides + // cannot be modified by the virtual cluster.) + // + // - OR, different virtual clusters should be able to + // use different values for the setting. + // + // (If this part of the condition does not hold, consider + // SystemOnly or SystemVisible instead.) + ApplicationLevel ) // Visibility describes how a user should feel confident that they can customize diff --git a/pkg/settings/settings_test.go b/pkg/settings/settings_test.go index a19183429406..a265dc5903c7 100644 --- a/pkg/settings/settings_test.go +++ b/pkg/settings/settings_test.go @@ -154,18 +154,18 @@ var changes = struct { }{} var boolTA = settings.RegisterBoolSetting(settings.SystemOnly, "bool.t", "desc", true) -var boolFA = settings.RegisterBoolSetting(settings.TenantReadOnly, "bool.f", "desc", false) -var strFooA = settings.RegisterStringSetting(settings.TenantWritable, "str.foo", "desc", "") +var boolFA = settings.RegisterBoolSetting(settings.SystemVisible, "bool.f", "desc", false) +var strFooA = settings.RegisterStringSetting(settings.ApplicationLevel, "str.foo", "desc", "") var strBarA = settings.RegisterStringSetting(settings.SystemOnly, "str.bar", "desc", "bar") -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.RegisterDurationSettingWithExplicitUnit(settings.TenantWritable, "d_with_explicit_unit", "desc", time.Second, settings.NonNegativeDuration, settings.WithPublic) -var pA = settings.RegisterProtobufSetting(settings.TenantWritable, "p", "desc", &dummyVersion{msg1: "foo"}) -var _ = settings.RegisterDurationSetting(settings.TenantWritable, "d_with_maximum", "desc", time.Second, settings.NonNegativeDurationWithMaximum(time.Hour)) +var i1A = settings.RegisterIntSetting(settings.ApplicationLevel, "i.1", "desc", 0) +var i2A = settings.RegisterIntSetting(settings.ApplicationLevel, "i.2", "desc", 5) +var fA = settings.RegisterFloatSetting(settings.SystemVisible, "f", "desc", 5.4) +var dA = settings.RegisterDurationSetting(settings.ApplicationLevel, "d", "desc", time.Second) +var duA = settings.RegisterDurationSettingWithExplicitUnit(settings.ApplicationLevel, "d_with_explicit_unit", "desc", time.Second, settings.NonNegativeDuration, settings.WithPublic) +var pA = settings.RegisterProtobufSetting(settings.ApplicationLevel, "p", "desc", &dummyVersion{msg1: "foo"}) +var _ = settings.RegisterDurationSetting(settings.ApplicationLevel, "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) +var byteSize = settings.RegisterByteSizeSetting(settings.ApplicationLevel, "zzz", "desc", mb) var mA = func() *settings.VersionSetting { s := settings.MakeVersionSetting(&dummyVersionSettingImpl{}) settings.RegisterVersionSetting(settings.SystemOnly, "v.1", "desc", &s) @@ -789,11 +789,11 @@ func batchRegisterSettings( } var overrideBool = settings.RegisterBoolSetting(settings.SystemOnly, "override.bool", "desc", true) -var overrideInt = settings.RegisterIntSetting(settings.TenantReadOnly, "override.int", "desc", 0) -var overrideDuration = settings.RegisterDurationSetting(settings.TenantWritable, "override.duration", "desc", time.Second) -var overrideFloat = settings.RegisterFloatSetting(settings.TenantWritable, "override.float", "desc", 1.0) -var overrideString = settings.RegisterStringSetting(settings.TenantWritable, "override.string", "desc", "foo") -var overrideProto = settings.RegisterProtobufSetting(settings.TenantWritable, "override.proto", "desc", &dummyVersion{msg1: "foo"}) +var overrideInt = settings.RegisterIntSetting(settings.SystemVisible, "override.int", "desc", 0) +var overrideDuration = settings.RegisterDurationSetting(settings.ApplicationLevel, "override.duration", "desc", time.Second) +var overrideFloat = settings.RegisterFloatSetting(settings.ApplicationLevel, "override.float", "desc", 1.0) +var overrideString = settings.RegisterStringSetting(settings.ApplicationLevel, "override.string", "desc", "foo") +var overrideProto = settings.RegisterProtobufSetting(settings.ApplicationLevel, "override.proto", "desc", &dummyVersion{msg1: "foo"}) func TestOverride(t *testing.T) { ctx := context.Background() diff --git a/pkg/settings/updater.go b/pkg/settings/updater.go index 25f76a5f281c..37185cc6b92a 100644 --- a/pkg/settings/updater.go +++ b/pkg/settings/updater.go @@ -248,13 +248,13 @@ func (u updater) SetFromStorage( } if !u.sv.SpecializedToVirtualCluster() /* system tenant */ || - d.Class() == TenantWritable { + d.Class() == ApplicationLevel { // The value is being loaded from the current virtual cluster's // system.settings. Load it as an active value. return u.setInternal(ctx, key, value, d, origin) } - // Here we are looking at a TenantReadOnly or SystemOnly setting + // Here we are looking at a SystemVisible or SystemOnly setting // from within a virtual cluster. if d.Class() == SystemOnly { @@ -267,7 +267,7 @@ func (u updater) SetFromStorage( return errors.AssertionFailedf("programming error: cannot set SystemOnly %q", key) } - if d.Class() != TenantReadOnly { + if d.Class() != SystemVisible { return errors.AssertionFailedf("unhandled class %v", d.Class()) } @@ -280,7 +280,7 @@ func (u updater) SetFromStorage( } }() - // We are receiving an alternate default for a TenantReadOnly + // We are receiving an alternate default for a SystemVisible // setting. Here we do not configure the main setting value (via // setInternal or .set on the setting itself): many tests use // .Override earlier and we do not want to change the override. diff --git a/pkg/settings/values.go b/pkg/settings/values.go index 38284bc930dc..4af12633e183 100644 --- a/pkg/settings/values.go +++ b/pkg/settings/values.go @@ -122,9 +122,9 @@ func (c *valuesContainer) checkForbidden(slot slotIdx) bool { if buildutil.CrdbTestBuild { const msg = `programming error: invalid access to SystemOnly setting %s from a virtual cluster! -TIP: use class TenantWritable for settings that configure just 1 +TIP: use class ApplicationLevel for settings that configure just 1 virtual cluster; SystemOnly for settings that affect only the shared -storage layer; and TenantReadOnly for settings that affect the storage +storage layer; and SystemVisible for settings that affect the storage layer and also must be visible to all virtual clusters. ` panic(errors.AssertionFailedf(msg, slotTable[slot].Name())) @@ -258,11 +258,11 @@ func (sv *Values) setOnChange(slot slotIdx, fn func(ctx context.Context)) { // TestingCopyForVirtualCluster makes a copy of the input Values in // the target Values for use when initializing a server for a virtual // cluster in tests. This is meant to propagate overrides -// to TenantWritable settings. +// to ApplicationLevel settings. func (sv *Values) TestingCopyForVirtualCluster(input *Values) { for slot := slotIdx(0); slot < slotIdx(len(registry)); slot++ { s := slotTable[slot] - if s.Class() != TenantWritable && s.Class() != TenantReadOnly { + if s.Class() != ApplicationLevel && s.Class() != SystemVisible { continue } diff --git a/pkg/spanconfig/spanconfigjob/job.go b/pkg/spanconfig/spanconfigjob/job.go index 480c1923513e..9ca10695853d 100644 --- a/pkg/spanconfig/spanconfigjob/job.go +++ b/pkg/spanconfig/spanconfigjob/job.go @@ -37,7 +37,7 @@ type resumer struct { var _ jobs.Resumer = (*resumer)(nil) var reconciliationJobCheckpointInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "spanconfig.reconciliation_job.checkpoint_interval", "the frequency at which the span config reconciliation job checkpoints itself", 5*time.Second, diff --git a/pkg/spanconfig/spanconfiglimiter/limiter.go b/pkg/spanconfig/spanconfiglimiter/limiter.go index 56a561d71edd..ac2aa7ea4e68 100644 --- a/pkg/spanconfig/spanconfiglimiter/limiter.go +++ b/pkg/spanconfig/spanconfiglimiter/limiter.go @@ -30,7 +30,7 @@ var _ spanconfig.Limiter = &Limiter{} // tenantLimitSetting controls how many span configs a secondary tenant is // allowed to install. It's settable only by the system tenant. var tenantLimitSetting = settings.RegisterIntSetting( - settings.TenantReadOnly, + settings.SystemVisible, "spanconfig.tenant_limit", "limit on the number of span configs that can be set up by a virtual cluster", 5000, diff --git a/pkg/spanconfig/spanconfigmanager/manager.go b/pkg/spanconfig/spanconfigmanager/manager.go index 123b8c682acd..62a771e883f3 100644 --- a/pkg/spanconfig/spanconfigmanager/manager.go +++ b/pkg/spanconfig/spanconfigmanager/manager.go @@ -32,7 +32,7 @@ import ( // will be started. It has no effect unless // spanconfig.reconciliation_job.enabled is configured. var checkReconciliationJobInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "spanconfig.reconciliation_job.check_interval", "the frequency at which to check if the span config reconciliation job exists (and to start it if not)", 10*time.Minute, @@ -44,7 +44,7 @@ var checkReconciliationJobInterval = settings.RegisterDurationSetting( // TODO(irfansharif): This should be a tenant read-only setting once the work // for #73349 is completed. var jobEnabledSetting = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "spanconfig.reconciliation_job.enabled", "enable the use of the kv accessor", true) diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go index 8cced95faaa6..fdabf188a259 100644 --- a/pkg/sql/alter_database.go +++ b/pkg/sql/alter_database.go @@ -289,7 +289,7 @@ type alterDatabaseDropRegionNode struct { } var allowDropFinalRegion = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.multiregion.drop_primary_region.enabled", "allows dropping the PRIMARY REGION of a database if it is the last region", true, diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index 280917fce380..05a56a32844d 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -71,7 +71,7 @@ const ( ) var changeOwnPasswordEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index b3ec914420b8..8f070d19666b 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -656,7 +656,7 @@ var defaultSingleQueryForRoleMembershipCache = util.ConstantWithMetamorphicTestB ) var useSingleQueryForRoleMembershipCache = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.auth.resolve_membership_single_scan.enabled", "determines whether to populate the role membership cache with a single scan", defaultSingleQueryForRoleMembershipCache, diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 5a0ba70cbceb..5e668bf933d1 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -79,7 +79,7 @@ const ( // entries for before we attempt to fill in a single index batch before queueing // it up for ingestion and progress reporting in the index backfiller processor. var indexBackfillBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.index_backfill.batch_size", "the number of rows for which we construct index entries in a single batch", 50000, @@ -89,7 +89,7 @@ var indexBackfillBatchSize = settings.RegisterIntSetting( // columnBackfillBatchSize is the maximum number of rows we update at once when // adding or removing columns. var columnBackfillBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.column_backfill.batch_size", "the number of rows updated at a time to add/remove columns", 200, @@ -99,7 +99,7 @@ var columnBackfillBatchSize = settings.RegisterIntSetting( // columnBackfillUpdateChunkSizeThresholdBytes is the byte size threshold beyond which // an update batch is run at once when adding or removing columns. var columnBackfillUpdateChunkSizeThresholdBytes = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.column_backfill.update_chunk_size_threshold_bytes", "the batch size in bytes above which an update is immediately run when adding/removing columns", 10<<20, /* 10 MiB */ diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 256e7a18a23d..4b4f0c18ebc2 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -50,7 +50,7 @@ import ( // not actually perform any checkpointing. The reason it has been moved here from // sql is to avoid any dependency cycles inside the declarative schema changer. var IndexBackfillCheckpointInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.index_backfill.checkpoint_interval", "the amount of time between index backfill checkpoint updates", 30*time.Second, diff --git a/pkg/sql/backfill/mvcc_index_merger.go b/pkg/sql/backfill/mvcc_index_merger.go index ab943f03e08f..a65b871cbaae 100644 --- a/pkg/sql/backfill/mvcc_index_merger.go +++ b/pkg/sql/backfill/mvcc_index_merger.go @@ -43,7 +43,7 @@ import ( // indexBackfillMergeBatchSize is the maximum number of rows we attempt to merge // in a single transaction during the merging process. var indexBackfillMergeBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.index_backfill.merge_batch_size", "the number of rows we merge between temporary and adding indexes in a single batch", 1000, @@ -54,7 +54,7 @@ var indexBackfillMergeBatchSize = settings.RegisterIntSetting( // merge from the temporary index in a single transaction during the merging // process. var indexBackfillMergeBatchBytes = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.index_backfill.merge_batch_bytes", "the max number of bytes we merge between temporary and adding indexes in a single batch", 16<<20, @@ -66,7 +66,7 @@ var indexBackfillMergeBatchBytes = settings.RegisterIntSetting( // default to 4 as higher values didn't seem to improve the index build times in // the schemachange/index/tpcc/w=1000 roachtest. var indexBackfillMergeNumWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.index_backfill.merge_num_workers", "the number of parallel merges per node in the cluster", 4, diff --git a/pkg/sql/catalog/hydrateddesccache/hydratedcache.go b/pkg/sql/catalog/hydrateddesccache/hydratedcache.go index 61dcb1a549de..0743fb292695 100644 --- a/pkg/sql/catalog/hydrateddesccache/hydratedcache.go +++ b/pkg/sql/catalog/hydrateddesccache/hydratedcache.go @@ -161,7 +161,7 @@ var ( // CacheSize controls the size of the LRU cache. var CacheSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.hydrated_tables.cache_size", "number of table descriptor versions retained in the hydratedtables LRU cache", 128, diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 8a2a6dd136da..77e391a06b15 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -60,14 +60,14 @@ var errReadOlderVersion = errors.New("read older descriptor version from store") // LeaseDuration controls the duration of sql descriptor leases. var LeaseDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.descriptor_lease_duration", "mean duration of sql descriptor leases, this actual duration is jitterred", base.DefaultDescriptorLeaseDuration) // LeaseJitterFraction controls the percent jitter around sql lease durations var LeaseJitterFraction = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.descriptor_lease_jitter_fraction", "mean duration of sql descriptor leases, this actual duration is jitterred", base.DefaultDescriptorLeaseJitterFraction, @@ -1224,7 +1224,7 @@ func (m *Manager) watchForUpdates(ctx context.Context, descUpdateCh chan<- catal // leaseRefreshLimit is the upper-limit on the number of descriptor leases // that will continuously have their lease refreshed. var leaseRefreshLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.tablecache.lease.refresh_limit", "maximum number of descriptors to periodically refresh leases for", 500, diff --git a/pkg/sql/catalog/lease/storage.go b/pkg/sql/catalog/lease/storage.go index 14eec95b42b6..db6c337628e4 100644 --- a/pkg/sql/catalog/lease/storage.go +++ b/pkg/sql/catalog/lease/storage.go @@ -81,7 +81,7 @@ type writer interface { // LeaseRenewalDuration controls the default time before a lease expires when // acquisition to renew the lease begins. var LeaseRenewalDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.descriptor_lease_renewal_fraction", "controls the default time before a lease expires when acquisition to renew the lease begins", base.DefaultDescriptorLeaseRenewalTimeout) @@ -89,7 +89,7 @@ var LeaseRenewalDuration = settings.RegisterDurationSetting( // LeaseRenewalCrossValidate controls if cross validation should be done during // lease renewal. var LeaseRenewalCrossValidate = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.descriptor_lease_renewal_cross_validation.enabled", "controls if cross validation should be done during lease renewal", base.DefaultLeaseRenewalCrossValidate) diff --git a/pkg/sql/catalog/randgen/randgen.go b/pkg/sql/catalog/randgen/randgen.go index d64ec2588ea3..470162e2dd3e 100644 --- a/pkg/sql/catalog/randgen/randgen.go +++ b/pkg/sql/catalog/randgen/randgen.go @@ -102,7 +102,7 @@ const genEnabledSettingName = "sql.schema.test_object_generator.enabled" // force disable the functionality from an unprivileged secondary // tenant. var genEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, genEnabledSettingName, "enable the generate_test_objects function", true, @@ -111,7 +111,7 @@ var genEnabled = settings.RegisterBoolSetting( const genEnabledForNonAdminsSettingName = "sql.schema.test_object_generator.non_admin.enabled" var genEnabledForNonAdmins = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, genEnabledForNonAdminsSettingName, "allow non-admin users to use the generate_test_objects function", false, diff --git a/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go b/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go index db266d0bd486..8381ce0e2bf0 100644 --- a/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go +++ b/pkg/sql/catalog/schematelemetry/schematelemetrycontroller/controller.go @@ -39,7 +39,7 @@ const SchemaTelemetryScheduleName = "sql-schema-telemetry" // SchemaTelemetryRecurrence is the cron-tab string specifying the recurrence // for schema telemetry job. var SchemaTelemetryRecurrence = settings.RegisterStringSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.schema.telemetry.recurrence", "cron-tab recurrence for SQL schema telemetry job", "@weekly", /* defaultValue */ diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index 137c0480780a..6da84a050b8e 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -306,7 +306,7 @@ func EvalShardBucketCount( // count for hash sharded index when bucket count is not specified in index // definition. var DefaultHashShardedIndexBucketCount = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.default_hash_sharded_index_bucket_count", "used as bucket count if bucket count is not specified in hash sharded index definition", 16, diff --git a/pkg/sql/closed_session_cache.go b/pkg/sql/closed_session_cache.go index 7d731a128c70..a12598231df3 100644 --- a/pkg/sql/closed_session_cache.go +++ b/pkg/sql/closed_session_cache.go @@ -30,7 +30,7 @@ type timeSource func() time.Time // ClosedSessionCacheCapacity is the cluster setting that controls the maximum number // of sessions in the cache. var ClosedSessionCacheCapacity = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.closed_session_cache.capacity", "the maximum number of sessions in the cache", 1000, // TODO(gtr): Totally arbitrary for now, adjust later. @@ -39,7 +39,7 @@ var ClosedSessionCacheCapacity = settings.RegisterIntSetting( // ClosedSessionCacheTimeToLive is the cluster setting that controls the maximum time // to live for a session's information in the cache, in seconds. var ClosedSessionCacheTimeToLive = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.closed_session_cache.time_to_live", "the maximum time to live, in seconds", 3600, // One hour diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index e8d78fbd7b7c..cbe1b8c99486 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -1886,7 +1886,7 @@ func (r *renderExprCountVisitor) VisitPost(expr tree.Expr) tree.Expr { } var renderWrappingRowCountThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.vectorize_render_wrapping.max_row_count", "determines the maximum number of estimated rows that flow through the render "+ "expressions up to which we handle those renders by wrapping a row-by-row processor", @@ -1895,7 +1895,7 @@ var renderWrappingRowCountThreshold = settings.RegisterIntSetting( ) var renderWrappingRenderCountThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.vectorize_render_wrapping.min_render_count", "determines the minimum number of render expressions for which we fall "+ "back to handling renders by wrapping a row-by-row processor", diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index e62930852590..464ea40682b8 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -513,7 +513,7 @@ const HashAggregationDiskSpillingEnabledSettingName = "sql.distsql.temp_storage. // HashAggregationDiskSpillingEnabled is a cluster setting that allows to // disable hash aggregator disk spilling. var HashAggregationDiskSpillingEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, HashAggregationDiskSpillingEnabledSettingName, "set to false to disable hash aggregator disk spilling "+ "(this will improve performance, but the query might hit the memory limit)", diff --git a/pkg/sql/colfetcher/cfetcher_wrapper.go b/pkg/sql/colfetcher/cfetcher_wrapper.go index 2df2717a3947..a524294e19af 100644 --- a/pkg/sql/colfetcher/cfetcher_wrapper.go +++ b/pkg/sql/colfetcher/cfetcher_wrapper.go @@ -38,7 +38,7 @@ import ( // DirectScansEnabled is a cluster setting that controls whether the KV // projection pushdown infrastructure can be used. var DirectScansEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.direct_columnar_scans.enabled", "set to true to enable the 'direct' columnar scans in the KV layer", directScansEnabledDefault, diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index ea66b7f16f2d..69524e38b973 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -469,7 +469,7 @@ const productionIndexJoinUsingStreamerBatchSize = 8 << 20 /* 8MiB */ // construct a single lookup KV batch by the ColIndexJoin when it is using the // Streamer API. var IndexJoinStreamerBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.index_join_streamer.batch_size", "size limit on the input rows to construct a single lookup KV batch "+ "(by the ColIndexJoin operator when using the Streamer API)", diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 9e9e38771065..28ab3e7fbef3 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -95,7 +95,7 @@ var errAcquireTimeout = pgerror.New( ) var fdCountingSemaphoreMaxRetries = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.acquire_vec_fds.max_retries", "determines the number of retries performed during the acquisition of "+ "file descriptors needed for disk-spilling operations, set to 0 for "+ diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index d86314936247..7d8c0454dbef 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -92,7 +92,7 @@ import ( ) var maxNumNonAdminConnections = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.max_connections_per_gateway", "the maximum number of SQL connections per gateway allowed at a given time "+ "(note: this will only limit future connection attempts and will not affect already established connections). "+ @@ -105,7 +105,7 @@ var maxNumNonAdminConnections = settings.RegisterIntSetting( // This setting may be extended one day to include an arbitrary list of users to exclude from connection limiting. // This setting may be removed one day. var maxNumNonRootConnections = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cockroach_cloud.max_client_connections_per_gateway", "this setting is intended to be used by Cockroach Cloud for limiting connections to serverless clusters. "+ "The maximum number of SQL connections per gateway allowed at a given time "+ @@ -121,7 +121,7 @@ var maxNumNonRootConnections = settings.RegisterIntSetting( // connections to serverless clusters. // This setting may be removed one day. var maxNumNonRootConnectionsReason = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.cockroach_cloud.max_client_connections_per_gateway_reason", "a reason to provide in the error message for connections that are denied due to "+ "server.cockroach_cloud.max_client_connections_per_gateway", @@ -3413,7 +3413,7 @@ func (ex *connExecutor) setTransactionModes( } var allowSnapshotIsolation = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.txn.snapshot_isolation_syntax.enabled", "set to true to allow transactions to use the SNAPSHOT isolation level. At "+ "the time of writing, this setting is intended only for usage by "+ @@ -3422,7 +3422,7 @@ var allowSnapshotIsolation = settings.RegisterBoolSetting( ) var allowReadCommittedIsolation = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.txn.read_committed_syntax.enabled", "set to true to allow transactions to use the READ COMMITTED isolation "+ "level if specified by BEGIN/SET commands", diff --git a/pkg/sql/contention/cluster_settings.go b/pkg/sql/contention/cluster_settings.go index ff6aeabbf275..e8351db48b95 100644 --- a/pkg/sql/contention/cluster_settings.go +++ b/pkg/sql/contention/cluster_settings.go @@ -19,7 +19,7 @@ import ( // TxnIDResolutionInterval is the cluster setting that controls how often the // Transaction ID Resolution is performed. var TxnIDResolutionInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.contention.event_store.resolution_interval", "the interval at which transaction fingerprint ID resolution is "+ "performed (set to 0 to disable)", @@ -29,7 +29,7 @@ var TxnIDResolutionInterval = settings.RegisterDurationSetting( // StoreCapacity is the cluster setting that controls the // maximum size of the contention event store. var StoreCapacity = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.contention.event_store.capacity", "the in-memory storage capacity per-node of contention event store", 64*1024*1024, // 64 MB per node. @@ -39,7 +39,7 @@ var StoreCapacity = settings.RegisterByteSizeSetting( // contention durations. Only the contention events whose duration exceeds the // threshold will be collected into crdb_internal.transaction_contention_events. var DurationThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.contention.event_store.duration_threshold", "minimum contention duration to cause the contention events to be collected "+ "into crdb_internal.transaction_contention_events", diff --git a/pkg/sql/contention/txnidcache/cluster_settings.go b/pkg/sql/contention/txnidcache/cluster_settings.go index 3b7750d836c0..2edb2551bd1f 100644 --- a/pkg/sql/contention/txnidcache/cluster_settings.go +++ b/pkg/sql/contention/txnidcache/cluster_settings.go @@ -14,7 +14,7 @@ import "github.com/cockroachdb/cockroach/pkg/settings" // MaxSize limits the maximum byte size can be used by the TxnIDCache. var MaxSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, `sql.contention.txn_id_cache.max_size`, "the maximum byte size TxnID cache will use (set to 0 to disable)", 64*1024*1024, // 64MiB diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index fbd4e201cb46..45b00480f500 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -706,7 +706,7 @@ CREATE TABLE crdb_internal.pg_catalog_table_is_implemented ( // is used to define the AS OF time for querying the system.table_statistics // table when building crdb_internal.table_row_statistics. var statsAsOfTimeClusterMode = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.crdb_internal.table_row_statistics.as_of_time", "historical query time used to build the crdb_internal.table_row_statistics table", -10*time.Second, diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index f34f17166c7f..78c972f23e11 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -46,7 +46,7 @@ import ( // createStatsPostEvents controls the cluster setting for logging // automatic table statistics collection to the event log. var createStatsPostEvents = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.post_events.enabled", "if set, an event is logged for every CREATE STATISTICS job", false, @@ -55,7 +55,7 @@ var createStatsPostEvents = settings.RegisterBoolSetting( // featureStatsEnabled is used to enable and disable the CREATE STATISTICS and // ANALYZE features. var featureStatsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.stats.enabled", "set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/sql/delegate/show_tables.go b/pkg/sql/delegate/show_tables.go index 4162ab894fce..5caab64ebbcb 100644 --- a/pkg/sql/delegate/show_tables.go +++ b/pkg/sql/delegate/show_tables.go @@ -21,7 +21,7 @@ import ( ) var showEstimatedRowCountClusterSetting = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.show_tables.estimated_row_count.enabled", "whether the estimated_row_count is shown on SHOW TABLES. Turning this off "+ "will improve SHOW TABLES performance.", diff --git a/pkg/sql/deprecatedshowranges/condition.go b/pkg/sql/deprecatedshowranges/condition.go index 4292fb23a33e..c4f922c4b017 100644 --- a/pkg/sql/deprecatedshowranges/condition.go +++ b/pkg/sql/deprecatedshowranges/condition.go @@ -99,7 +99,7 @@ const ShowRangesDeprecatedBehaviorSettingName = "sql.show_ranges_deprecated_beha // ShowRangesDeprecatedBehaviorSetting is the setting that controls // the behavior. Exported for use in tests. var ShowRangesDeprecatedBehaviorSetting = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, ShowRangesDeprecatedBehaviorSettingName, "if set, SHOW RANGES and crdb_internal.ranges{_no_leases} behave with deprecated pre-v23.1 semantics."+ " NB: the new SHOW RANGES interface has richer WITH options "+ diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index c6d48c59a455..c2dfacf8e60a 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -63,7 +63,7 @@ var ( // whether the CREATE privilege is given to the `public` role on the `public` // schema at the time the schema is created. var PublicSchemaCreatePrivilegeEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.auth.public_schema_create_privilege.enabled", "determines whether to grant all users the CREATE privileges on the public "+ "schema when it is created", @@ -347,7 +347,7 @@ const DefaultPrimaryRegionClusterSettingName = "sql.defaults.primary_region" // DefaultPrimaryRegion is a cluster setting that contains the default primary region. var DefaultPrimaryRegion = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, DefaultPrimaryRegionClusterSettingName, `if not empty, all databases created without a PRIMARY REGION will `+ `implicitly have the given PRIMARY REGION`, @@ -367,7 +367,7 @@ const SecondaryTenantsMultiRegionAbstractionsEnabledSettingName = "sql.virtual_c // already been configured. It only affects regions being added to new // databases. var SecondaryTenantsMultiRegionAbstractionsEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.multi_region.allow_abstractions_for_secondary_tenants.enabled", // internal key, name defined above "allow the use of multi-region abstractions and syntax in virtual clusters", false, diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 6c9e5e1844c2..14013583ef0a 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1729,7 +1729,7 @@ const defaultLocalScansConcurrencyLimit = 1024 // "additional" we mean having more processors than one in the same stage of the // physical plan. var localScansConcurrencyLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.local_scans.concurrency_limit", "maximum number of additional goroutines for performing scans in local plans", defaultLocalScansConcurrencyLimit, diff --git a/pkg/sql/distsql_plan_stats.go b/pkg/sql/distsql_plan_stats.go index 53c46078f86f..17941d1d95d5 100644 --- a/pkg/sql/distsql_plan_stats.go +++ b/pkg/sql/distsql_plan_stats.go @@ -48,7 +48,7 @@ type requestedStat struct { // construction. For larger tables, it may be beneficial to increase this number // to get a more accurate distribution. var histogramSamples = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.histogram_samples.count", "number of rows sampled for histogram construction during table statistics collection", 10000, @@ -62,7 +62,7 @@ var histogramSamples = settings.RegisterIntSetting( // The lowest TTL we recommend is 10 minutes. This value must be lower than // that. var maxTimestampAge = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.max_timestamp_age", "maximum age of timestamp during table statistics collection", 5*time.Minute, diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index bd3761d6db32..12169e3f6967 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -61,7 +61,7 @@ import ( ) var settingDistSQLNumRunners = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.num_runners", "determines the number of DistSQL runner goroutines used for issuing SetupFlow RPCs", // We use GOMAXPROCS instead of NumCPU because the former could be adjusted @@ -1911,7 +1911,7 @@ func (dsp *DistSQLPlanner) planAndRunSubquery( } var distributedQueryRerunAsLocalEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.distributed_query_rerun_locally.enabled", "determines whether the distributed plans can be rerun locally for some errors", true, @@ -2202,7 +2202,7 @@ func (dsp *DistSQLPlanner) PlanAndRunCascadesAndChecks( } var parallelizeChecks = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.parallelize_checks.enabled", "determines whether FOREIGN KEY and UNIQUE constraint checks are performed in parallel", true, @@ -2211,7 +2211,7 @@ var parallelizeChecks = settings.RegisterBoolSetting( // parallelChecksConcurrencyLimit controls the maximum number of additional // goroutines that can be used to run checks in parallel. var parallelChecksConcurrencyLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.parallelize_checks.concurrency_limit", "maximum number of additional goroutines to run checks in parallel", // The default here is picked somewhat arbitrarily - the thinking is that we diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go index 7de178d2c91b..e1406da93cb3 100644 --- a/pkg/sql/event_log.go +++ b/pkg/sql/event_log.go @@ -433,7 +433,7 @@ func LogEventForJobs( } var eventLogSystemTableEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.eventlog.enabled", "if set, logged notable events are also stored in the table system.eventlog", true, diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index 4f672361ede1..564cd25ac2d5 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -47,7 +47,7 @@ import ( // logStatementsExecuteEnabled causes the Executor to log executed // statements and, if any, resulting errors. var logStatementsExecuteEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.trace.log_statement_execute", "set to true to enable logging of all executed statements", false, @@ -55,7 +55,7 @@ var logStatementsExecuteEnabled = settings.RegisterBoolSetting( settings.WithPublic) var slowQueryLogThreshold = settings.RegisterDurationSettingWithExplicitUnit( - settings.TenantWritable, + settings.ApplicationLevel, "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", @@ -65,7 +65,7 @@ var slowQueryLogThreshold = settings.RegisterDurationSettingWithExplicitUnit( ) var slowInternalQueryLogEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.log.slow_query.internal_queries.enabled", "when set to true, internal queries which exceed the slow query log threshold "+ "are logged to a separate log. Must have the slow query log enabled for this "+ @@ -74,7 +74,7 @@ var slowInternalQueryLogEnabled = settings.RegisterBoolSetting( settings.WithPublic) var slowQueryLogFullTableScans = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.log.slow_query.experimental_full_table_scans.enabled", "when set to true, statements that perform a full table/index scan will be logged to the "+ "slow query log even if they do not meet the latency threshold. Must have the slow query "+ @@ -83,14 +83,14 @@ var slowQueryLogFullTableScans = settings.RegisterBoolSetting( settings.WithPublic) var adminAuditLogEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.log.admin_audit.enabled", "when set, log SQL queries that are executed by a user with admin privileges", false, ) var telemetryLoggingEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.query_sampling.enabled", "when set to true, executed queries will emit an event on the telemetry logging channel", // Note: Usage of an env var here makes it possible to set a default without diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 06170bce1cf7..944d08bcd8bb 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -126,7 +126,7 @@ import ( // ClusterOrganization is the organization name. var ClusterOrganization = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cluster.organization", "organization name", "", @@ -141,7 +141,7 @@ func ClusterIsInternal(sv *settings.Values) bool { // ClusterSecret is a cluster specific secret. This setting is // non-reportable. var ClusterSecret = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "cluster.secret", "cluster specific secret", "", @@ -155,7 +155,7 @@ var ClusterSecret = settings.RegisterStringSetting( // 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 = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.default_int_size", "the size, in bytes, of an INT type", 8, @@ -171,7 +171,7 @@ var defaultIntSize = settings.RegisterIntSetting( const allowCrossDatabaseFKsSetting = "sql.cross_db_fks.enabled" var allowCrossDatabaseFKs = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, allowCrossDatabaseFKsSetting, "if true, creating foreign key references across databases is allowed", false, @@ -180,7 +180,7 @@ var allowCrossDatabaseFKs = settings.RegisterBoolSetting( const allowCrossDatabaseViewsSetting = "sql.cross_db_views.enabled" var allowCrossDatabaseViews = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, allowCrossDatabaseViewsSetting, "if true, creating views that refer to other databases is allowed", false, @@ -189,7 +189,7 @@ var allowCrossDatabaseViews = settings.RegisterBoolSetting( const allowCrossDatabaseSeqOwnerSetting = "sql.cross_db_sequence_owners.enabled" var allowCrossDatabaseSeqOwner = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, allowCrossDatabaseSeqOwnerSetting, "if true, creating sequences owned by tables from other databases is allowed", false, @@ -198,7 +198,7 @@ var allowCrossDatabaseSeqOwner = settings.RegisterBoolSetting( const allowCrossDatabaseSeqReferencesSetting = "sql.cross_db_sequence_references.enabled" var allowCrossDatabaseSeqReferences = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, allowCrossDatabaseSeqReferencesSetting, "if true, sequences referenced by tables from other databases are allowed", false, @@ -209,7 +209,7 @@ var allowCrossDatabaseSeqReferences = settings.RegisterBoolSetting( // // This setting has no effect on zone configurations that have already been set. var SecondaryTenantZoneConfigsEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.zone_configs.allow_for_secondary_tenant.enabled", "enable the use of ALTER CONFIGURE ZONE in virtual clusters", false, @@ -220,7 +220,7 @@ var SecondaryTenantZoneConfigsEnabled = settings.RegisterBoolSetting( // run ALTER TABLE/INDEX ... SPLIT AT statements. It has no effect for the // system tenant. var SecondaryTenantSplitAtEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.split_at.allow_for_secondary_tenant.enabled", "enable the use of ALTER TABLE/INDEX ... SPLIT AT in virtual clusters", false, @@ -231,7 +231,7 @@ var SecondaryTenantSplitAtEnabled = settings.RegisterBoolSetting( // run ALTER TABLE/INDEX ... SCATTER statements. It has no effect for the // system tenant. var SecondaryTenantScatterEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.scatter.allow_for_secondary_tenant.enabled", "enable the use of ALTER TABLE/INDEX ... SCATTER in virtual clusters", false, @@ -246,7 +246,7 @@ var SecondaryTenantScatterEnabled = settings.RegisterBoolSetting( // all execution because traces are gathered for all transactions even // if they are not output. var traceTxnThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.trace.txn.enable_threshold", "enables tracing on all transactions; transactions open for longer than "+ "this duration will have their trace logged (set to 0 to disable); "+ @@ -261,7 +261,7 @@ var traceTxnThreshold = settings.RegisterDurationSetting( // to be able to reduce the noise associated with a larger transaction (e.g. // round trips to client). var TraceStmtThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.trace.stmt.enable_threshold", "enables tracing on all statements; statements executing for longer than "+ "this duration will have their trace logged (set to 0 to disable); "+ @@ -276,7 +276,7 @@ var TraceStmtThreshold = settings.RegisterDurationSetting( // non-trivial performance impact and also reveals SQL statements // which may be a privacy concern. var traceSessionEventLogEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.trace.session_eventlog.enabled", "set to true to enable session tracing; "+ "note that enabling this may have a negative performance impact", @@ -290,7 +290,7 @@ const ReorderJoinsLimitClusterSettingName = "sql.defaults.reorder_joins_limit" // ReorderJoinsLimitClusterValue controls the cluster default for the maximum // number of joins reordered. var ReorderJoinsLimitClusterValue = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, ReorderJoinsLimitClusterSettingName, "default number of joins to reorder", opt.DefaultJoinOrderLimit, @@ -299,14 +299,14 @@ var ReorderJoinsLimitClusterValue = settings.RegisterIntSetting( ) var requireExplicitPrimaryKeysClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.require_explicit_primary_keys.enabled", "default value for requiring explicit primary keys in CREATE TABLE statements", false, settings.WithPublic) var placementEnabledClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.multiregion_placement_policy.enabled", "default value for enable_multiregion_placement_policy;"+ " allows for use of PLACEMENT RESTRICTED", @@ -314,7 +314,7 @@ var placementEnabledClusterMode = settings.RegisterBoolSetting( ) var autoRehomingEnabledClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.experimental_auto_rehoming.enabled", "default value for experimental_enable_auto_rehoming;"+ " allows for rows in REGIONAL BY ROW tables to be auto-rehomed on UPDATE", @@ -322,7 +322,7 @@ var autoRehomingEnabledClusterMode = settings.RegisterBoolSetting( ) var onUpdateRehomeRowEnabledClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.on_update_rehome_row.enabled", "default value for on_update_rehome_row;"+ " enables ON UPDATE rehome_row() expressions to trigger on updates", @@ -330,21 +330,21 @@ var onUpdateRehomeRowEnabledClusterMode = settings.RegisterBoolSetting( settings.WithPublic) var temporaryTablesEnabledClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.experimental_temporary_tables.enabled", "default value for experimental_enable_temp_tables; allows for use of temporary tables by default", false, settings.WithPublic) var implicitColumnPartitioningEnabledClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.experimental_implicit_column_partitioning.enabled", "default value for experimental_enable_temp_tables; allows for the use of implicit column partitioning", false, settings.WithPublic) var overrideMultiRegionZoneConfigClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.override_multi_region_zone_config.enabled", "default value for override_multi_region_zone_config; "+ "allows for overriding the zone configs of a multi-region table or database", @@ -352,7 +352,7 @@ var overrideMultiRegionZoneConfigClusterMode = settings.RegisterBoolSetting( settings.WithPublic) var maxHashShardedIndexRangePreSplit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.hash_sharded_range_pre_split.max", "max pre-split ranges to have when adding hash sharded index to an existing table", 16, @@ -360,14 +360,14 @@ var maxHashShardedIndexRangePreSplit = settings.RegisterIntSetting( settings.WithPublic) var zigzagJoinClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.zigzag_join.enabled", "default value for enable_zigzag_join session setting; disallows use of zig-zag join by default", false, settings.WithPublic) var optDrivenFKCascadesClusterLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.foreign_key_cascades_limit", "default value for foreign_key_cascades_limit session setting; limits the number of cascading operations that run as part of a single query", 10000, @@ -375,7 +375,7 @@ var optDrivenFKCascadesClusterLimit = settings.RegisterIntSetting( settings.WithPublic) var preferLookupJoinsForFKs = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, @@ -388,7 +388,7 @@ var preferLookupJoinsForFKs = settings.RegisterBoolSetting( // haven't been collected. Collection of histograms is controlled by the // cluster setting sql.stats.histogram_collection.enabled. var optUseHistogramsClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.optimizer_use_histograms.enabled", "default value for optimizer_use_histograms session setting; enables usage of histograms in the optimizer by default", true, @@ -401,7 +401,7 @@ var optUseHistogramsClusterMode = settings.RegisterBoolSetting( // if they haven't been collected. Collection of multi-column stats is // controlled by the cluster setting sql.stats.multi_column_collection.enabled. var optUseMultiColStatsClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, @@ -413,7 +413,7 @@ var optUseMultiColStatsClusterMode = settings.RegisterBoolSetting( // searched for matching rows before remote nodes, in the hope that the // execution engine can avoid visiting remote nodes. var localityOptimizedSearchMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.locality_optimized_partitioned_index_scan.enabled", "default value for locality_optimized_partitioned_index_scan session setting; "+ "enables searching for rows in the current region before searching remote regions", @@ -421,21 +421,21 @@ var localityOptimizedSearchMode = settings.RegisterBoolSetting( settings.WithPublic) var implicitSelectForUpdateClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, settings.WithPublic) var insertFastPathClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.insert_fast_path.enabled", "default value for enable_insert_fast_path session setting; enables a specialized insert path", true, settings.WithPublic) var experimentalAlterColumnTypeGeneralMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.experimental_alter_column_type.enabled", "default value for experimental_alter_column_type session setting; "+ "enables the use of ALTER COLUMN TYPE for general conversions", @@ -443,7 +443,7 @@ var experimentalAlterColumnTypeGeneralMode = settings.RegisterBoolSetting( settings.WithPublic) var clusterStatementTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.statement_timeout", "default value for the statement_timeout; "+ "default value for the statement_timeout session setting; controls the "+ @@ -454,7 +454,7 @@ var clusterStatementTimeout = settings.RegisterDurationSetting( settings.WithPublic) var clusterLockTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.lock_timeout", "default value for the lock_timeout; "+ "default value for the lock_timeout session setting; controls the "+ @@ -466,7 +466,7 @@ var clusterLockTimeout = settings.RegisterDurationSetting( settings.WithPublic) var clusterIdleInSessionTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.idle_in_session_timeout", "default value for the idle_in_session_timeout; "+ "default value for the idle_in_session_timeout session setting; controls the "+ @@ -477,7 +477,7 @@ var clusterIdleInSessionTimeout = settings.RegisterDurationSetting( settings.WithPublic) var clusterIdleInTransactionSessionTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.idle_in_transaction_session_timeout", "default value for the idle_in_transaction_session_timeout; controls the "+ "duration a session is permitted to idle in a transaction before the "+ @@ -489,7 +489,7 @@ var clusterIdleInTransactionSessionTimeout = settings.RegisterDurationSetting( // TODO(rytaft): remove this once unique without index constraints are fully // supported. var experimentalUniqueWithoutIndexConstraintsMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.experimental_enable_unique_without_index_constraints.enabled", "default value for experimental_enable_unique_without_index_constraints session setting;"+ "disables unique without index constraints by default", @@ -497,7 +497,7 @@ var experimentalUniqueWithoutIndexConstraintsMode = settings.RegisterBoolSetting settings.WithPublic) var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.use_declarative_schema_changer", "default value for use_declarative_schema_changer session setting;"+ "disables new schema changer by default", @@ -511,14 +511,14 @@ var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting( settings.WithPublic) var stubCatalogTablesEnabledClusterValue = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, `sql.defaults.stub_catalog_tables.enabled`, `default value for stub_catalog_tables session setting`, true, settings.WithPublic) var experimentalComputedColumnRewrites = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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) ...]'", @@ -530,7 +530,7 @@ var experimentalComputedColumnRewrites = settings.RegisterStringSetting( ) var propagateInputOrdering = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, `sql.defaults.propagate_input_ordering.enabled`, `default value for the experimental propagate_input_ordering session variable`, false, @@ -539,7 +539,7 @@ var propagateInputOrdering = settings.RegisterBoolSetting( // settingWorkMemBytes is a cluster setting that determines the maximum amount // of RAM that a processor can use. var settingWorkMemBytes = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.temp_storage.workmem", "maximum amount of memory in bytes a processor can use before falling back to temp storage", execinfra.DefaultMemoryLimit, /* 64MiB */ @@ -555,7 +555,7 @@ const ExperimentalDistSQLPlanningClusterSettingName = "sql.defaults.experimental // optimizer-driven DistSQL planning that sidesteps intermediate planNode // transition when going from opt.Expr to DistSQL processor specs. var experimentalDistSQLPlanningClusterMode = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, ExperimentalDistSQLPlanningClusterSettingName, "default experimental_distsql_planning mode; enables experimental opt-driven DistSQL planning", "off", @@ -572,7 +572,7 @@ const VectorizeClusterSettingName = "sql.defaults.vectorize" // VectorizeClusterMode controls the cluster default for when automatic // vectorization is enabled. var VectorizeClusterMode = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, VectorizeClusterSettingName, "default vectorize mode", "on", @@ -587,7 +587,7 @@ var VectorizeClusterMode = settings.RegisterEnumSetting( // DistSQLClusterExecMode controls the cluster default for when DistSQL is used. var DistSQLClusterExecMode = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.distsql", "default distributed SQL execution mode", "auto", @@ -602,7 +602,7 @@ var DistSQLClusterExecMode = settings.RegisterEnumSetting( // SerialNormalizationMode controls how the SERIAL type is interpreted in table // definitions. var SerialNormalizationMode = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.serial_normalization", "default handling of SERIAL in table definitions", "rowid", @@ -616,7 +616,7 @@ var SerialNormalizationMode = settings.RegisterEnumSetting( settings.WithPublic) var disallowFullTableScans = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, `sql.defaults.disallow_full_table_scans.enabled`, "setting to true rejects queries that have planned a full table scan", false, @@ -624,7 +624,7 @@ var disallowFullTableScans = settings.RegisterBoolSetting( // intervalStyle controls intervals representation. var intervalStyle = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.intervalstyle", "default value for IntervalStyle session setting", strings.ToLower(duration.IntervalStyle_POSTGRES.String()), @@ -645,7 +645,7 @@ var dateStyleEnumMap = map[int64]string{ // dateStyle controls dates representation. var dateStyle = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.datestyle", "default value for DateStyle session setting", pgdate.DefaultDateStyle().SQLString(), @@ -653,7 +653,7 @@ var dateStyle = settings.RegisterEnumSetting( settings.WithPublic) var txnRowsWrittenLog = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.transaction_rows_written_log", "the threshold for the number of rows written by a SQL transaction "+ "which - once exceeded - will trigger a logging event to SQL_PERF (or "+ @@ -663,7 +663,7 @@ var txnRowsWrittenLog = settings.RegisterIntSetting( settings.WithPublic) var txnRowsWrittenErr = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.transaction_rows_written_err", "the limit for the number of rows written by a SQL transaction which - "+ "once exceeded - will fail the transaction (or will trigger a logging "+ @@ -673,7 +673,7 @@ var txnRowsWrittenErr = settings.RegisterIntSetting( settings.WithPublic) var txnRowsReadLog = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.transaction_rows_read_log", "the threshold for the number of rows read by a SQL transaction "+ "which - once exceeded - will trigger a logging event to SQL_PERF (or "+ @@ -683,7 +683,7 @@ var txnRowsReadLog = settings.RegisterIntSetting( settings.WithPublic) var txnRowsReadErr = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.transaction_rows_read_err", "the limit for the number of rows read by a SQL transaction which - "+ "once exceeded - will fail the transaction (or will trigger a logging "+ @@ -695,7 +695,7 @@ var txnRowsReadErr = settings.RegisterIntSetting( // This is a float setting (rather than an int setting) because the optimizer // uses floating point for calculating row estimates. var largeFullScanRows = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.large_full_scan_rows", "default value for large_full_scan_rows session setting which determines "+ "the maximum table size allowed for a full scan when disallow_full_table_scans "+ @@ -704,14 +704,14 @@ var largeFullScanRows = settings.RegisterFloatSetting( settings.WithPublic) var costScansWithDefaultColSize = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, `sql.defaults.cost_scans_with_default_col_size.enabled`, "setting to true uses the same size for all columns to compute scan cost", false, settings.WithPublic) var enableSuperRegions = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.super_regions.enabled", "default value for enable_super_regions; "+ "allows for the usage of super regions", @@ -719,7 +719,7 @@ var enableSuperRegions = settings.RegisterBoolSetting( settings.WithPublic) var overrideAlterPrimaryRegionInSuperRegion = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.override_alter_primary_region_in_super_region.enabled", "default value for override_alter_primary_region_in_super_region; "+ "allows for altering the primary region even if the primary region is a "+ diff --git a/pkg/sql/execinfra/readerbase.go b/pkg/sql/execinfra/readerbase.go index fbfd2b6cfa68..fda11b3ae2b0 100644 --- a/pkg/sql/execinfra/readerbase.go +++ b/pkg/sql/execinfra/readerbase.go @@ -230,7 +230,7 @@ func (flowCtx *FlowCtx) UseStreamer() (bool, *kv.Txn, error) { // session variable. // TODO(yuzefovich): consider removing this at some point. var UseStreamerEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.use_streamer.enabled", "determines whether the usage of the Streamer API is allowed. "+ "Enabling this will increase the speed of lookup/index joins "+ @@ -248,7 +248,7 @@ const joinReaderIndexJoinStrategyBatchSizeDefault = 4 << 20 /* 4 MiB */ // used to construct a single lookup KV batch by // rowexec.joinReaderIndexJoinStrategy as well as colfetcher.ColIndexJoin. var JoinReaderIndexJoinStrategyBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.join_reader_index_join_strategy.batch_size", "size limit on the input rows to construct a single lookup KV batch "+ "(by the joinReader processor and the ColIndexJoin operator (when the "+ diff --git a/pkg/sql/execinfra/utils.go b/pkg/sql/execinfra/utils.go index 8a4620fa62c8..2c4efdf36e5c 100644 --- a/pkg/sql/execinfra/utils.go +++ b/pkg/sql/execinfra/utils.go @@ -59,7 +59,7 @@ func HydrateTypesInDatumInfo( // This setting is defined here instead of in package 'sql' to avoid // a dependency cycle. var IncludeRUEstimateInExplainAnalyze = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.tenant_ru_estimation.enabled", "determines whether EXPLAIN ANALYZE should return an estimate for the query's RU consumption", true, diff --git a/pkg/sql/export.go b/pkg/sql/export.go index 1c1d0e8ef7d3..832a0080cb3f 100644 --- a/pkg/sql/export.go +++ b/pkg/sql/export.go @@ -95,7 +95,7 @@ var exportOptionExpectValues = map[string]exprutil.KVStringOptValidate{ // featureExportEnabled is used to enable and disable the EXPORT feature. var featureExportEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.export.enabled", "set to true to enable exports, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/sql/flowinfra/flow_registry.go b/pkg/sql/flowinfra/flow_registry.go index 224fe3a8baca..53352b7623c7 100644 --- a/pkg/sql/flowinfra/flow_registry.go +++ b/pkg/sql/flowinfra/flow_registry.go @@ -40,7 +40,7 @@ func IsNoInboundStreamConnectionError(err error) bool { // SettingFlowStreamTimeout is a cluster setting that sets the default flow // stream timeout. var SettingFlowStreamTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.flow_stream_timeout", "amount of time incoming streams wait for a flow to be set up before erroring out", 10*time.Second, diff --git a/pkg/sql/gcjob/gc_job.go b/pkg/sql/gcjob/gc_job.go index 4f7a9326cd2f..cfc5c8da461f 100644 --- a/pkg/sql/gcjob/gc_job.go +++ b/pkg/sql/gcjob/gc_job.go @@ -55,7 +55,7 @@ func SetSmallMaxGCIntervalForTest() func() { } var idleWaitDuration = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.gc_job.idle_wait_duration", "after this duration of waiting for an update, the gc job will mark itself idle", time.Second, @@ -360,7 +360,7 @@ func waitForGC( // EmptySpanPollInterval is the interval at which the GC job will poll the // spans to determine whether the data have been garbage collected. var EmptySpanPollInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.gc_job.wait_for_gc.interval", "interval at which the GC job should poll to see if the deleted data has been GC'd", 5*time.Minute, diff --git a/pkg/sql/idxusage/cluster_settings.go b/pkg/sql/idxusage/cluster_settings.go index c9b7b3b6f712..c0043202f8e1 100644 --- a/pkg/sql/idxusage/cluster_settings.go +++ b/pkg/sql/idxusage/cluster_settings.go @@ -14,6 +14,6 @@ import "github.com/cockroachdb/cockroach/pkg/settings" // Enable determines whether to collect per-index usage statistics. var Enable = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.index_usage_stats.enabled", "collect per index usage statistics", true, /* defaultValue */ settings.WithPublic) diff --git a/pkg/sql/idxusage/index_usage_stats_rec.go b/pkg/sql/idxusage/index_usage_stats_rec.go index 6386712ef09e..05685133adeb 100644 --- a/pkg/sql/idxusage/index_usage_stats_rec.go +++ b/pkg/sql/idxusage/index_usage_stats_rec.go @@ -43,7 +43,7 @@ const defaultUnusedIndexDuration = 7 * 24 * time.Hour // DropUnusedIndexDuration registers the index unuse duration at which we // begin to recommend dropping the index. var DropUnusedIndexDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.index_recommendation.drop_unused_duration", "the index unuse duration at which we begin to recommend dropping the index", defaultUnusedIndexDuration, diff --git a/pkg/sql/importer/export_base.go b/pkg/sql/importer/export_base.go index fc60d91bebd8..6ddd6969265a 100644 --- a/pkg/sql/importer/export_base.go +++ b/pkg/sql/importer/export_base.go @@ -20,7 +20,7 @@ import ( // export file is written out. Since it's difficult to calculate the number of // bytes that will be created, we use this multiplier for estimation. var eventMemoryMultipier = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "export.event_memory_multiplier", "the amount of memory required to export a datum is multiplied by this factor", 3, diff --git a/pkg/sql/importer/import_job.go b/pkg/sql/importer/import_job.go index 17fce4929669..32fe2b9f6e01 100644 --- a/pkg/sql/importer/import_job.go +++ b/pkg/sql/importer/import_job.go @@ -99,7 +99,7 @@ func (r *importResumer) DumpTraceAfterRun() bool { var _ jobs.Resumer = &importResumer{} var processorsPerNode = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.processors_per_node", "number of input processors to run on each sql instance", 1, settings.PositiveInt, diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index ccc6d9dbfc97..fe3376f0813f 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -205,7 +205,7 @@ var allowedIntoFormats = map[string]struct{}{ // featureImportEnabled is used to enable and disable the IMPORT feature. var featureImportEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.import.enabled", "set to true to enable imports, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/sql/importer/import_processor.go b/pkg/sql/importer/import_processor.go index d0c162cc927a..ef79797eac16 100644 --- a/pkg/sql/importer/import_processor.go +++ b/pkg/sql/importer/import_processor.go @@ -57,7 +57,7 @@ var progressUpdateInterval = time.Second * 10 var importPKAdderBufferSize = func() *settings.ByteSizeSetting { s := settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.pk_buffer_size", "the initial size of the BulkAdder buffer handling primary index imports", 32<<20, @@ -67,7 +67,7 @@ var importPKAdderBufferSize = func() *settings.ByteSizeSetting { var importPKAdderMaxBufferSize = func() *settings.ByteSizeSetting { s := settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.max_pk_buffer_size", "the maximum size of the BulkAdder buffer handling primary index imports", 128<<20, @@ -77,7 +77,7 @@ var importPKAdderMaxBufferSize = func() *settings.ByteSizeSetting { var importIndexAdderBufferSize = func() *settings.ByteSizeSetting { s := settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.index_buffer_size", "the initial size of the BulkAdder buffer handling secondary index imports", 32<<20, @@ -87,7 +87,7 @@ var importIndexAdderBufferSize = func() *settings.ByteSizeSetting { var importIndexAdderMaxBufferSize = func() *settings.ByteSizeSetting { s := settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "kv.bulk_ingest.max_index_buffer_size", "the maximum size of the BulkAdder buffer handling secondary index imports", 512<<20, @@ -96,7 +96,7 @@ var importIndexAdderMaxBufferSize = func() *settings.ByteSizeSetting { }() var readerParallelismSetting = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.reader_parallelism", "number of parallel workers to use to convert read data for formats that support parallel conversion; 0 indicates number of cores", 0, diff --git a/pkg/sql/importer/import_processor_planning.go b/pkg/sql/importer/import_processor_planning.go index 86ea46d3a5af..b978a01450cf 100644 --- a/pkg/sql/importer/import_processor_planning.go +++ b/pkg/sql/importer/import_processor_planning.go @@ -39,14 +39,14 @@ import ( ) var replanThreshold = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.replan_flow_threshold", "fraction of initial flow instances that would be added or updated above which an IMPORT is restarted from its last checkpoint (0=disabled)", 0.0, ) var replanFrequency = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.replan_flow_frequency", "frequency at which IMPORT checks to see if restarting would update its physical execution plan", time.Minute*2, diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 04eabfbfca03..5c1efa7e2f4d 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -53,7 +53,7 @@ import ( ) var collectTxnStatsSampleRate = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.txn_stats.sample_rate", "the probability that a given transaction will collect execution statistics (displayed in the DB Console)", 0.01, @@ -322,7 +322,7 @@ func (c *inFlightTraceCollector) finish() { } var inFlightTraceCollectorPollInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stmt_diagnostics.in_flight_trace_collector.poll_interval", "determines the interval between polling done by the in-flight trace "+ "collector for the statement bundle, set to zero to disable", @@ -331,7 +331,7 @@ var inFlightTraceCollectorPollInterval = settings.RegisterDurationSetting( ) var timeoutTraceCollectionEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stmt_diagnostics.timeout_trace_collection.enabled", "determines whether the in-flight trace collection is performed when building "+ "the statement bundle if the timeout is detected", diff --git a/pkg/sql/join_token.go b/pkg/sql/join_token.go index ec55a8c7d695..ae9c6fde33b4 100644 --- a/pkg/sql/join_token.go +++ b/pkg/sql/join_token.go @@ -26,7 +26,7 @@ import ( // FeatureTLSAutoJoinEnabled is used to enable and disable the TLS auto-join // feature. var FeatureTLSAutoJoinEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.tls_auto_join.enabled", "set to true to enable tls auto join through join tokens, false to disable; default is false", false, diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index e2b14cccf5e6..cffc19ec2dbc 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1770,7 +1770,7 @@ func (t *logicTest) newCluster( } for settingName, value := range toa.clusterSettings { - t.waitForTenantReadOnlyClusterSettingToTakeEffectOrFatal( + t.waitForSystemVisibleClusterSettingToTakeEffectOrFatal( settingName, value, params.ServerArgs.Insecure, ) } @@ -1778,10 +1778,10 @@ func (t *logicTest) newCluster( t.setSessionUser(username.RootUser, 0 /* nodeIdx */, false /* newSession */) } -// waitForTenantReadOnlyClusterSettingToTakeEffectOrFatal waits until all tenant +// waitForSystemVisibleClusterSettingToTakeEffectOrFatal waits until all tenant // servers are aware about the supplied setting's expected value. Fatal's if // this doesn't happen within the SucceedsSoonDuration. -func (t *logicTest) waitForTenantReadOnlyClusterSettingToTakeEffectOrFatal( +func (t *logicTest) waitForSystemVisibleClusterSettingToTakeEffectOrFatal( settingName string, expValue string, insecure bool, ) { // Wait until all tenant servers are aware of the setting override. diff --git a/pkg/sql/notice.go b/pkg/sql/notice.go index 7454197fa534..00cbe644227e 100644 --- a/pkg/sql/notice.go +++ b/pkg/sql/notice.go @@ -22,7 +22,7 @@ import ( // NoticesEnabled is the cluster setting that allows users // to enable notices. var NoticesEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.notices.enabled", "enable notices in the server/client protocol being sent", true, diff --git a/pkg/sql/opt/optbuilder/mutation_builder_unique.go b/pkg/sql/opt/optbuilder/mutation_builder_unique.go index a58d4811d684..29dd938c1827 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder_unique.go +++ b/pkg/sql/opt/optbuilder/mutation_builder_unique.go @@ -26,7 +26,7 @@ import ( // UniquenessChecksForGenRandomUUIDClusterMode controls the cluster setting for // enabling uniqueness checks for UUID columns set to gen_random_uuid(). var UniquenessChecksForGenRandomUUIDClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled", "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", diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index 4e3e912aa2a9..ca1794ee8bda 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -28,7 +28,7 @@ import ( // TODO(michae2): Remove this when #70731 is fixed. var multipleModificationsOfTableEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.multiple_modifications_of_table.enabled", "if true, allow statements containing multiple INSERT ON CONFLICT, UPSERT, UPDATE, or DELETE "+ "subqueries modifying the same table, at the risk of data corruption if the same row is "+ diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index 491235bb014b..fc97d0e44343 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -491,7 +491,7 @@ func authCertPassword( // // It is exported for use in tests. var AutoSelectPasswordAuth = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.cert_password_method.auto_scram_promotion.enabled", "whether to automatically promote cert-password authentication to use SCRAM", true, diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go index f8a0f6c20afc..a427b9fb839b 100644 --- a/pkg/sql/pgwire/hba_conf.go +++ b/pkg/sql/pgwire/hba_conf.go @@ -81,7 +81,7 @@ const serverHBAConfSetting = "server.host_based_authentication.configuration" // connAuthConf is the cluster setting that holds the HBA // configuration. var connAuthConf = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, serverHBAConfSetting, "host-based authentication configuration to use during connection authentication", "", diff --git a/pkg/sql/pgwire/ident_map_conf.go b/pkg/sql/pgwire/ident_map_conf.go index 6a7e5adbb815..73fa3512a9b5 100644 --- a/pkg/sql/pgwire/ident_map_conf.go +++ b/pkg/sql/pgwire/ident_map_conf.go @@ -26,7 +26,7 @@ const serverIdentityMapSetting = "server.identity_map.configuration" // ConnIdentityMapConf maps system-identities to database-usernames using the pg_ident.conf format. var ConnIdentityMapConf = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, serverIdentityMapSetting, "system-identity to database-username mappings", "", diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index 2166e9411f82..7ed338cdbcf6 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -58,7 +58,7 @@ const readBufferMaxMessageSizeClusterSettingName = "sql.conn.max_read_buffer_mes // ReadBufferMaxMessageSizeClusterSetting is the cluster setting for configuring // ReadBuffer default message sizes. var ReadBufferMaxMessageSizeClusterSetting = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, readBufferMaxMessageSizeClusterSettingName, "maximum buffer size to allow for ingesting sql statements. Connections must be restarted for this to take effect.", defaultMaxReadBufferMessageSize, diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index 1ef07dbe5520..0e15c9103f89 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -62,7 +62,7 @@ import ( // The "results_buffer_size" connection parameter can be used to override this // default for an individual connection. var connResultsBufferSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.results_buffer.size", "default size of the buffer that accumulates results for a statement or a batch "+ "of statements before they are sent to the client. This can be overridden on "+ @@ -77,14 +77,14 @@ var connResultsBufferSize = settings.RegisterByteSizeSetting( settings.WithPublic) var logConnAuth = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, sql.ConnAuditingClusterSettingName, "if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes)", false, settings.WithPublic) var logSessionAuth = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, sql.AuthAuditingClusterSettingName, "if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)", false, diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 01b89c1bc457..efbc7a01b2d5 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -45,7 +45,7 @@ import ( ) var queryCacheEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.query_cache.enabled", "enable the query cache", true, ) diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 382f1588de9a..ee6c72d3223e 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -866,7 +866,7 @@ var metamorphicDefaultUseIndexLookupForDescriptorsInDatabase = util.ConstantWith // namespace table should be used to fetch the set of descriptors needed to // materialize most system tables. var useIndexLookupForDescriptorsInDatabase = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.catalog.virtual_tables.use_index_lookup_for_descriptors_in_database.enabled", "if enabled, virtual tables will do a lookup against the namespace table to"+ " find the descriptors in a database instead of scanning all descriptors", diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index b270fa02c7a7..eb14c474ab9a 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -34,19 +34,19 @@ import ( // TODO (msbutler): tune these var rollbackBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.predicate_delete_range_batch_size", "the number of ranges to include in a single Predicate Based DeleteRange request", 10) var predicateDeleteRangeNumWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "bulkio.import.predicate_delete_range_parallelism", "the number of workers used to issue Predicate Based DeleteRange request", 4) var maxRevertSpanNumWorkers = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.revert.max_span_parallelism", "the maximum number of workers used to issue RevertRange request", 8, diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index b806b2f05932..811c95685877 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -44,7 +44,7 @@ const ( ) var maxRowSizeLog = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.guardrails.max_row_size_log", "maximum size of row (or column family if multiple column families are in use) that SQL can "+ "write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF "+ @@ -55,7 +55,7 @@ var maxRowSizeLog = settings.RegisterByteSizeSetting( ) var maxRowSizeErr = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.guardrails.max_row_size_err", "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", diff --git a/pkg/sql/rowexec/columnbackfiller.go b/pkg/sql/rowexec/columnbackfiller.go index 14a98805bd15..7f51d4389cbf 100644 --- a/pkg/sql/rowexec/columnbackfiller.go +++ b/pkg/sql/rowexec/columnbackfiller.go @@ -48,7 +48,7 @@ var _ chunkBackfiller = &columnBackfiller{} // Each function retains a reference to its corresponding TxnCoordSender, so we // need to be careful not to accumulate an unbounded number of these functions. var backfillerMaxCommitWaitFns = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "schemachanger.backfiller.max_commit_wait_fns", "the maximum number of commit-wait functions that the columnBackfiller will accumulate before consuming them to reclaim memory", 128, diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index db5df157568b..595a00f7b3c4 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -58,12 +58,12 @@ type indexBackfiller struct { var _ execinfra.Processor = &indexBackfiller{} var backfillerBufferSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "schemachanger.backfiller.buffer_size", "the initial size of the BulkAdder buffer handling index backfills", 32<<20, ) var backfillerMaxBufferSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "schemachanger.backfiller.max_buffer_size", "the maximum size of the BulkAdder buffer handling index backfills", 512<<20, ) diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index b71b18cb20b5..cfda23f22dc7 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -266,7 +266,7 @@ const joinReaderProcName = "join reader" // ParallelizeMultiKeyLookupJoinsEnabled determines whether the joinReader // parallelizes KV batches in all cases. var ParallelizeMultiKeyLookupJoinsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.parallelize_multi_key_lookup_joins.enabled", "determines whether KV batches are executed in parallel for lookup joins in all cases. "+ "Enabling this will increase the speed of lookup joins when each input row might get "+ diff --git a/pkg/sql/rowexec/joinreader_strategies.go b/pkg/sql/rowexec/joinreader_strategies.go index 5d4a51495dd6..ef14b2318ddc 100644 --- a/pkg/sql/rowexec/joinreader_strategies.go +++ b/pkg/sql/rowexec/joinreader_strategies.go @@ -163,7 +163,7 @@ const joinReaderNoOrderingStrategyBatchSizeDefault = 2 << 20 /* 2 MiB */ // JoinReaderNoOrderingStrategyBatchSize determines the size of input batches // used to construct a single lookup KV batch by joinReaderNoOrderingStrategy. var JoinReaderNoOrderingStrategyBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.join_reader_no_ordering_strategy.batch_size", "size limit on the input rows to construct a single lookup KV batch", joinReaderNoOrderingStrategyBatchSizeDefault, @@ -578,7 +578,7 @@ const joinReaderOrderingStrategyBatchSizeDefault = 100 << 10 /* 100 KiB */ // JoinReaderOrderingStrategyBatchSize determines the size of input batches used // to construct a single lookup KV batch by joinReaderOrderingStrategy. var JoinReaderOrderingStrategyBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.distsql.join_reader_ordering_strategy.batch_size", "size limit on the input rows to construct a single lookup KV batch", joinReaderOrderingStrategyBatchSizeDefault, diff --git a/pkg/sql/scheduledlogging/captured_index_usage_stats.go b/pkg/sql/scheduledlogging/captured_index_usage_stats.go index d03c7563aca4..799845aee867 100644 --- a/pkg/sql/scheduledlogging/captured_index_usage_stats.go +++ b/pkg/sql/scheduledlogging/captured_index_usage_stats.go @@ -31,14 +31,14 @@ import ( ) var telemetryCaptureIndexUsageStatsEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.capture_index_usage_stats.enabled", "enable/disable capturing index usage statistics to the telemetry logging channel", true, ) var telemetryCaptureIndexUsageStatsInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.telemetry.capture_index_usage_stats.interval", "the scheduled interval time between capturing index usage statistics when capturing index usage statistics is enabled", 8*time.Hour, @@ -46,7 +46,7 @@ var telemetryCaptureIndexUsageStatsInterval = settings.RegisterDurationSetting( ) var telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.telemetry.capture_index_usage_stats.check_enabled_interval", "the scheduled interval time between checks to see if index usage statistics has been enabled", 10*time.Minute, @@ -54,7 +54,7 @@ var telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval = settings.Registe ) var telemetryCaptureIndexUsageStatsLoggingDelay = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.telemetry.capture_index_usage_stats.logging_delay", "the time delay between emitting individual index usage stats logs, this is done to "+ "mitigate the log-line limit of 10 logs per second on the telemetry pipeline", diff --git a/pkg/sql/schema_change_cluster_setting.go b/pkg/sql/schema_change_cluster_setting.go index d9022474e971..49e9bf2cb572 100644 --- a/pkg/sql/schema_change_cluster_setting.go +++ b/pkg/sql/schema_change_cluster_setting.go @@ -23,7 +23,7 @@ import ( // any features that require schema changes. Documentation for which features // are covered TBD. var featureSchemaChangeEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "feature.schema_change.enabled", "set to true to enable schema changes, false to disable; default is true", featureflag.FeatureFlagEnabledDefault, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 8608f4cfd89c..d2843c7c760d 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -69,7 +69,7 @@ import ( ) var schemaChangeJobMaxRetryBackoff = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "schemachanger.job.max_retry_backoff", "the exponential back off when retrying jobs for schema changes", 20*time.Second, diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go index d06dd2285fc7..1bc7877a06ec 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/statement_control.go @@ -31,7 +31,7 @@ type statementsForceControl map[string]bool // // Note: We can only control statements implemented in declarative schema changer. var forceDeclarativeStatements = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.schema.force_declarative_statements", "forcefully enable or disable declarative schema changer for specific statements", "", diff --git a/pkg/sql/schemachanger/scrun/scrun.go b/pkg/sql/schemachanger/scrun/scrun.go index 4dbb6aae8d7a..d87d35e3c8c1 100644 --- a/pkg/sql/schemachanger/scrun/scrun.go +++ b/pkg/sql/schemachanger/scrun/scrun.go @@ -36,7 +36,7 @@ import ( ) var enforcePlannerSanityCheck = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.schemachanger.strict_planning_sanity_check.enabled", "enforce strict sanity checks in the declarative schema changer planner", buildutil.CrdbTestBuild, diff --git a/pkg/sql/sem/eval/settings.go b/pkg/sql/sem/eval/settings.go index ae1a90610333..7e467dee56a8 100644 --- a/pkg/sql/sem/eval/settings.go +++ b/pkg/sql/sem/eval/settings.go @@ -21,7 +21,7 @@ import ( const experimentalBox2DClusterSettingName = "sql.spatial.experimental_box2d_comparison_operators.enabled" var experimentalBox2DClusterSetting = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, experimentalBox2DClusterSettingName, "enables the use of certain experimental box2d comparison operators", false, diff --git a/pkg/sql/serial.go b/pkg/sql/serial.go index 0d0d25c49d92..3f780f1154cc 100644 --- a/pkg/sql/serial.go +++ b/pkg/sql/serial.go @@ -50,7 +50,7 @@ var virtualSequenceOpts = tree.SequenceOptions{ // cachedSequencesCacheSize is the default cache size used when // SessionNormalizationMode is SerialUsesCachedSQLSequences. var cachedSequencesCacheSizeSetting = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.defaults.serial_sequences_cache_size", "the default cache size when the session's serial normalization mode is set to cached sequences"+ "A cache size of 1 means no caching. Any cache size less than 1 is invalid.", diff --git a/pkg/sql/session_revival_token.go b/pkg/sql/session_revival_token.go index ca640d08aa20..ebe131af1721 100644 --- a/pkg/sql/session_revival_token.go +++ b/pkg/sql/session_revival_token.go @@ -25,7 +25,7 @@ import ( // setting since this is only intended to be used by CockroachDB-serverless // at the time of this writing. var AllowSessionRevival = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "server.user_login.session_revival_token.enabled", "if set, the cluster is able to create session revival tokens and use them "+ "to authenticate a new session", diff --git a/pkg/sql/session_state.go b/pkg/sql/session_state.go index e9cba1250190..6092bc50bab2 100644 --- a/pkg/sql/session_state.go +++ b/pkg/sql/session_state.go @@ -31,7 +31,7 @@ import ( ) var maxSerializedSessionSize = settings.RegisterByteSizeSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.session_transfer.max_session_size", "if set to non-zero, then serializing a session will fail if it requires more"+ "than the specified size", diff --git a/pkg/sql/sessioninit/cache.go b/pkg/sql/sessioninit/cache.go index 792bad9eae75..1771dd660b7a 100644 --- a/pkg/sql/sessioninit/cache.go +++ b/pkg/sql/sessioninit/cache.go @@ -36,7 +36,7 @@ const CacheEnabledSettingName = "server.authentication_cache.enabled" // CacheEnabled is a cluster setting that determines if the // sessioninit.Cache and associated logic is enabled. var CacheEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, CacheEnabledSettingName, "enables a cache used during authentication to avoid lookups to system tables "+ "when retrieving per-user authentication-related information", diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index 0568fb00fd19..62973e29a7f3 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -149,7 +149,7 @@ func (p *planner) SetClusterSetting( // The Lookup call above should never return SystemOnly settings if this // is a tenant. return nil, errors.AssertionFailedf("looked up system-only setting") - case settings.TenantReadOnly: + case settings.SystemVisible: return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "setting %s is only settable by the operator", name) } } diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index e59f43fa9286..a094b12cac9b 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -1078,7 +1078,7 @@ func validateZoneAttrsAndLocalitiesForSystemTenant( // SecondaryTenantZoneConfigsEnabled that allows virtual clusters to modify all // type of constraints in zone configs (i.e. not only zones and regions). var secondaryTenantsAllZoneConfigsEnabled = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled", "enable unrestricted usage of ALTER CONFIGURE ZONE in virtual clusters", false, diff --git a/pkg/sql/sql_activity_update_job.go b/pkg/sql/sql_activity_update_job.go index 9b5b4a6b8274..65d1ec6918dd 100644 --- a/pkg/sql/sql_activity_update_job.go +++ b/pkg/sql/sql_activity_update_job.go @@ -33,7 +33,7 @@ import ( // sqlStatsActivityFlushEnabled the stats activity flush job. var sqlStatsActivityFlushEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.activity.flush.enabled", "enable the flush to the system statement and transaction activity tables", true) @@ -41,7 +41,7 @@ var sqlStatsActivityFlushEnabled = settings.RegisterBoolSetting( // sqlStatsActivityTopCount is the cluster setting that controls the number of // rows selected to be inserted into the activity tables var sqlStatsActivityTopCount = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.activity.top.max", "the limit per column for the top number of statistics to be flushed "+ "to the activity tables", @@ -57,7 +57,7 @@ var sqlStatsActivityTopCount = settings.RegisterIntSetting( // are likely the same for several columns, so it should still give 3 days // of history for the default settings var sqlStatsActivityMaxPersistedRows = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.activity.persisted_rows.max", "maximum number of rows of statement and transaction"+ " activity that will be persisted in the system tables", diff --git a/pkg/sql/sqlinstance/instancestorage/instancestorage.go b/pkg/sql/sqlinstance/instancestorage/instancestorage.go index ed8ae8b7d17a..a01d78c6c262 100644 --- a/pkg/sql/sqlinstance/instancestorage/instancestorage.go +++ b/pkg/sql/sqlinstance/instancestorage/instancestorage.go @@ -48,7 +48,7 @@ import ( // ReclaimLoopInterval is the interval at which expired instance IDs are // reclaimed and new ones will be preallocated. var ReclaimLoopInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.instance_id.reclaim_interval", "interval at which instance IDs are reclaimed and preallocated", 10*time.Minute, @@ -58,7 +58,7 @@ var ReclaimLoopInterval = settings.RegisterDurationSetting( // PreallocatedCount refers to the number of preallocated instance IDs within // the system.sql_instances table. var PreallocatedCount = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.instance_id.preallocated_count", "number of preallocated instance IDs within the system.sql_instances table", 10, diff --git a/pkg/sql/sqlliveness/slinstance/slinstance.go b/pkg/sql/sqlliveness/slinstance/slinstance.go index f6b327063368..1d937357f943 100644 --- a/pkg/sql/sqlliveness/slinstance/slinstance.go +++ b/pkg/sql/sqlliveness/slinstance/slinstance.go @@ -39,7 +39,7 @@ import ( var ( // DefaultTTL specifies the time to expiration when a session is created. DefaultTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.sqlliveness.ttl", "default sqlliveness session ttl", 40*time.Second, @@ -47,7 +47,7 @@ var ( ) // DefaultHeartBeat specifies the period between attempts to extend a session. DefaultHeartBeat = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.sqlliveness.heartbeat", "duration heart beats to push session expiration further out in time", 5*time.Second, diff --git a/pkg/sql/sqlliveness/slstorage/slstorage.go b/pkg/sql/sqlliveness/slstorage/slstorage.go index 7cc85ac0ce60..7dc989edfa69 100644 --- a/pkg/sql/sqlliveness/slstorage/slstorage.go +++ b/pkg/sql/sqlliveness/slstorage/slstorage.go @@ -44,7 +44,7 @@ import ( // GCInterval specifies duration between attempts to delete extant // sessions that have expired. var GCInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.sqlliveness.gc_interval", "duration between attempts to delete extant sessions that have expired", time.Hour, @@ -56,7 +56,7 @@ var GCInterval = settings.RegisterDurationSetting( // // [(1-GCJitter) * GCInterval, (1+GCJitter) * GCInterval] var GCJitter = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.sqlliveness.gc_jitter", "jitter fraction on the duration between attempts to delete extant sessions that have expired", .15, @@ -70,7 +70,7 @@ var GCJitter = settings.RegisterFloatSetting( // increasing the cache size dynamically. The entries are just bytes each so // this should not be a big deal. var CacheSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.sqlliveness.storage_session_cache_size", "number of session entries to store in the LRU", 1024) diff --git a/pkg/sql/sqlstats/cluster_settings.go b/pkg/sql/sqlstats/cluster_settings.go index 47c8b36e96ec..ebac65c89670 100644 --- a/pkg/sql/sqlstats/cluster_settings.go +++ b/pkg/sql/sqlstats/cluster_settings.go @@ -18,7 +18,7 @@ import ( // StmtStatsEnable determines whether to collect per-statement statistics. var StmtStatsEnable = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.enabled", "collect per-statement query statistics", true, settings.WithPublic) @@ -26,7 +26,7 @@ var StmtStatsEnable = settings.RegisterBoolSetting( // transactions statistics for a single transaction. This defaults to 1000, and // currently is non-configurable (hidden setting). var TxnStatsNumStmtFingerprintIDsToRecord = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.transaction_details.max_statement_ids", "max number of statement fingerprint IDs to store for transaction statistics", 1000, @@ -36,14 +36,14 @@ var TxnStatsNumStmtFingerprintIDsToRecord = settings.RegisterIntSetting( // TxnStatsEnable determines whether to collect per-application transaction // statistics. var TxnStatsEnable = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.transaction_details.enabled", "collect per-application transaction statistics", true, settings.WithPublic) // StatsCollectionLatencyThreshold specifies the minimum amount of time // consumed by a SQL statement before it is collected for statistics reporting. var StatsCollectionLatencyThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.threshold", "minimum execution time to cause statement statistics to be collected. "+ "If configured, no transaction stats are collected.", @@ -53,7 +53,7 @@ var StatsCollectionLatencyThreshold = settings.RegisterDurationSetting( // DumpStmtStatsToLogBeforeReset specifies whether we dump the statements // statistics to logs before being reset. var DumpStmtStatsToLogBeforeReset = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.dump_to_logs", "dump collected statement statistics to node logs when periodically cleared", false, @@ -63,7 +63,7 @@ var DumpStmtStatsToLogBeforeReset = settings.RegisterBoolSetting( // SampleLogicalPlans specifies whether we periodically sample the logical plan // for each fingerprint. var SampleLogicalPlans = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.plan_collection.enabled", "periodically save a logical plan for each fingerprint", false, @@ -72,7 +72,7 @@ var SampleLogicalPlans = settings.RegisterBoolSetting( // LogicalPlanCollectionPeriod specifies the interval between collections of // logical plans for each fingerprint. var LogicalPlanCollectionPeriod = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.plan_collection.period", "the time until a new logical plan is collected", 5*time.Minute, @@ -82,7 +82,7 @@ var LogicalPlanCollectionPeriod = settings.RegisterDurationSetting( // MaxMemSQLStatsStmtFingerprints specifies the maximum of unique statement // fingerprints we store in memory. var MaxMemSQLStatsStmtFingerprints = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.max_mem_stmt_fingerprints", "the maximum number of statement fingerprints stored in memory", 100000, @@ -91,7 +91,7 @@ var MaxMemSQLStatsStmtFingerprints = settings.RegisterIntSetting( // MaxMemSQLStatsTxnFingerprints specifies the maximum of unique transaction // fingerprints we store in memory. var MaxMemSQLStatsTxnFingerprints = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.max_mem_txn_fingerprints", "the maximum number of transaction fingerprints stored in memory", 100000, @@ -100,7 +100,7 @@ var MaxMemSQLStatsTxnFingerprints = settings.RegisterIntSetting( // MaxMemReportedSQLStatsStmtFingerprints specifies the maximum of unique statement // fingerprints we store in memory. var MaxMemReportedSQLStatsStmtFingerprints = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.max_mem_reported_stmt_fingerprints", "the maximum number of reported statement fingerprints stored in memory", 100000, @@ -109,7 +109,7 @@ var MaxMemReportedSQLStatsStmtFingerprints = settings.RegisterIntSetting( // MaxMemReportedSQLStatsTxnFingerprints specifies the maximum of unique transaction // fingerprints we store in memory. var MaxMemReportedSQLStatsTxnFingerprints = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.max_mem_reported_txn_fingerprints", "the maximum number of reported transaction fingerprints stored in memory", 100000, @@ -144,7 +144,7 @@ var MaxMemReportedSQLStatsTxnFingerprints = settings.RegisterIntSetting( // The total amount of memory consumed will still be constrained by the // top-level memory monitor created for SQL Stats. var MaxSQLStatsStmtFingerprintsPerExplicitTxn = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.max_stmt_fingerprints_per_explicit_txn", "the maximum number of statement fingerprints stored per explicit transaction", 2000, @@ -153,7 +153,7 @@ var MaxSQLStatsStmtFingerprintsPerExplicitTxn = settings.RegisterIntSetting( // MaxSQLStatReset is the cluster setting that controls at what interval SQL // statement statistics must be flushed within. var MaxSQLStatReset = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "diagnostics.forced_sql_stat_reset.interval", "interval after which the reported SQL Stats are reset even "+ "if not collected by telemetry reporter. It has a max value of 24H.", @@ -164,7 +164,7 @@ var MaxSQLStatReset = settings.RegisterDurationSetting( // SampleIndexRecommendation specifies whether we generate an index recommendation // for each fingerprint ID. var SampleIndexRecommendation = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.index_recommendation_collection.enabled", "generate an index recommendation for each fingerprint ID", true, @@ -173,7 +173,7 @@ var SampleIndexRecommendation = settings.RegisterBoolSetting( // MaxMemReportedSampleIndexRecommendations specifies the maximum of unique index // recommendations info we store in memory. var MaxMemReportedSampleIndexRecommendations = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.max_mem_reported_idx_recommendations", "the maximum number of reported index recommendation info stored in memory", 5000, @@ -182,7 +182,7 @@ var MaxMemReportedSampleIndexRecommendations = settings.RegisterIntSetting( // GatewayNodeEnabled specifies whether we save the gateway node id for each fingerprint // during sql stats collection, otherwise the value will be set to 0. var GatewayNodeEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.statement_details.gateway_node.enabled", "save the gateway node for each statement fingerprint. If false, the value will "+ "be stored as 0.", diff --git a/pkg/sql/sqlstats/insights/insights.go b/pkg/sql/sqlstats/insights/insights.go index fb61e284308f..a3100e2709e5 100644 --- a/pkg/sql/sqlstats/insights/insights.go +++ b/pkg/sql/sqlstats/insights/insights.go @@ -26,7 +26,7 @@ import ( // ExecutionInsightsCapacity limits the number of execution insights retained in memory. // As further insights are had, the oldest ones are evicted. var ExecutionInsightsCapacity = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.execution_insights_capacity", "the size of the per-node store of execution insights", 1000, @@ -37,7 +37,7 @@ var ExecutionInsightsCapacity = settings.RegisterIntSetting( // considered slow. A LatencyThreshold of 0 (the default) disables this // detection. var LatencyThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.latency_threshold", "amount of time after which an executing statement is considered slow. Use 0 to disable.", 100*time.Millisecond, @@ -48,7 +48,7 @@ var LatencyThreshold = settings.RegisterDurationSetting( // p99 latency while generally excluding uninteresting executions less than // 100ms. var AnomalyDetectionEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.anomaly_detection.enabled", "enable per-fingerprint latency recording and anomaly detection", true, @@ -61,7 +61,7 @@ var AnomalyDetectionEnabled = settings.RegisterBoolSetting( // and any potential slow execution must also cross this threshold to be // reported (this is a UX optimization, removing noise). var AnomalyDetectionLatencyThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.anomaly_detection.latency_threshold", "statements must surpass this threshold to trigger anomaly detection and identification", 50*time.Millisecond, @@ -73,7 +73,7 @@ var AnomalyDetectionLatencyThreshold = settings.RegisterDurationSetting( // an eye on the metrics for memory usage and evictions to avoid introducing // churn. var AnomalyDetectionMemoryLimit = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.anomaly_detection.memory_limit", "the maximum amount of memory allowed for tracking statement latencies", 1024*1024, @@ -82,7 +82,7 @@ var AnomalyDetectionMemoryLimit = settings.RegisterByteSizeSetting( // HighRetryCountThreshold sets the number of times a slow statement must have // been retried to be marked as having a high retry count. var HighRetryCountThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.insights.high_retry_count.threshold", "the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem", 10, diff --git a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go index 5421c684a1e0..377bda66051f 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go +++ b/pkg/sql/sqlstats/persistedsqlstats/cluster_settings.go @@ -21,7 +21,7 @@ import ( // SQLStatsFlushInterval is the cluster setting that controls how often the SQL // stats are flushed to system table. var SQLStatsFlushInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.flush.interval", "the interval at which SQL execution statistics are flushed to disk, "+ "this value must be less than or equal to 1 hour", @@ -35,7 +35,7 @@ var SQLStatsFlushInterval = settings.RegisterDurationSetting( // generated in a short span of time, which in turn cause memory pressure), the // flush operation will be aborted. var MinimumInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.flush.minimum_interval", "the minimum interval that SQL stats can be flushes to disk. If a "+ "flush operation starts within less than the minimum interval, the flush "+ @@ -48,7 +48,7 @@ var MinimumInterval = settings.RegisterDurationSetting( // older in-memory SQL stats to be discarded when flushing to persisted tables // is disabled. var DiscardInMemoryStatsWhenFlushDisabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.flush.force_cleanup.enabled", "if set, older SQL stats are discarded periodically when flushing to "+ "persisted tables is disabled", @@ -58,7 +58,7 @@ var DiscardInMemoryStatsWhenFlushDisabled = settings.RegisterBoolSetting( // SQLStatsFlushEnabled is the cluster setting that controls if the sqlstats // subsystem persists the statistics into system table. var SQLStatsFlushEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.flush.enabled", "if set, SQL execution statistics are periodically flushed to disk", true, /* defaultValue */ @@ -71,7 +71,7 @@ var SQLStatsFlushEnabled = settings.RegisterBoolSetting( // // (1 + SQLStatsFlushJitter) * SQLStatsFlushInterval)] var SQLStatsFlushJitter = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.flush.jitter", "jitter fraction on the duration between sql stats flushes", 0.15, @@ -81,7 +81,7 @@ var SQLStatsFlushJitter = settings.RegisterFloatSetting( // SQLStatsMaxPersistedRows specifies maximum number of rows that will be // retained in system.statement_statistics and system.transaction_statistics. var SQLStatsMaxPersistedRows = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.persisted_rows.max", "maximum number of rows of statement and transaction statistics that "+ "will be persisted in the system tables before compaction begins", @@ -91,7 +91,7 @@ var SQLStatsMaxPersistedRows = settings.RegisterIntSetting( // SQLStatsCleanupRecurrence is the cron-tab string specifying the recurrence // for SQL Stats cleanup job. var SQLStatsCleanupRecurrence = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.cleanup.recurrence", "cron-tab recurrence for SQL Stats cleanup job", "@hourly", /* defaultValue */ @@ -107,7 +107,7 @@ var SQLStatsCleanupRecurrence = settings.RegisterStringSetting( // SQLStatsAggregationInterval is the cluster setting that controls the aggregation // interval for stats when we flush to disk. var SQLStatsAggregationInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.aggregation.interval", "the interval at which we aggregate SQL execution statistics upon flush, "+ "this value must be greater than or equal to sql.stats.flush.interval", @@ -119,7 +119,7 @@ var SQLStatsAggregationInterval = settings.RegisterDurationSetting( // how many rows in the statement/transaction_statistics tables gets deleted // per transaction in the Automatic SQL Stats Compaction Job. var CompactionJobRowsToDeletePerTxn = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.cleanup.rows_to_delete_per_txn", "number of rows the compaction job deletes from system table per iteration", 10000, @@ -130,7 +130,7 @@ var CompactionJobRowsToDeletePerTxn = settings.RegisterIntSetting( // sql stats system tables to grow past the number of rows set by // sql.stats.persisted_row.max. var sqlStatsLimitTableSizeEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.limit_table_size.enabled", "controls whether we allow statement and transaction statistics tables "+ "to grow past sql.stats.persisted_rows.max", @@ -141,7 +141,7 @@ var sqlStatsLimitTableSizeEnabled = settings.RegisterBoolSetting( // interval the check is done if the statement and transaction statistics // tables have grown past the sql.stats.persisted_rows.max. var sqlStatsLimitTableCheckInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.limit_table_size_check.interval", "controls what interval the check is done if the statement and "+ "transaction statistics tables have grown past sql.stats.persisted_rows.max", diff --git a/pkg/sql/sqlstats/sslocal/cluster_settings.go b/pkg/sql/sqlstats/sslocal/cluster_settings.go index 4af3d044e5ae..287dbcee773f 100644 --- a/pkg/sql/sqlstats/sslocal/cluster_settings.go +++ b/pkg/sql/sqlstats/sslocal/cluster_settings.go @@ -19,7 +19,7 @@ import "github.com/cockroachdb/cockroach/pkg/settings" // higher-cardinality data in the system.statement_statistics table than // the cleanup job is able to keep up with. See #78338. var AssociateStmtWithTxnFingerprint = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.associate_stmt_with_txn_fingerprint.enabled", "whether to segment per-statement query statistics by transaction fingerprint", true, diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_counter.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_counter.go index 5d0c7a4cb75f..18918058f158 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_counter.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_counter.go @@ -64,7 +64,7 @@ type SQLStatsAtomicCounters struct { // DiscardedStatsLogInterval specifies the interval between log emissions for discarded // statement and transaction statistics due to reaching the SQL statistics memory limit. var DiscardedStatsLogInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.metrics.discarded_stats_log.interval", "interval between log emissions for discarded statistics due to SQL statistics memory limit", 1*time.Minute, diff --git a/pkg/sql/stats/automatic_stats.go b/pkg/sql/stats/automatic_stats.go index 3cb0214f5332..5256777fb5ae 100644 --- a/pkg/sql/stats/automatic_stats.go +++ b/pkg/sql/stats/automatic_stats.go @@ -39,7 +39,7 @@ import ( // AutomaticStatisticsClusterMode controls the cluster setting for enabling // automatic table statistics collection. var AutomaticStatisticsClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, catpb.AutoStatsEnabledSettingName, "automatic statistics collection mode", true, @@ -49,7 +49,7 @@ var AutomaticStatisticsClusterMode = settings.RegisterBoolSetting( // statistics usage by the optimizer for planning queries involving system // tables. var UseStatisticsOnSystemTables = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, catpb.UseStatsOnSystemTables, "when true, enables use of statistics on system tables by the query optimizer", true, @@ -60,7 +60,7 @@ var UseStatisticsOnSystemTables = settings.RegisterBoolSetting( // via a true setting of sql.stats.automatic_collection.enabled for this flag to // have any effect. var AutomaticStatisticsOnSystemTables = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, catpb.AutoStatsOnSystemTables, "when true, enables automatic collection of statistics on system tables", true, @@ -69,7 +69,7 @@ var AutomaticStatisticsOnSystemTables = settings.RegisterBoolSetting( // MultiColumnStatisticsClusterMode controls the cluster setting for enabling // automatic collection of multi-column statistics. var MultiColumnStatisticsClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.multi_column_collection.enabled", "multi-column statistics collection mode", true, @@ -80,7 +80,7 @@ var MultiColumnStatisticsClusterMode = settings.RegisterBoolSetting( // statistics (in high load scenarios). This value can be tuned to trade off // the runtime vs performance impact of automatic stats. var AutomaticStatisticsMaxIdleTime = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.automatic_collection.max_fraction_idle", "maximum fraction of time that automatic statistics sampler processors are idle", 0.9, @@ -92,7 +92,7 @@ var AutomaticStatisticsMaxIdleTime = settings.RegisterFloatSetting( // statistics on that table are refreshed, in addition to the constant value // AutomaticStatisticsMinStaleRows. var AutomaticStatisticsFractionStaleRows = settings.RegisterFloatSetting( - settings.TenantWritable, + settings.ApplicationLevel, catpb.AutoStatsFractionStaleSettingName, "target fraction of stale rows per table that will trigger a statistics refresh", 0.2, @@ -104,7 +104,7 @@ var AutomaticStatisticsFractionStaleRows = settings.RegisterFloatSetting( // number of rows that should be updated before a table is refreshed, in // addition to the fraction AutomaticStatisticsFractionStaleRows. var AutomaticStatisticsMinStaleRows = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, catpb.AutoStatsMinStaleSettingName, "target minimum number of stale rows per table that will trigger a statistics refresh", 500, @@ -115,7 +115,7 @@ var AutomaticStatisticsMinStaleRows = settings.RegisterIntSetting( // statsGarbageCollectionInterval controls the interval between running an // internal query to delete stats for dropped tables. var statsGarbageCollectionInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.garbage_collection_interval", "interval between deleting stats for dropped tables, set to 0 to disable", time.Hour, @@ -127,7 +127,7 @@ var statsGarbageCollectionInterval = settings.RegisterDurationSetting( // beyond the limit will need to wait out statsGarbageCollectionInterval until // the next "sweep"). var statsGarbageCollectionLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.garbage_collection_limit", "limit on the number of dropped tables that stats are deleted for as part of a single statement", 1000, diff --git a/pkg/sql/stats/delete_stats.go b/pkg/sql/stats/delete_stats.go index 1792bf5e1541..01f581a11750 100644 --- a/pkg/sql/stats/delete_stats.go +++ b/pkg/sql/stats/delete_stats.go @@ -40,7 +40,7 @@ const ( // TableStatisticsRetentionPeriod controls the cluster setting for the // retention period of statistics that are not collected by default. var TableStatisticsRetentionPeriod = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.non_default_columns.min_retention_period", "minimum retention period for table statistics collected on non-default columns", defaultKeepTime, diff --git a/pkg/sql/stats/forecast.go b/pkg/sql/stats/forecast.go index 7d3b15e2dbd9..a6c3e598d701 100644 --- a/pkg/sql/stats/forecast.go +++ b/pkg/sql/stats/forecast.go @@ -36,7 +36,7 @@ import ( // UseStatisticsForecasts controls whether statistics forecasts are generated in // the stats cache. var UseStatisticsForecasts = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.forecasts.enabled", "when true, enables generation of statistics forecasts by default for all tables", true, diff --git a/pkg/sql/stats/histogram.go b/pkg/sql/stats/histogram.go index 180b75d5464e..e0af9a7dc0c3 100644 --- a/pkg/sql/stats/histogram.go +++ b/pkg/sql/stats/histogram.go @@ -29,7 +29,7 @@ import ( // DefaultHistogramBuckets is the maximum number of histogram buckets to build // when creating statistics. var DefaultHistogramBuckets = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.histogram_buckets.count", "maximum number of histogram buckets to build during table statistics collection", 200, @@ -39,7 +39,7 @@ var DefaultHistogramBuckets = settings.RegisterIntSetting( // HistogramClusterMode controls the cluster setting for enabling // histogram collection. var HistogramClusterMode = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stats.histogram_collection.enabled", "histogram collection mode", true, diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics.go b/pkg/sql/stmtdiagnostics/statement_diagnostics.go index 53fa814f66eb..5040df5f3eeb 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics.go @@ -33,7 +33,7 @@ import ( ) var pollingInterval = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "sql.stmt_diagnostics.poll_interval", "rate at which the stmtdiagnostics.Registry polls for requests, set to zero to disable", 10*time.Second, @@ -41,7 +41,7 @@ var pollingInterval = settings.RegisterDurationSetting( ) var bundleChunkSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stmt_diagnostics.bundle_chunk_size", "chunk size for statement diagnostic bundles", 1024*1024, @@ -70,7 +70,7 @@ var bundleChunkSize = settings.RegisterByteSizeSetting( // bounded set of bundles around per-request/fingerprint. See #82896 for more // details. var collectUntilExpiration = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.stmt_diagnostics.collect_continuously.enabled", "collect diagnostic bundles continuously until request expiration (to be "+ "used with care, only has an effect if the diagnostic request has an "+ diff --git a/pkg/sql/tablewriter.go b/pkg/sql/tablewriter.go index 7ec988f37f45..878f95afaef0 100644 --- a/pkg/sql/tablewriter.go +++ b/pkg/sql/tablewriter.go @@ -151,7 +151,7 @@ type tableWriterBase struct { } var maxBatchBytes = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.mutations.mutation_batch_byte_size", "byte size - in key and value lengths -- for mutation batches", 4<<20, diff --git a/pkg/sql/telemetry_logging.go b/pkg/sql/telemetry_logging.go index 2611f432643a..70c39d6552ed 100644 --- a/pkg/sql/telemetry_logging.go +++ b/pkg/sql/telemetry_logging.go @@ -28,7 +28,7 @@ import ( const defaultMaxEventFrequency = 8 var TelemetryMaxEventFrequency = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.query_sampling.max_event_frequency", "the max event frequency at which we sample executions for telemetry, "+ "note that it is recommended that this value shares a log-line limit of 10 "+ @@ -40,14 +40,14 @@ var TelemetryMaxEventFrequency = settings.RegisterIntSetting( ) var telemetryInternalQueriesEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.query_sampling.internal.enabled", "when set to true, internal queries will be sampled in telemetry logging", false, settings.WithPublic) var telemetryInternalConsoleQueriesEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.query_sampling.internal_console.enabled", "when set to true, all internal queries used to populated the UI Console"+ "will be logged into telemetry", @@ -60,7 +60,7 @@ const ( ) var telemetrySamplingMode = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.query_sampling.mode", "the execution level used for telemetry sampling. If set to 'statement', events "+ "are sampled at the statement execution level. If set to 'transaction', events are "+ @@ -75,7 +75,7 @@ var telemetrySamplingMode = settings.RegisterEnumSetting( ) var telemetryTrackedTxnsLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.telemetry.txn_mode.tracking_limit", "the maximum number of transactions tracked at one time for which we will send "+ "all statements to telemetry", diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index d69e156cb904..8306093ef85c 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -47,7 +47,7 @@ import ( // TempObjectCleanupInterval is a ClusterSetting controlling how often // temporary objects get cleaned up. var TempObjectCleanupInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.temp_object_cleaner.cleanup_interval", "how often to clean up orphaned temporary objects", 30*time.Minute, @@ -56,7 +56,7 @@ var TempObjectCleanupInterval = settings.RegisterDurationSetting( // TempObjectWaitInterval is a ClusterSetting controlling how long // after a creation a temporary object will be cleaned up. var TempObjectWaitInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.temp_object_cleaner.wait_interval", "how long after creation a temporary object will be cleaned up", 30*time.Minute, diff --git a/pkg/sql/tenant_accessors.go b/pkg/sql/tenant_accessors.go index 51c99274f6e8..a62069941c80 100644 --- a/pkg/sql/tenant_accessors.go +++ b/pkg/sql/tenant_accessors.go @@ -199,7 +199,7 @@ func GetExtendedTenantInfo( } var defaultTenantConfigTemplate = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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 diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 08075349996d..e08c1e3fd5f7 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -147,7 +147,7 @@ func (t *truncateNode) Close(context.Context) {} // split points that we re-create on a table after a truncate. It's scaled by // the number of nodes in the cluster. var PreservedSplitCountMultiple = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.truncate.preserved_split_count_multiple", "set to non-zero to cause TRUNCATE to preserve range splits from the "+ "table's indexes. The multiple given will be multiplied with the number of "+ diff --git a/pkg/sql/ttl/ttlbase/ttl_helpers.go b/pkg/sql/ttl/ttlbase/ttl_helpers.go index 39df129b3306..4fac728adbac 100644 --- a/pkg/sql/ttl/ttlbase/ttl_helpers.go +++ b/pkg/sql/ttl/ttlbase/ttl_helpers.go @@ -33,7 +33,7 @@ const ( var ( defaultSelectBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.ttl.default_select_batch_size", "default amount of rows to select in a single query during a TTL job", DefaultSelectBatchSizeValue, @@ -41,7 +41,7 @@ var ( settings.WithPublic, ) defaultDeleteBatchSize = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.ttl.default_delete_batch_size", "default amount of rows to delete in a single query during a TTL job", 100, @@ -49,7 +49,7 @@ var ( settings.WithPublic, ) defaultDeleteRateLimit = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.ttl.default_delete_rate_limit", "default delete rate limit (rows per second) per node for each TTL job. Use 0 to signify no rate limit.", 0, @@ -57,7 +57,7 @@ var ( settings.WithPublic, ) jobEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.ttl.job.enabled", "whether the TTL job is enabled", true, diff --git a/pkg/sql/txn_fingerprint_id_cache.go b/pkg/sql/txn_fingerprint_id_cache.go index 7d7751a0fbb0..80ad11d59e74 100644 --- a/pkg/sql/txn_fingerprint_id_cache.go +++ b/pkg/sql/txn_fingerprint_id_cache.go @@ -25,7 +25,7 @@ import ( // capacity of the txn fingerprint ID cache. The cache will be resized // on the next insert or get operation. var TxnFingerprintIDCacheCapacity = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "sql.txn_fingerprint_id_cache.capacity", "the maximum number of txn fingerprint IDs stored", 100, diff --git a/pkg/sql/user.go b/pkg/sql/user.go index f8bc9b81b44f..c07fa83a38e6 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -442,7 +442,7 @@ WHERE } var userLoginTimeout = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "server.user_login.timeout", "timeout after which client authentication times out if some system range is unavailable (0 = no timeout)", 10*time.Second, diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index a5143db040c7..3b4769f289e2 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -1660,14 +1660,14 @@ func ClearRangeWithHeuristic( } var ingestDelayL0Threshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "rocksdb.ingest_backpressure.l0_file_count_threshold", "number of L0 files after which to backpressure SST ingestions", 20, ) var ingestDelayTime = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "rocksdb.ingest_backpressure.max_delay", "maximum amount of time to backpressure a single SST ingestion", time.Second*5, diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index e4576fe6c30f..ff0586771978 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -69,7 +69,7 @@ const ( ) var minWALSyncInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "rocksdb.min_wal_sync_interval", "minimum duration between syncs of the RocksDB WAL", 0*time.Millisecond, @@ -91,7 +91,7 @@ var minWALSyncInterval = settings.RegisterDurationSetting( // V23_1_MVCCRangeTombstonesUnconditionallyEnabled, the feature is // unconditionally enabled. var MVCCRangeTombstonesEnabledInMixedClusters = settings.RegisterBoolSetting( - settings.TenantReadOnly, + settings.SystemVisible, "storage.mvcc.range_tombstones.enabled", "controls the use of MVCC range tombstones in mixed version clusters; range tombstones are always on in finalized 23.1 clusters", false) @@ -110,7 +110,7 @@ func CanUseMVCCRangeTombstones(ctx context.Context, st *cluster.Settings) bool { // MaxConflictsPerLockConflictError sets maximum number of locks returned in // LockConflictError in operations that return multiple locks per error. var MaxConflictsPerLockConflictError = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "storage.mvcc.max_intents_per_error", "maximum number of locks returned in errors during evaluation", MaxConflictsPerLockConflictErrorDefault, diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index dde7e17216c9..d526bd746adb 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -70,7 +70,7 @@ var maxEfosWait = envutil.EnvOrDefaultDuration("COCKROACH_EFOS_MAX_WAIT", 3*time // MaxSyncDuration is the threshold above which an observed engine sync duration // triggers either a warning or a fatal error. var MaxSyncDuration = settings.RegisterDurationSetting( - settings.TenantReadOnly, + settings.SystemVisible, "storage.max_sync_duration", "maximum duration for disk operations; any operations that take longer"+ " than this setting trigger a warning log entry or process crash", @@ -80,7 +80,7 @@ var MaxSyncDuration = settings.RegisterDurationSetting( // MaxSyncDurationFatalOnExceeded governs whether disk stalls longer than // MaxSyncDuration fatal the Cockroach process. Defaults to true. var MaxSyncDurationFatalOnExceeded = settings.RegisterBoolSetting( - settings.TenantWritable, // used for temp storage in virtual cluster servers + settings.ApplicationLevel, // used for temp storage in virtual cluster servers "storage.max_sync_duration.fatal.enabled", "if true, fatal the process when a disk operation exceeds storage.max_sync_duration", true, @@ -92,7 +92,7 @@ var MaxSyncDurationFatalOnExceeded = settings.RegisterBoolSetting( // compactions, and does not eagerly change the encoding of existing sstables. // Reads can correctly read both kinds of sstables. var ValueBlocksEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, // used for temp storage in virtual cluster servers + settings.ApplicationLevel, // used for temp storage in virtual cluster servers "storage.value_blocks.enabled", "set to true to enable writing of value blocks in sstables", util.ConstantWithMetamorphicTestBool( @@ -123,7 +123,7 @@ var UseEFOS = settings.RegisterBoolSetting( // // This cluster setting will be removed in a subsequent release. var IngestAsFlushable = settings.RegisterBoolSetting( - settings.TenantWritable, // used to init temp storage in virtual cluster servers + settings.ApplicationLevel, // used to init temp storage in virtual cluster servers "storage.ingest_as_flushable.enabled", "set to true to enable lazy ingestion of sstables", util.ConstantWithMetamorphicTestBool( diff --git a/pkg/ts/db.go b/pkg/ts/db.go index a111b9111ecc..f1985d8f56d1 100644 --- a/pkg/ts/db.go +++ b/pkg/ts/db.go @@ -41,7 +41,7 @@ var ( // TimeseriesStorageEnabled controls whether to store timeseries data to disk. var TimeseriesStorageEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "timeseries.storage.enabled", "if set, periodic timeseries data is stored within the cluster; disabling is not recommended "+ "unless you are storing the data elsewhere", @@ -52,7 +52,7 @@ var TimeseriesStorageEnabled = settings.RegisterBoolSetting( // at he 10 second resolution. Data older than this is subject to being "rolled // up" into the 30 minute resolution and then deleted. var Resolution10sStorageTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "timeseries.storage.resolution_10s.ttl", "the maximum age of time series data stored at the 10 second resolution. Data older than this "+ "is subject to rollup and deletion.", @@ -63,7 +63,7 @@ var Resolution10sStorageTTL = settings.RegisterDurationSetting( // retained at he 30 minute resolution. Data older than this is subject to // deletion. var Resolution30mStorageTTL = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "timeseries.storage.resolution_30m.ttl", "the maximum age of time series data stored at the 30 minute resolution. Data older than this "+ "is subject to deletion.", diff --git a/pkg/ui/ui.go b/pkg/ui/ui.go index ec918b7d2f1b..f579a4dbb310 100644 --- a/pkg/ui/ui.go +++ b/pkg/ui/ui.go @@ -39,7 +39,7 @@ const ( ) var _ = settings.RegisterEnumSetting( - settings.TenantWritable, + settings.ApplicationLevel, "ui.display_timezone", "the timezone used to format timestamps in the ui", "Etc/UTC", diff --git a/pkg/util/admission/io_load_listener.go b/pkg/util/admission/io_load_listener.go index ea4f66507514..eea6ea03b993 100644 --- a/pkg/util/admission/io_load_listener.go +++ b/pkg/util/admission/io_load_listener.go @@ -72,7 +72,7 @@ var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting( // L0FileCountOverloadThreshold sets a file count threshold that signals an // overloaded store. var L0FileCountOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.l0_file_count_overload_threshold", "when the L0 file count exceeds this theshold, the store is considered overloaded", l0FileCountOverloadThreshold, settings.PositiveInt) @@ -80,7 +80,7 @@ var L0FileCountOverloadThreshold = settings.RegisterIntSetting( // L0SubLevelCountOverloadThreshold sets a sub-level count threshold that // signals an overloaded store. var L0SubLevelCountOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.l0_sub_level_count_overload_threshold", "when the L0 sub-level count exceeds this threshold, the store is considered overloaded", l0SubLevelCountOverloadThreshold, settings.PositiveInt) diff --git a/pkg/util/admission/kv_slot_adjuster.go b/pkg/util/admission/kv_slot_adjuster.go index 8410af15d5c1..14a85d6852ac 100644 --- a/pkg/util/admission/kv_slot_adjuster.go +++ b/pkg/util/admission/kv_slot_adjuster.go @@ -22,7 +22,7 @@ import ( // which the CPU will be considered overloaded, when running in a node that // executes KV operations. var KVSlotAdjusterOverloadThreshold = settings.RegisterIntSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.kv_slot_adjuster.overload_threshold", "when the number of runnable goroutines per CPU is greater than this threshold, the "+ "slot adjuster considers the cpu to be overloaded", diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index 99151b3e78ed..84b61fc3b714 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -42,7 +42,7 @@ import ( // KVAdmissionControlEnabled). Only the system tenant can modify these // settings in the storage servers, while a regular tenant can modify these // settings for their SQL-only servers. Which is why these are typically -// TenantWritable. +// ApplicationLevel. // KVAdmissionControlEnabled controls whether KV server-side admission control // is enabled. @@ -70,7 +70,7 @@ var KVBulkOnlyAdmissionControlEnabled = settings.RegisterBoolSetting( // SQLKVResponseAdmissionControlEnabled controls whether response processing // in SQL, for KV requests, is enabled. var SQLKVResponseAdmissionControlEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.sql_kv_response.enabled", "when true, work performed by the SQL layer when receiving a KV response is subject to "+ "admission control", @@ -80,7 +80,7 @@ var SQLKVResponseAdmissionControlEnabled = settings.RegisterBoolSetting( // SQLSQLResponseAdmissionControlEnabled controls whether response processing // in SQL, for DistSQL requests, is enabled. var SQLSQLResponseAdmissionControlEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.sql_sql_response.enabled", "when true, work performed by the SQL layer when receiving a DistSQL response is subject "+ "to admission control", @@ -121,14 +121,14 @@ var KVStoresTenantWeightsEnabled = settings.RegisterBoolSetting( // enabled, and we don't want to carry forward unnecessarily granular // settings. var EpochLIFOEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.epoch_lifo.enabled", "when true, epoch-LIFO behavior is enabled when there is significant delay in admission", false, settings.WithPublic) var epochLIFOEpochDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.epoch_lifo.epoch_duration", "the duration of an epoch, for epoch-LIFO admission control ordering", epochLength, @@ -140,7 +140,7 @@ var epochLIFOEpochDuration = settings.RegisterDurationSetting( }), settings.WithPublic) var epochLIFOEpochClosingDeltaDuration = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.epoch_lifo.epoch_closing_delta_duration", "the delta duration before closing an epoch, for epoch-LIFO admission control ordering", epochClosingDelta, @@ -152,7 +152,7 @@ var epochLIFOEpochClosingDeltaDuration = settings.RegisterDurationSetting( }), settings.WithPublic) var epochLIFOQueueDelayThresholdToSwitchToLIFO = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "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, @@ -164,7 +164,7 @@ var epochLIFOQueueDelayThresholdToSwitchToLIFO = settings.RegisterDurationSettin }), settings.WithPublic) var rangeSequencerGCThreshold = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "admission.replication_control.range_sequencer_gc_threshold", "the inactive duration for a range sequencer after it's garbage collected", 5*time.Minute, diff --git a/pkg/util/log/logcrash/crash_reporting.go b/pkg/util/log/logcrash/crash_reporting.go index af04f8b07c50..d5ea9cdc0f97 100644 --- a/pkg/util/log/logcrash/crash_reporting.go +++ b/pkg/util/log/logcrash/crash_reporting.go @@ -53,7 +53,7 @@ var ( // Doing this, rather than just using a default of `true`, means that a node // will not errantly send a report using a default before loading settings. DiagnosticsReportingEnabled = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "diagnostics.reporting.enabled", "enable reporting diagnostic metrics to cockroach labs", false, @@ -61,7 +61,7 @@ var ( // CrashReports wraps "diagnostics.reporting.send_crash_reports.enabled". CrashReports = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "diagnostics.reporting.send_crash_reports", "send crash and panic reports", true, @@ -70,7 +70,7 @@ var ( // PanicOnAssertions wraps "debug.panic_on_failed_assertions.enabled" PanicOnAssertions = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "debug.panic_on_failed_assertions", "panic when an assertion fails rather than reporting", false, diff --git a/pkg/util/schedulerlatency/sampler.go b/pkg/util/schedulerlatency/sampler.go index 77fb34e17dbd..66349d46f46b 100644 --- a/pkg/util/schedulerlatency/sampler.go +++ b/pkg/util/schedulerlatency/sampler.go @@ -34,7 +34,7 @@ import ( // requests in work queues as part of this tick). Might be worth checking for // grantees more frequently independent of this sample period. var samplePeriod = settings.RegisterDurationSetting( - settings.TenantWritable, // used in virtual clusters + settings.ApplicationLevel, // used in virtual clusters "scheduler_latency.sample_period", "controls the duration between consecutive scheduler latency samples", 100*time.Millisecond, @@ -47,7 +47,7 @@ var samplePeriod = settings.RegisterDurationSetting( ) var sampleDuration = settings.RegisterDurationSetting( - settings.TenantWritable, // used in virtual clusters + settings.ApplicationLevel, // used in virtual clusters "scheduler_latency.sample_duration", "controls the duration over which each scheduler latency sample is a measurement over", 2500*time.Millisecond, diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index c632f3ee420a..c27ddc915b2d 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -106,14 +106,14 @@ const ( // resolved via #58610, this setting can be removed so that all traces // have redactability enabled. var enableTraceRedactable = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.redactable.enabled", "set to true to enable finer-grainer redactability for unstructured events in traces", true, ) var enableNetTrace = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.debug.enable", "if set, traces for recent requests can be seen at https:///debug/requests", false, @@ -121,7 +121,7 @@ var enableNetTrace = settings.RegisterBoolSetting( settings.WithPublic) var openTelemetryCollector = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.opentelemetry.collector", "address of an OpenTelemetry trace collector to receive "+ "traces using the otel gRPC protocol, as :. "+ @@ -138,7 +138,7 @@ var openTelemetryCollector = settings.RegisterStringSetting( ) var jaegerAgent = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.jaeger.agent", "the address of a Jaeger agent to receive traces using the "+ "Jaeger UDP Thrift protocol, as :. "+ @@ -157,7 +157,7 @@ var jaegerAgent = settings.RegisterStringSetting( // ZipkinCollector is the cluster setting that specifies the Zipkin instance // to send traces to, if any. var ZipkinCollector = settings.RegisterStringSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.zipkin.collector", "the address of a Zipkin instance to receive traces, as :. "+ "If no port is specified, 9411 will be used.", @@ -178,14 +178,14 @@ var ZipkinCollector = settings.RegisterStringSetting( // finished. When disabled, span creation is short-circuited for a small // performance improvement. var EnableActiveSpansRegistry = settings.RegisterBoolSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.span_registry.enabled", "if set, ongoing traces can be seen at https:///#/debug/tracez", envutil.EnvOrDefaultBool("COCKROACH_REAL_SPANS", true), settings.WithPublic) var periodicSnapshotInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.ApplicationLevel, "trace.snapshot.rate", "if non-zero, interval at which background trace snapshots are captured", 0,