From 944f33b38f9ecd1db2c24aaa490c3b6055d5fac1 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 13 Apr 2022 16:49:10 +0200 Subject: [PATCH] cli: extend `gen settings-list` We want to have a way to print all settings with their class and visibility for internal investigations. This is also useful for public-facing documentation, where we want to split settings according to what users can expect from SET CLUSTER SETTINGS. The generated files are as follows: - `settings-for-tenant.txt`: a TSV file containing settings that tenants can access (i.e. excluding `system-only` settings which cannot be set/accessed by tenants). NEW: a new `class` column, containing either: - `tenant-rw`: the tenant can use both SHOW and SET - `tenant-ro`: the tenant can use only SHOW - `settings.html`: complete list of settings. NEW: the same `class` column as above. This column also contain `system-only` setting in this file. Also, a new `visibility` column, which separates public vs reserved settings. By request from the docs team, the column header and the labels to use for the various class should be configurable. The requested texts are used in the generation rules in BAZEL.bzl. Release note: None --- docs/generated/settings/BUILD.bazel | 4 +- .../settings/settings-for-tenants.txt | 424 +++++++-------- docs/generated/settings/settings.html | 490 +++++++++--------- pkg/cli/gen.go | 55 +- pkg/settings/registry.go | 1 + 5 files changed, 507 insertions(+), 467 deletions(-) diff --git a/docs/generated/settings/BUILD.bazel b/docs/generated/settings/BUILD.bazel index 35deaa72e728..e387b558f604 100644 --- a/docs/generated/settings/BUILD.bazel +++ b/docs/generated/settings/BUILD.bazel @@ -1,7 +1,7 @@ genrule( name = "settings", outs = ["settings.html"], - cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --format=rawhtml > $@", + cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --show-class --format=rawhtml --class-header-label 'Supported Deployments' --class-labels 'Dedicated/Self-Hosted,Serverless/Dedicated/Self-Hosted (read-only),Serverless/Dedicated/Self-Hosted' > $@", exec_tools = ["//pkg/cmd/cockroach-short"], visibility = [ ":__pkg__", @@ -12,7 +12,7 @@ genrule( genrule( name = "settings_for_tenants", outs = ["settings-for-tenants.txt"], - cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --without-system-only > $@", + cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --show-class --without-system-only > $@", exec_tools = ["//pkg/cmd/cockroach-short"], visibility = [ ":__pkg__", diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 6501f5f7c87e..cf44a163950b 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -1,294 +1,294 @@ -Setting Type Default Description -admission.epoch_lifo.enabled boolean false when true, epoch-LIFO behavior is enabled when there is significant delay in admission -admission.epoch_lifo.epoch_closing_delta_duration duration 5ms the delta duration before closing an epoch, for epoch-LIFO admission control ordering -admission.epoch_lifo.epoch_duration duration 100ms the duration of an epoch, for epoch-LIFO admission control ordering -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 -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 -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 -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. -bulkio.backup.file_size byte size 128 MiB target size for individual data files produced during BACKUP -bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail -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 -bulkio.stream_ingestion.minimum_flush_interval duration 5s the minimum timestamp between flushes; flushes may still occur if internal buffers fill up -changefeed.backfill.scan_request_size integer 524288 the maximum number of bytes returned by each scan request -changefeed.balance_range_distribution.enable boolean false if enabled, the ranges are balanced equally among all nodes -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 -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 -changefeed.fast_gzip.enabled boolean true use fast gzip implementation -changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds -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 -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. -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 -cloudstorage.http.custom_ca string custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage -cloudstorage.timeout duration 10m0s the timeout for import/export storage operations -cluster.organization string organization name -cluster.preserve_downgrade_option string disable (automatic or manual) cluster version upgrade from the specified version until reset -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. -diagnostics.reporting.enabled boolean true enable reporting diagnostic metrics to cockroach labs -diagnostics.reporting.interval duration 1h0m0s interval at which diagnostics data should be reported -enterprise.license string the encoded cluster license -external.graphite.endpoint string if nonempty, push server metrics to the Graphite or Carbon server at the specified host:port -external.graphite.interval duration 10s the interval at which metrics are pushed to Graphite (if enabled) -feature.backup.enabled boolean true set to true to enable backups, false to disable; default is true -feature.changefeed.enabled boolean true set to true to enable changefeeds, false to disable; default is true -feature.export.enabled boolean true set to true to enable exports, false to disable; default is true -feature.import.enabled boolean true set to true to enable imports, false to disable; default is true -feature.restore.enabled boolean true set to true to enable restore, false to disable; default is true -feature.schema_change.enabled boolean true set to true to enable schema changes, false to disable; default is true -feature.stats.enabled boolean true set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true -jobs.retention_time duration 336h0m0s the amount of time for which records for completed jobs are retained -kv.protectedts.reconciliation.interval duration 5m0s the frequency for reconciling jobs with protected timestamp records -kv.rangefeed.enabled boolean false if set, rangefeed registration is enabled -kv.rangefeed.range_stuck_threshold duration 1m0s restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.closed_timestamp.side_transport_interval takes precedence) -kv.transaction.max_intents_bytes integer 4194304 maximum number of bytes used to track locks in transactions -kv.transaction.max_refresh_spans_bytes integer 4194304 maximum number of bytes used to track refresh spans in serializable transactions -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 -schedules.backup.gc_protection.enabled boolean true enable chaining of GC protection across backups run as part of a schedule -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] -security.ocsp.timeout duration 3s timeout before considering the OCSP server unreachable -server.auth_log.sql_connections.enabled boolean false if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes) -server.auth_log.sql_sessions.enabled boolean false if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes) -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 -server.child_metrics.enabled boolean false enables the exporting of child metrics, additional prometheus time series with extra labels -server.clock.forward_jump_check_enabled boolean false if enabled, forward clock jumps > max_offset/2 will cause a panic -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. -server.eventlog.enabled boolean true if set, logged notable events are also stored in the table system.eventlog -server.eventlog.ttl duration 2160h0m0s if nonzero, entries in system.eventlog older than this duration are periodically purged -server.host_based_authentication.configuration string host-based authentication configuration to use during connection authentication -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. -server.identity_map.configuration string system-identity to database-username mappings -server.log_gc.max_deletions_per_cycle integer 1000 the maximum number of entries to delete on each purge of log-like system tables -server.log_gc.period duration 1h0m0s the period at which log-like system tables are checked for old entries -server.max_connections_per_gateway integer -1 the maximum number of non-superuser 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. -server.oidc_authentication.autologin boolean false if true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint -server.oidc_authentication.button_text string Login 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) -server.oidc_authentication.claim_json_key string sets JSON key of principal to extract from payload after OIDC authentication completes (usually email or sid) -server.oidc_authentication.client_id string sets OIDC client id -server.oidc_authentication.client_secret string sets OIDC client secret -server.oidc_authentication.enabled boolean false enables or disabled OIDC login for the DB Console -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) -server.oidc_authentication.provider_url string sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve) -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) -server.oidc_authentication.scopes string openid sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`) -server.rangelog.ttl duration 720h0m0s if nonzero, entries in system.rangelog older than this duration are periodically purged -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) -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.) -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) -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 -server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM -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 -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. -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] -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) -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) -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 -server.user_login.timeout duration 10s timeout after which client authentication times out if some system range is unavailable (0 = no timeout) -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 -server.web_session.purge.ttl duration 1h0m0s if nonzero, entries in system.web_sessions older than this duration are periodically purged -server.web_session_timeout duration 168h0m0s the duration that a newly created web session will be valid -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 -sql.auth.resolve_membership_single_scan.enabled boolean true determines whether to populate the role membership cache with a single scan -sql.closed_session_cache.capacity integer 1000 the maximum number of sessions in the cache -sql.closed_session_cache.time_to_live integer 3600 the maximum time to live, in seconds -sql.contention.event_store.capacity byte size 64 MiB the in-memory storage capacity per-node of contention event store -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 -sql.contention.txn_id_cache.max_size byte size 64 MiB the maximum byte size TxnID cache will use (set to 0 to disable) -sql.cross_db_fks.enabled boolean false if true, creating foreign key references across databases is allowed -sql.cross_db_sequence_owners.enabled boolean false if true, creating sequences owned by tables from other databases is allowed -sql.cross_db_sequence_references.enabled boolean false if true, sequences referenced by tables from other databases are allowed -sql.cross_db_views.enabled boolean false if true, creating views that refer to other databases is allowed +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.scan_request_size integer 524288 the maximum number of bytes returned by each scan request tenant-rw +changefeed.balance_range_distribution.enable boolean false if enabled, the ranges are balanced equally among all nodes 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.node_throttle_config string specifies node level throttling configuration for all changefeeeds 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.protectedts.reconciliation.interval duration 5m0s the frequency for reconciling jobs with protected timestamp records tenant-ro +kv.rangefeed.enabled boolean false if set, rangefeed registration is enabled tenant-rw +kv.rangefeed.range_stuck_threshold duration 1m0s restart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.closed_timestamp.side_transport_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.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.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 non-superuser 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 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 Login 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.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.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 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw 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" +This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html" tenant-rw sql.defaults.zigzag_join.enabled boolean true "default value for enable_zigzag_join session setting; allows 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" -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 -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 -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 -sql.insights.anomaly_detection.enabled boolean true enable per-fingerprint latency recording and anomaly detection -sql.insights.anomaly_detection.latency_threshold duration 50ms statements must surpass this threshold to trigger anomaly detection and identification -sql.insights.anomaly_detection.memory_limit byte size 1.0 MiB the maximum amount of memory allowed for tracking statement latencies -sql.insights.execution_insights_capacity integer 1000 the size of the per-node store of execution insights -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 -sql.insights.latency_threshold duration 100ms amount of time after which an executing statement is considered slow. Use 0 to disable. -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. -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. -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 -sql.metrics.index_usage_stats.enabled boolean true collect per index usage statistics -sql.metrics.max_mem_reported_stmt_fingerprints integer 100000 the maximum number of reported statement fingerprints stored in memory -sql.metrics.max_mem_reported_txn_fingerprints integer 100000 the maximum number of reported transaction fingerprints stored in memory -sql.metrics.max_mem_stmt_fingerprints integer 100000 the maximum number of statement fingerprints stored in memory -sql.metrics.max_mem_txn_fingerprints integer 100000 the maximum number of transaction fingerprints stored in memory -sql.metrics.statement_details.dump_to_logs boolean false dump collected statement statistics to node logs when periodically cleared -sql.metrics.statement_details.enabled boolean true collect per-statement query statistics -sql.metrics.statement_details.gateway_node.enabled boolean true save the gateway node for each statement fingerprint. If false, the value will be stored as 0. -sql.metrics.statement_details.index_recommendation_collection.enabled boolean true generate an index recommendation for each fingerprint ID -sql.metrics.statement_details.max_mem_reported_idx_recommendations integer 5000 the maximum number of reported index recommendation info stored in memory -sql.metrics.statement_details.plan_collection.enabled boolean false periodically save a logical plan for each fingerprint -sql.metrics.statement_details.plan_collection.period duration 5m0s the time until a new logical plan is collected -sql.metrics.statement_details.threshold duration 0s minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected. -sql.metrics.transaction_details.enabled boolean true collect per-application transaction statistics -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) -sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region -sql.notices.enabled boolean true enable notices in the server/client protocol being sent -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 -sql.pgwire.multiple_active_portals.enabled boolean false if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan -sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job -sql.show_ranges_deprecated_behavior.enabled boolean true 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. -sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators -sql.stats.automatic_collection.enabled boolean true automatic statistics collection mode -sql.stats.automatic_collection.fraction_stale_rows float 0.2 target fraction of stale rows per table that will trigger a statistics refresh -sql.stats.automatic_collection.min_stale_rows integer 500 target minimum number of stale rows per table that will trigger a statistics refresh -sql.stats.cleanup.recurrence string @hourly cron-tab recurrence for SQL Stats cleanup job -sql.stats.flush.enabled boolean true if set, SQL execution statistics are periodically flushed to disk -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 -sql.stats.forecasts.enabled boolean true when true, enables generation of statistics forecasts by default for all tables -sql.stats.histogram_collection.enabled boolean true histogram collection mode -sql.stats.multi_column_collection.enabled boolean true multi-column statistics collection mode -sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum retention period for table statistics collected on non-default columns -sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables -sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job -sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request -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 -sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer -sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables -sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel -sql.temp_object_cleaner.cleanup_interval duration 30m0s how often to clean up orphaned temporary objects -sql.temp_object_cleaner.wait_interval duration 30m0s how long after creation a temporary object will be cleaned up -sql.trace.log_statement_execute boolean false set to true to enable logging of executed statements -sql.trace.session_eventlog.enabled boolean false set to true to enable session tracing; note that enabling this may have a negative performance impact -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 -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) -sql.ttl.default_delete_batch_size integer 100 default amount of rows to delete in a single query during a TTL job -sql.ttl.default_delete_rate_limit integer 0 default delete rate limit for all TTL jobs. Use 0 to signify no rate limit. -sql.ttl.default_select_batch_size integer 500 default amount of rows to select in a single query during a TTL job -sql.ttl.job.enabled boolean true whether the TTL job is enabled -sql.txn_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored -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 -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. -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. -trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests -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. -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. -trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured -trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez -trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000023.1-4 set the active cluster version in the format '.' +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.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.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 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 true 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.pgwire.multiple_active_portals.enabled boolean false if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan 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 true 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.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_collection.enabled boolean true histogram collection mode 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 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.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.trace.log_statement_execute boolean false set to true to enable logging of 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 for all TTL jobs. 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_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored 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.enable 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 +version version 1000023.1-4 set the active cluster version in the format '.' tenant-rw diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 214f77e83f61..09702cf092d0 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -1,249 +1,249 @@ - + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
SettingTypeDefaultDescription
SettingTypeDefaultDescriptionSupported Deployments
admission.disk_bandwidth_tokens.elastic.enabled
booleantruewhen true, and provisioned bandwidth for the disk corresponding to a store is configured, tokens for elastic work will be limited if disk bandwidth becomes a bottleneck
admission.epoch_lifo.enabled
booleanfalsewhen true, epoch-LIFO behavior is enabled when there is significant delay in admission
admission.epoch_lifo.epoch_closing_delta_duration
duration5msthe delta duration before closing an epoch, for epoch-LIFO admission control ordering
admission.epoch_lifo.epoch_duration
duration100msthe duration of an epoch, for epoch-LIFO admission control ordering
admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo
duration105msthe queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering
admission.kv.enabled
booleantruewhen true, work performed by the KV layer is subject to admission control
admission.kv.stores.tenant_weights.enabled
booleanfalsewhen true, tenant weights are enabled for KV-stores admission control
admission.kv.tenant_weights.enabled
booleanfalsewhen true, tenant weights are enabled for KV admission control
admission.sql_kv_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a KV response is subject to admission control
admission.sql_sql_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control
bulkio.backup.deprecated_full_backup_with_subdir.enabled
booleanfalsewhen 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.
bulkio.backup.file_size
byte size128 MiBtarget size for individual data files produced during BACKUP
bulkio.backup.read_timeout
duration5m0samount of time after which a read attempt is considered timed out, which causes the backup to fail
bulkio.backup.read_with_priority_after
duration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying reads
bulkio.stream_ingestion.minimum_flush_interval
duration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill up
changefeed.backfill.scan_request_size
integer524288the maximum number of bytes returned by each scan request
changefeed.balance_range_distribution.enable
booleanfalseif enabled, the ranges are balanced equally among all nodes
changefeed.event_consumer_worker_queue_size
integer16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can buffer
changefeed.event_consumer_workers
integer0the 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
changefeed.fast_gzip.enabled
booleantrueuse fast gzip implementation
changefeed.node_throttle_config
stringspecifies node level throttling configuration for all changefeeeds
changefeed.schema_feed.read_with_priority_after
duration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disables
changefeed.sink_io_workers
integer0the 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.
cloudstorage.azure.concurrent_upload_buffers
integer1controls 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
cloudstorage.http.custom_ca
stringcustom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage
cloudstorage.timeout
duration10m0sthe timeout for import/export storage operations
cluster.organization
stringorganization name
cluster.preserve_downgrade_option
stringdisable (automatic or manual) cluster version upgrade from the specified version until reset
diagnostics.active_query_dumps.enabled
booleantrueexperimental: enable dumping of anonymized active queries to disk when node is under memory pressure
diagnostics.forced_sql_stat_reset.interval
duration2h0m0sinterval after which the reported SQL Stats are reset even if not collected by telemetry reporter. It has a max value of 24H.
diagnostics.reporting.enabled
booleantrueenable reporting diagnostic metrics to cockroach labs
diagnostics.reporting.interval
duration1h0m0sinterval at which diagnostics data should be reported
enterprise.license
stringthe encoded cluster license
external.graphite.endpoint
stringif nonempty, push server metrics to the Graphite or Carbon server at the specified host:port
external.graphite.interval
duration10sthe interval at which metrics are pushed to Graphite (if enabled)
feature.backup.enabled
booleantrueset to true to enable backups, false to disable; default is true
feature.changefeed.enabled
booleantrueset to true to enable changefeeds, false to disable; default is true
feature.export.enabled
booleantrueset to true to enable exports, false to disable; default is true
feature.import.enabled
booleantrueset to true to enable imports, false to disable; default is true
feature.restore.enabled
booleantrueset to true to enable restore, false to disable; default is true
feature.schema_change.enabled
booleantrueset to true to enable schema changes, false to disable; default is true
feature.stats.enabled
booleantrueset to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true
jobs.retention_time
duration336h0m0sthe amount of time for which records for completed jobs are retained
kv.allocator.load_based_lease_rebalancing.enabled
booleantrueset to enable rebalancing of range leases based on load and latency
kv.allocator.load_based_rebalancing
enumerationleases and replicaswhether to rebalance based on the distribution of load across stores [off = 0, leases = 1, leases and replicas = 2]
kv.allocator.load_based_rebalancing.objective
enumerationcpuwhat objective does the cluster use to rebalance; if set to `qps` the cluster will attempt to balance qps among stores, if set to `cpu` the cluster will attempt to balance cpu usage among stores [qps = 0, cpu = 1]
kv.allocator.load_based_rebalancing_interval
duration1m0sthe rough interval at which each store will check for load-based lease / replica rebalancing opportunities
kv.allocator.qps_rebalance_threshold
float0.1minimum fraction away from the mean a store's QPS (such as queries per second) can be before it is considered overfull or underfull
kv.allocator.range_rebalance_threshold
float0.05minimum fraction away from the mean a store's range count can be before it is considered overfull or underfull
kv.allocator.store_cpu_rebalance_threshold
float0.1minimum fraction away from the mean a store's cpu usage can be before it is considered overfull or underfull
kv.bulk_io_write.max_rate
byte size1.0 TiBthe rate limit (bytes/sec) to use for writes to disk on behalf of bulk io ops
kv.bulk_sst.max_allowed_overage
byte size64 MiBif positive, allowed size in excess of target size for SSTs 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
kv.bulk_sst.target_size
byte size16 MiBtarget 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
kv.closed_timestamp.follower_reads_enabled
booleantrueallow (all) replicas to serve consistent historical reads based on closed timestamp information
kv.log_range_and_node_events.enabled
booleantrueset to true to transactionally log range events (e.g., split, merge, add/remove voter/non-voter) into system.rangelogand node join and restart events into system.eventolog
kv.protectedts.reconciliation.interval
duration5m0sthe frequency for reconciling jobs with protected timestamp records
kv.range_split.by_load_enabled
booleantrueallow automatic splits of ranges based on where load is concentrated
kv.range_split.load_cpu_threshold
duration500msthe CPU use per second over which, the range becomes a candidate for load based splitting
kv.range_split.load_qps_threshold
integer2500the QPS over which, the range becomes a candidate for load based splitting
kv.rangefeed.enabled
booleanfalseif set, rangefeed registration is enabled
kv.rangefeed.range_stuck_threshold
duration1m0srestart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.closed_timestamp.side_transport_interval takes precedence)
kv.replica_circuit_breaker.slow_replication_threshold
duration1m0sduration after which slow proposals trip the per-Replica circuit breaker (zero duration disables breakers)
kv.replica_stats.addsst_request_size_factor
integer50000the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1
kv.replication_reports.interval
duration1m0sthe frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable)
kv.snapshot_rebalance.max_rate
byte size32 MiBthe rate limit (bytes/sec) to use for rebalance and upreplication snapshots
kv.snapshot_recovery.max_rate
byte size32 MiBthe rate limit (bytes/sec) to use for recovery snapshots
kv.transaction.max_intents_bytes
integer4194304maximum number of bytes used to track locks in transactions
kv.transaction.max_refresh_spans_bytes
integer4194304maximum number of bytes used to track refresh spans in serializable transactions
kv.transaction.reject_over_max_intents_budget.enabled
booleanfalseif set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressed
kvadmission.store.provisioned_bandwidth
byte size0 Bif set to a non-zero value, this is used as the provisioned bandwidth (in bytes/s), for each store. It can be over-ridden on a per-store basis using the --store flag
schedules.backup.gc_protection.enabled
booleantrueenable chaining of GC protection across backups run as part of a schedule
security.ocsp.mode
enumerationoffuse 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]
security.ocsp.timeout
duration3stimeout before considering the OCSP server unreachable
server.auth_log.sql_connections.enabled
booleanfalseif set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes)
server.auth_log.sql_sessions.enabled
booleanfalseif set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)
server.authentication_cache.enabled
booleantrueenables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information
server.child_metrics.enabled
booleanfalseenables the exporting of child metrics, additional prometheus time series with extra labels
server.clock.forward_jump_check_enabled
booleanfalseif enabled, forward clock jumps > max_offset/2 will cause a panic
server.clock.persist_upper_bound_interval
duration0sthe 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.
server.consistency_check.max_rate
byte size8.0 MiBthe rate limit (bytes/sec) to use for consistency checks; used in conjunction with server.consistency_check.interval to control the frequency of consistency checks. Note that setting this too high can negatively impact performance.
server.controller.default_tenant
stringsystemname of the tenant to use to serve requests when clients don't specify a tenant
server.eventlog.enabled
booleantrueif set, logged notable events are also stored in the table system.eventlog
server.eventlog.ttl
duration2160h0m0sif nonzero, entries in system.eventlog older than this duration are periodically purged
server.host_based_authentication.configuration
stringhost-based authentication configuration to use during connection authentication
server.hsts.enabled
booleanfalseif 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.
server.identity_map.configuration
stringsystem-identity to database-username mappings
server.log_gc.max_deletions_per_cycle
integer1000the maximum number of entries to delete on each purge of log-like system tables
server.log_gc.period
duration1h0m0sthe period at which log-like system tables are checked for old entries
server.max_connections_per_gateway
integer-1the maximum number of non-superuser 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.
server.oidc_authentication.autologin
booleanfalseif true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpoint
server.oidc_authentication.button_text
stringLogin with your OIDC providertext to show on button on DB Console login page to login with your OIDC provider (only shown if OIDC is enabled)
server.oidc_authentication.claim_json_key
stringsets JSON key of principal to extract from payload after OIDC authentication completes (usually email or sid)
server.oidc_authentication.client_id
stringsets OIDC client id
server.oidc_authentication.client_secret
stringsets OIDC client secret
server.oidc_authentication.enabled
booleanfalseenables or disabled OIDC login for the DB Console
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)
server.oidc_authentication.provider_url
stringsets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)
server.oidc_authentication.redirect_url
stringhttps://localhost:8080/oidc/v1/callbacksets 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)
server.oidc_authentication.scopes
stringopenidsets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`)
server.rangelog.ttl
duration720h0m0sif nonzero, entries in system.rangelog older than this duration are periodically purged
server.secondary_tenants.redact_trace.enabled
booleantruecontrols if server side traces are redacted for tenant operations
server.shutdown.connection_wait
duration0sthe 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)
server.shutdown.drain_wait
duration0sthe 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.)
server.shutdown.lease_transfer_wait
duration5sthe timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.query_wait
duration10sthe 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)
server.time_until_store_dead
duration5m0sthe time after which if there is no new gossiped information about a store, it is considered dead
server.user_login.cert_password_method.auto_scram_promotion.enabled
booleantruewhether to automatically promote cert-password authentication to use SCRAM
server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled
booleantrueif server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcrypt
server.user_login.min_password_length
integer1the 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.
server.user_login.password_encryption
enumerationscram-sha-256which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3]
server.user_login.password_hashes.default_cost.crdb_bcrypt
integer10the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31)
server.user_login.password_hashes.default_cost.scram_sha_256
integer10610the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000)
server.user_login.rehash_scram_stored_passwords_on_cost_change.enabled
booleantrueif 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
server.user_login.timeout
duration10stimeout after which client authentication times out if some system range is unavailable (0 = no timeout)
server.user_login.upgrade_bcrypt_stored_passwords_to_scram.enabled
booleantrueif server.user_login.password_encryption=scram-sha-256, this controls whether to automatically re-encode stored passwords using crdb-bcrypt to scram-sha-256
server.web_session.purge.ttl
duration1h0m0sif nonzero, entries in system.web_sessions older than this duration are periodically purged
server.web_session_timeout
duration168h0m0sthe duration that a newly created web session will be valid
spanconfig.bounds.enabled
booleantruedictates whether span config bounds are consulted when serving span configs for secondary tenants
sql.auth.change_own_password.enabled
booleanfalsecontrols whether a user is allowed to change their own password, even if they have no other privileges
sql.auth.resolve_membership_single_scan.enabled
booleantruedetermines whether to populate the role membership cache with a single scan
sql.closed_session_cache.capacity
integer1000the maximum number of sessions in the cache
sql.closed_session_cache.time_to_live
integer3600the maximum time to live, in seconds
sql.contention.event_store.capacity
byte size64 MiBthe in-memory storage capacity per-node of contention event store
sql.contention.event_store.duration_threshold
duration0sminimum contention duration to cause the contention events to be collected into crdb_internal.transaction_contention_events
sql.contention.txn_id_cache.max_size
byte size64 MiBthe maximum byte size TxnID cache will use (set to 0 to disable)
sql.cross_db_fks.enabled
booleanfalseif true, creating foreign key references across databases is allowed
sql.cross_db_sequence_owners.enabled
booleanfalseif true, creating sequences owned by tables from other databases is allowed
sql.cross_db_sequence_references.enabled
booleanfalseif true, sequences referenced by tables from other databases are allowed
sql.cross_db_views.enabled
booleanfalseif true, creating views that refer to other databases is allowed
sql.defaults.cost_scans_with_default_col_size.enabled
booleanfalsesetting 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
sql.defaults.datestyle
enumerationiso, mdydefault 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
sql.defaults.default_hash_sharded_index_bucket_count
integer16used 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
sql.defaults.default_int_size
integer8the 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
sql.defaults.disallow_full_table_scans.enabled
booleanfalsesetting 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
sql.defaults.distsql
enumerationautodefault 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
sql.defaults.experimental_alter_column_type.enabled
booleanfalsedefault 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
sql.defaults.experimental_distsql_planning
enumerationoffdefault 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
sql.defaults.experimental_enable_unique_without_index_constraints.enabled
booleanfalsedefault 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
sql.defaults.experimental_implicit_column_partitioning.enabled
booleanfalsedefault 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
sql.defaults.experimental_temporary_tables.enabled
booleanfalsedefault 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
sql.defaults.foreign_key_cascades_limit
integer10000default 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
sql.defaults.idle_in_session_timeout
duration0sdefault 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
sql.defaults.idle_in_transaction_session_timeout
duration0sdefault 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
sql.defaults.implicit_select_for_update.enabled
booleantruedefault 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
sql.defaults.insert_fast_path.enabled
booleantruedefault 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
sql.defaults.intervalstyle
enumerationpostgresdefault 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
sql.defaults.large_full_scan_rows
float1000default 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
sql.defaults.locality_optimized_partitioned_index_scan.enabled
booleantruedefault 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
sql.defaults.lock_timeout
duration0sdefault 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
sql.defaults.on_update_rehome_row.enabled
booleantruedefault 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
sql.defaults.optimizer_use_histograms.enabled
booleantruedefault 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
sql.defaults.optimizer_use_multicol_stats.enabled
booleantruedefault 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
sql.defaults.override_alter_primary_region_in_super_region.enabled
booleanfalsedefault 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
sql.defaults.override_multi_region_zone_config.enabled
booleanfalsedefault 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
sql.defaults.prefer_lookup_joins_for_fks.enabled
booleanfalsedefault 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
sql.defaults.primary_region
stringif 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
sql.defaults.reorder_joins_limit
integer8default 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
sql.defaults.require_explicit_primary_keys.enabled
booleanfalsedefault 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
sql.defaults.results_buffer.size
byte size16 KiBdefault 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
sql.defaults.serial_normalization
enumerationrowiddefault 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
sql.defaults.statement_timeout
duration0sdefault 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
sql.defaults.stub_catalog_tables.enabled
booleantruedefault 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
sql.defaults.super_regions.enabled
booleanfalsedefault 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
sql.defaults.transaction_rows_read_err
integer0the 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
sql.defaults.transaction_rows_read_log
integer0the 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
sql.defaults.transaction_rows_written_err
integer0the 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
sql.defaults.transaction_rows_written_log
integer0the 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
sql.defaults.use_declarative_schema_changer
enumerationondefault 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
sql.defaults.vectorize
enumerationondefault 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
sql.defaults.zigzag_join.enabled
booleantruedefault value for enable_zigzag_join session setting; allows 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
sql.distsql.temp_storage.workmem
byte size64 MiBmaximum amount of memory in bytes a processor can use before falling back to temp storage
sql.guardrails.max_row_size_err
byte size512 MiBmaximum 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
sql.guardrails.max_row_size_log
byte size64 MiBmaximum 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
sql.hash_sharded_range_pre_split.max
integer16max pre-split ranges to have when adding hash sharded index to an existing table
sql.insights.anomaly_detection.enabled
booleantrueenable per-fingerprint latency recording and anomaly detection
sql.insights.anomaly_detection.latency_threshold
duration50msstatements must surpass this threshold to trigger anomaly detection and identification
sql.insights.anomaly_detection.memory_limit
byte size1.0 MiBthe maximum amount of memory allowed for tracking statement latencies
sql.insights.execution_insights_capacity
integer1000the size of the per-node store of execution insights
sql.insights.high_retry_count.threshold
integer10the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problem
sql.insights.latency_threshold
duration100msamount of time after which an executing statement is considered slow. Use 0 to disable.
sql.log.slow_query.experimental_full_table_scans.enabled
booleanfalsewhen 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.
sql.log.slow_query.internal_queries.enabled
booleanfalsewhen 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.
sql.log.slow_query.latency_threshold
duration0swhen set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node
sql.metrics.index_usage_stats.enabled
booleantruecollect per index usage statistics
sql.metrics.max_mem_reported_stmt_fingerprints
integer100000the maximum number of reported statement fingerprints stored in memory
sql.metrics.max_mem_reported_txn_fingerprints
integer100000the maximum number of reported transaction fingerprints stored in memory
sql.metrics.max_mem_stmt_fingerprints
integer100000the maximum number of statement fingerprints stored in memory
sql.metrics.max_mem_txn_fingerprints
integer100000the maximum number of transaction fingerprints stored in memory
sql.metrics.statement_details.dump_to_logs
booleanfalsedump collected statement statistics to node logs when periodically cleared
sql.metrics.statement_details.enabled
booleantruecollect per-statement query statistics
sql.metrics.statement_details.gateway_node.enabled
booleantruesave the gateway node for each statement fingerprint. If false, the value will be stored as 0.
sql.metrics.statement_details.index_recommendation_collection.enabled
booleantruegenerate an index recommendation for each fingerprint ID
sql.metrics.statement_details.max_mem_reported_idx_recommendations
integer5000the maximum number of reported index recommendation info stored in memory
sql.metrics.statement_details.plan_collection.enabled
booleanfalseperiodically save a logical plan for each fingerprint
sql.metrics.statement_details.plan_collection.period
duration5m0sthe time until a new logical plan is collected
sql.metrics.statement_details.threshold
duration0sminimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected.
sql.metrics.transaction_details.enabled
booleantruecollect per-application transaction statistics
sql.multiple_modifications_of_table.enabled
booleanfalseif 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)
sql.multiregion.drop_primary_region.enabled
booleantrueallows dropping the PRIMARY REGION of a database if it is the last region
sql.notices.enabled
booleantrueenable notices in the server/client protocol being sent
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled
booleanfalseif 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
sql.pgwire.multiple_active_portals.enabled
booleanfalseif true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan
sql.schema.telemetry.recurrence
string@weeklycron-tab recurrence for SQL schema telemetry job
sql.show_ranges_deprecated_behavior.enabled
booleantrueif 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.
sql.spatial.experimental_box2d_comparison_operators.enabled
booleanfalseenables the use of certain experimental box2d comparison operators
sql.stats.automatic_collection.enabled
booleantrueautomatic statistics collection mode
sql.stats.automatic_collection.fraction_stale_rows
float0.2target fraction of stale rows per table that will trigger a statistics refresh
sql.stats.automatic_collection.min_stale_rows
integer500target minimum number of stale rows per table that will trigger a statistics refresh
sql.stats.cleanup.recurrence
string@hourlycron-tab recurrence for SQL Stats cleanup job
sql.stats.flush.enabled
booleantrueif set, SQL execution statistics are periodically flushed to disk
sql.stats.flush.interval
duration10m0sthe interval at which SQL execution statistics are flushed to disk, this value must be less than or equal to 1 hour
sql.stats.forecasts.enabled
booleantruewhen true, enables generation of statistics forecasts by default for all tables
sql.stats.histogram_collection.enabled
booleantruehistogram collection mode
sql.stats.multi_column_collection.enabled
booleantruemulti-column statistics collection mode
sql.stats.non_default_columns.min_retention_period
duration24h0m0sminimum retention period for table statistics collected on non-default columns
sql.stats.persisted_rows.max
integer1000000maximum number of rows of statement and transaction statistics that will be persisted in the system tables
sql.stats.post_events.enabled
booleanfalseif set, an event is logged for every CREATE STATISTICS job
sql.stats.response.max
integer20000the maximum number of statements and transaction stats returned in a CombinedStatements request
sql.stats.response.show_internal.enabled
booleanfalsecontrols 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
sql.stats.system_tables.enabled
booleantruewhen true, enables use of statistics on system tables by the query optimizer
sql.stats.system_tables_autostats.enabled
booleantruewhen true, enables automatic collection of statistics on system tables
sql.telemetry.query_sampling.enabled
booleanfalsewhen set to true, executed queries will emit an event on the telemetry logging channel
sql.temp_object_cleaner.cleanup_interval
duration30m0show often to clean up orphaned temporary objects
sql.temp_object_cleaner.wait_interval
duration30m0show long after creation a temporary object will be cleaned up
sql.trace.log_statement_execute
booleanfalseset to true to enable logging of executed statements
sql.trace.session_eventlog.enabled
booleanfalseset to true to enable session tracing; note that enabling this may have a negative performance impact
sql.trace.stmt.enable_threshold
duration0senables 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
sql.trace.txn.enable_threshold
duration0senables 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)
sql.ttl.default_delete_batch_size
integer100default amount of rows to delete in a single query during a TTL job
sql.ttl.default_delete_rate_limit
integer0default delete rate limit for all TTL jobs. Use 0 to signify no rate limit.
sql.ttl.default_select_batch_size
integer500default amount of rows to select in a single query during a TTL job
sql.ttl.job.enabled
booleantruewhether the TTL job is enabled
sql.txn_fingerprint_id_cache.capacity
integer100the maximum number of txn fingerprint IDs stored
storage.value_blocks.enabled
booleantrueset to true to enable writing of value blocks in sstables
timeseries.storage.enabled
booleantrueif set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere
timeseries.storage.resolution_10s.ttl
duration240h0m0sthe maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion.
timeseries.storage.resolution_30m.ttl
duration2160h0m0sthe maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion.
trace.debug.enable
booleanfalseif set, traces for recent requests can be seen at https://<ui>/debug/requests
trace.jaeger.agent
stringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.snapshot.rate
duration0sif non-zero, interval at which background trace snapshots are captured
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version
version1000023.1-4set the active cluster version in the format '<major>.<minor>'
admission.disk_bandwidth_tokens.elastic.enabled
booleantruewhen true, and provisioned bandwidth for the disk corresponding to a store is configured, tokens for elastic work will be limited if disk bandwidth becomes a bottleneckDedicated/Self-Hosted
admission.epoch_lifo.enabled
booleanfalsewhen true, epoch-LIFO behavior is enabled when there is significant delay in admissionServerless/Dedicated/Self-Hosted
admission.epoch_lifo.epoch_closing_delta_duration
duration5msthe delta duration before closing an epoch, for epoch-LIFO admission control orderingServerless/Dedicated/Self-Hosted
admission.epoch_lifo.epoch_duration
duration100msthe duration of an epoch, for epoch-LIFO admission control orderingServerless/Dedicated/Self-Hosted
admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo
duration105msthe queue delay encountered by a (tenant,priority) for switching to epoch-LIFO orderingServerless/Dedicated/Self-Hosted
admission.kv.enabled
booleantruewhen true, work performed by the KV layer is subject to admission controlDedicated/Self-Hosted
admission.kv.stores.tenant_weights.enabled
booleanfalsewhen true, tenant weights are enabled for KV-stores admission controlDedicated/Self-Hosted
admission.kv.tenant_weights.enabled
booleanfalsewhen true, tenant weights are enabled for KV admission controlDedicated/Self-Hosted
admission.sql_kv_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a KV response is subject to admission controlServerless/Dedicated/Self-Hosted
admission.sql_sql_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a DistSQL response is subject to admission controlServerless/Dedicated/Self-Hosted
bulkio.backup.deprecated_full_backup_with_subdir.enabled
booleanfalsewhen 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.Serverless/Dedicated/Self-Hosted
bulkio.backup.file_size
byte size128 MiBtarget size for individual data files produced during BACKUPServerless/Dedicated/Self-Hosted
bulkio.backup.read_timeout
duration5m0samount of time after which a read attempt is considered timed out, which causes the backup to failServerless/Dedicated/Self-Hosted
bulkio.backup.read_with_priority_after
duration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying readsServerless/Dedicated/Self-Hosted
bulkio.stream_ingestion.minimum_flush_interval
duration5sthe minimum timestamp between flushes; flushes may still occur if internal buffers fill upServerless/Dedicated/Self-Hosted
changefeed.backfill.scan_request_size
integer524288the maximum number of bytes returned by each scan requestServerless/Dedicated/Self-Hosted
changefeed.balance_range_distribution.enable
booleanfalseif enabled, the ranges are balanced equally among all nodesServerless/Dedicated/Self-Hosted
changefeed.event_consumer_worker_queue_size
integer16if changefeed.event_consumer_workers is enabled, this setting sets the maxmimum number of events which a worker can bufferServerless/Dedicated/Self-Hosted
changefeed.event_consumer_workers
integer0the 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 disabledServerless/Dedicated/Self-Hosted
changefeed.fast_gzip.enabled
booleantrueuse fast gzip implementationServerless/Dedicated/Self-Hosted
changefeed.node_throttle_config
stringspecifies node level throttling configuration for all changefeeedsServerless/Dedicated/Self-Hosted
changefeed.schema_feed.read_with_priority_after
duration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disablesServerless/Dedicated/Self-Hosted
changefeed.sink_io_workers
integer0the 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.Serverless/Dedicated/Self-Hosted
cloudstorage.azure.concurrent_upload_buffers
integer1controls 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 uploadServerless/Dedicated/Self-Hosted
cloudstorage.http.custom_ca
stringcustom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storageServerless/Dedicated/Self-Hosted
cloudstorage.timeout
duration10m0sthe timeout for import/export storage operationsServerless/Dedicated/Self-Hosted
cluster.organization
stringorganization nameServerless/Dedicated/Self-Hosted
cluster.preserve_downgrade_option
stringdisable (automatic or manual) cluster version upgrade from the specified version until resetServerless/Dedicated/Self-Hosted
diagnostics.active_query_dumps.enabled
booleantrueexperimental: enable dumping of anonymized active queries to disk when node is under memory pressureDedicated/Self-Hosted
diagnostics.forced_sql_stat_reset.interval
duration2h0m0sinterval after which the reported SQL Stats are reset even if not collected by telemetry reporter. It has a max value of 24H.Serverless/Dedicated/Self-Hosted
diagnostics.reporting.enabled
booleantrueenable reporting diagnostic metrics to cockroach labsServerless/Dedicated/Self-Hosted
diagnostics.reporting.interval
duration1h0m0sinterval at which diagnostics data should be reportedServerless/Dedicated/Self-Hosted
enterprise.license
stringthe encoded cluster licenseServerless/Dedicated/Self-Hosted
external.graphite.endpoint
stringif nonempty, push server metrics to the Graphite or Carbon server at the specified host:portServerless/Dedicated/Self-Hosted
external.graphite.interval
duration10sthe interval at which metrics are pushed to Graphite (if enabled)Serverless/Dedicated/Self-Hosted
feature.backup.enabled
booleantrueset to true to enable backups, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.changefeed.enabled
booleantrueset to true to enable changefeeds, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.export.enabled
booleantrueset to true to enable exports, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.import.enabled
booleantrueset to true to enable imports, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.restore.enabled
booleantrueset to true to enable restore, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.schema_change.enabled
booleantrueset to true to enable schema changes, false to disable; default is trueServerless/Dedicated/Self-Hosted
feature.stats.enabled
booleantrueset to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is trueServerless/Dedicated/Self-Hosted
jobs.retention_time
duration336h0m0sthe amount of time for which records for completed jobs are retainedServerless/Dedicated/Self-Hosted
kv.allocator.load_based_lease_rebalancing.enabled
booleantrueset to enable rebalancing of range leases based on load and latencyDedicated/Self-Hosted
kv.allocator.load_based_rebalancing
enumerationleases and replicaswhether to rebalance based on the distribution of load across stores [off = 0, leases = 1, leases and replicas = 2]Dedicated/Self-Hosted
kv.allocator.load_based_rebalancing.objective
enumerationcpuwhat objective does the cluster use to rebalance; if set to `qps` the cluster will attempt to balance qps among stores, if set to `cpu` the cluster will attempt to balance cpu usage among stores [qps = 0, cpu = 1]Dedicated/Self-Hosted
kv.allocator.load_based_rebalancing_interval
duration1m0sthe rough interval at which each store will check for load-based lease / replica rebalancing opportunitiesDedicated/Self-Hosted
kv.allocator.qps_rebalance_threshold
float0.1minimum fraction away from the mean a store's QPS (such as queries per second) can be before it is considered overfull or underfullDedicated/Self-Hosted
kv.allocator.range_rebalance_threshold
float0.05minimum fraction away from the mean a store's range count can be before it is considered overfull or underfullDedicated/Self-Hosted
kv.allocator.store_cpu_rebalance_threshold
float0.1minimum fraction away from the mean a store's cpu usage can be before it is considered overfull or underfullDedicated/Self-Hosted
kv.bulk_io_write.max_rate
byte size1.0 TiBthe rate limit (bytes/sec) to use for writes to disk on behalf of bulk io opsDedicated/Self-Hosted
kv.bulk_sst.max_allowed_overage
byte size64 MiBif positive, allowed size in excess of target size for SSTs 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 memoryDedicated/Self-Hosted
kv.bulk_sst.target_size
byte size16 MiBtarget 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 memoryDedicated/Self-Hosted
kv.closed_timestamp.follower_reads_enabled
booleantrueallow (all) replicas to serve consistent historical reads based on closed timestamp informationDedicated/Self-Hosted
kv.log_range_and_node_events.enabled
booleantrueset to true to transactionally log range events (e.g., split, merge, add/remove voter/non-voter) into system.rangelogand node join and restart events into system.eventologDedicated/Self-Hosted
kv.protectedts.reconciliation.interval
duration5m0sthe frequency for reconciling jobs with protected timestamp recordsServerless/Dedicated/Self-Hosted (read-only)
kv.range_split.by_load_enabled
booleantrueallow automatic splits of ranges based on where load is concentratedDedicated/Self-Hosted
kv.range_split.load_cpu_threshold
duration500msthe CPU use per second over which, the range becomes a candidate for load based splittingDedicated/Self-Hosted
kv.range_split.load_qps_threshold
integer2500the QPS over which, the range becomes a candidate for load based splittingDedicated/Self-Hosted
kv.rangefeed.enabled
booleanfalseif set, rangefeed registration is enabledServerless/Dedicated/Self-Hosted
kv.rangefeed.range_stuck_threshold
duration1m0srestart rangefeeds if they don't emit anything for the specified threshold; 0 disables (kv.closed_timestamp.side_transport_interval takes precedence)Serverless/Dedicated/Self-Hosted
kv.replica_circuit_breaker.slow_replication_threshold
duration1m0sduration after which slow proposals trip the per-Replica circuit breaker (zero duration disables breakers)Dedicated/Self-Hosted
kv.replica_stats.addsst_request_size_factor
integer50000the divisor that is applied to addsstable request sizes, then recorded in a leaseholders QPS; 0 means all requests are treated as cost 1Dedicated/Self-Hosted
kv.replication_reports.interval
duration1m0sthe frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable)Dedicated/Self-Hosted
kv.snapshot_rebalance.max_rate
byte size32 MiBthe rate limit (bytes/sec) to use for rebalance and upreplication snapshotsDedicated/Self-Hosted
kv.snapshot_recovery.max_rate
byte size32 MiBthe rate limit (bytes/sec) to use for recovery snapshotsDedicated/Self-Hosted
kv.transaction.max_intents_bytes
integer4194304maximum number of bytes used to track locks in transactionsServerless/Dedicated/Self-Hosted
kv.transaction.max_refresh_spans_bytes
integer4194304maximum number of bytes used to track refresh spans in serializable transactionsServerless/Dedicated/Self-Hosted
kv.transaction.reject_over_max_intents_budget.enabled
booleanfalseif set, transactions that exceed their lock tracking budget (kv.transaction.max_intents_bytes) are rejected instead of having their lock spans imprecisely compressedServerless/Dedicated/Self-Hosted
kvadmission.store.provisioned_bandwidth
byte size0 Bif set to a non-zero value, this is used as the provisioned bandwidth (in bytes/s), for each store. It can be over-ridden on a per-store basis using the --store flagDedicated/Self-Hosted
schedules.backup.gc_protection.enabled
booleantrueenable chaining of GC protection across backups run as part of a scheduleServerless/Dedicated/Self-Hosted
security.ocsp.mode
enumerationoffuse 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]Serverless/Dedicated/Self-Hosted
security.ocsp.timeout
duration3stimeout before considering the OCSP server unreachableServerless/Dedicated/Self-Hosted
server.auth_log.sql_connections.enabled
booleanfalseif set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes)Serverless/Dedicated/Self-Hosted
server.auth_log.sql_sessions.enabled
booleanfalseif set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)Serverless/Dedicated/Self-Hosted
server.authentication_cache.enabled
booleantrueenables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related informationServerless/Dedicated/Self-Hosted
server.child_metrics.enabled
booleanfalseenables the exporting of child metrics, additional prometheus time series with extra labelsServerless/Dedicated/Self-Hosted
server.clock.forward_jump_check_enabled
booleanfalseif enabled, forward clock jumps > max_offset/2 will cause a panicServerless/Dedicated/Self-Hosted
server.clock.persist_upper_bound_interval
duration0sthe 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.Serverless/Dedicated/Self-Hosted
server.consistency_check.max_rate
byte size8.0 MiBthe rate limit (bytes/sec) to use for consistency checks; used in conjunction with server.consistency_check.interval to control the frequency of consistency checks. Note that setting this too high can negatively impact performance.Dedicated/Self-Hosted
server.controller.default_tenant
stringsystemname of the tenant to use to serve requests when clients don't specify a tenantDedicated/Self-Hosted
server.eventlog.enabled
booleantrueif set, logged notable events are also stored in the table system.eventlogServerless/Dedicated/Self-Hosted
server.eventlog.ttl
duration2160h0m0sif nonzero, entries in system.eventlog older than this duration are periodically purgedServerless/Dedicated/Self-Hosted
server.host_based_authentication.configuration
stringhost-based authentication configuration to use during connection authenticationServerless/Dedicated/Self-Hosted
server.hsts.enabled
booleanfalseif 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.Serverless/Dedicated/Self-Hosted
server.identity_map.configuration
stringsystem-identity to database-username mappingsServerless/Dedicated/Self-Hosted
server.log_gc.max_deletions_per_cycle
integer1000the maximum number of entries to delete on each purge of log-like system tablesServerless/Dedicated/Self-Hosted
server.log_gc.period
duration1h0m0sthe period at which log-like system tables are checked for old entriesServerless/Dedicated/Self-Hosted
server.max_connections_per_gateway
integer-1the maximum number of non-superuser 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.Serverless/Dedicated/Self-Hosted
server.oidc_authentication.autologin
booleanfalseif true, logged-out visitors to the DB Console will be automatically redirected to the OIDC login endpointServerless/Dedicated/Self-Hosted
server.oidc_authentication.button_text
stringLogin with your OIDC providertext to show on button on DB Console login page to login with your OIDC provider (only shown if OIDC is enabled)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.claim_json_key
stringsets JSON key of principal to extract from payload after OIDC authentication completes (usually email or sid)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.client_id
stringsets OIDC client idServerless/Dedicated/Self-Hosted
server.oidc_authentication.client_secret
stringsets OIDC client secretServerless/Dedicated/Self-Hosted
server.oidc_authentication.enabled
booleanfalseenables or disabled OIDC login for the DB ConsoleServerless/Dedicated/Self-Hosted
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)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.provider_url
stringsets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.redirect_url
stringhttps://localhost:8080/oidc/v1/callbacksets 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) Serverless/Dedicated/Self-Hosted
server.oidc_authentication.scopes
stringopenidsets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`)Serverless/Dedicated/Self-Hosted
server.rangelog.ttl
duration720h0m0sif nonzero, entries in system.rangelog older than this duration are periodically purgedServerless/Dedicated/Self-Hosted
server.secondary_tenants.redact_trace.enabled
booleantruecontrols if server side traces are redacted for tenant operationsDedicated/Self-Hosted
server.shutdown.connection_wait
duration0sthe 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)Serverless/Dedicated/Self-Hosted
server.shutdown.drain_wait
duration0sthe 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.)Serverless/Dedicated/Self-Hosted
server.shutdown.lease_transfer_wait
duration5sthe timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)Dedicated/Self-Hosted
server.shutdown.query_wait
duration10sthe 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)Serverless/Dedicated/Self-Hosted
server.time_until_store_dead
duration5m0sthe time after which if there is no new gossiped information about a store, it is considered deadServerless/Dedicated/Self-Hosted
server.user_login.cert_password_method.auto_scram_promotion.enabled
booleantruewhether to automatically promote cert-password authentication to use SCRAMServerless/Dedicated/Self-Hosted
server.user_login.downgrade_scram_stored_passwords_to_bcrypt.enabled
booleantrueif server.user_login.password_encryption=crdb-bcrypt, this controls whether to automatically re-encode stored passwords using scram-sha-256 to crdb-bcryptServerless/Dedicated/Self-Hosted
server.user_login.min_password_length
integer1the 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.Serverless/Dedicated/Self-Hosted
server.user_login.password_encryption
enumerationscram-sha-256which hash method to use to encode cleartext passwords passed via ALTER/CREATE USER/ROLE WITH PASSWORD [crdb-bcrypt = 2, scram-sha-256 = 3]Serverless/Dedicated/Self-Hosted
server.user_login.password_hashes.default_cost.crdb_bcrypt
integer10the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method crdb-bcrypt (allowed range: 4-31)Serverless/Dedicated/Self-Hosted
server.user_login.password_hashes.default_cost.scram_sha_256
integer10610the hashing cost to use when storing passwords supplied as cleartext by SQL clients with the hashing method scram-sha-256 (allowed range: 4096-240000000000)Serverless/Dedicated/Self-Hosted
server.user_login.rehash_scram_stored_passwords_on_cost_change.enabled
booleantrueif 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 costServerless/Dedicated/Self-Hosted
server.user_login.timeout
duration10stimeout after which client authentication times out if some system range is unavailable (0 = no timeout)Serverless/Dedicated/Self-Hosted
server.user_login.upgrade_bcrypt_stored_passwords_to_scram.enabled
booleantrueif server.user_login.password_encryption=scram-sha-256, this controls whether to automatically re-encode stored passwords using crdb-bcrypt to scram-sha-256Serverless/Dedicated/Self-Hosted
server.web_session.purge.ttl
duration1h0m0sif nonzero, entries in system.web_sessions older than this duration are periodically purgedServerless/Dedicated/Self-Hosted
server.web_session_timeout
duration168h0m0sthe duration that a newly created web session will be validServerless/Dedicated/Self-Hosted
spanconfig.bounds.enabled
booleantruedictates whether span config bounds are consulted when serving span configs for secondary tenantsDedicated/Self-Hosted
sql.auth.change_own_password.enabled
booleanfalsecontrols whether a user is allowed to change their own password, even if they have no other privilegesServerless/Dedicated/Self-Hosted
sql.auth.resolve_membership_single_scan.enabled
booleantruedetermines whether to populate the role membership cache with a single scanServerless/Dedicated/Self-Hosted
sql.closed_session_cache.capacity
integer1000the maximum number of sessions in the cacheServerless/Dedicated/Self-Hosted
sql.closed_session_cache.time_to_live
integer3600the maximum time to live, in secondsServerless/Dedicated/Self-Hosted
sql.contention.event_store.capacity
byte size64 MiBthe in-memory storage capacity per-node of contention event storeServerless/Dedicated/Self-Hosted
sql.contention.event_store.duration_threshold
duration0sminimum contention duration to cause the contention events to be collected into crdb_internal.transaction_contention_eventsServerless/Dedicated/Self-Hosted
sql.contention.txn_id_cache.max_size
byte size64 MiBthe maximum byte size TxnID cache will use (set to 0 to disable)Serverless/Dedicated/Self-Hosted
sql.cross_db_fks.enabled
booleanfalseif true, creating foreign key references across databases is allowedServerless/Dedicated/Self-Hosted
sql.cross_db_sequence_owners.enabled
booleanfalseif true, creating sequences owned by tables from other databases is allowedServerless/Dedicated/Self-Hosted
sql.cross_db_sequence_references.enabled
booleanfalseif true, sequences referenced by tables from other databases are allowedServerless/Dedicated/Self-Hosted
sql.cross_db_views.enabled
booleanfalseif true, creating views that refer to other databases is allowedServerless/Dedicated/Self-Hosted
sql.defaults.cost_scans_with_default_col_size.enabled
booleanfalsesetting 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
Serverless/Dedicated/Self-Hosted
sql.defaults.datestyle
enumerationiso, mdydefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.default_hash_sharded_index_bucket_count
integer16used 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
Serverless/Dedicated/Self-Hosted
sql.defaults.default_int_size
integer8the 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
Serverless/Dedicated/Self-Hosted
sql.defaults.disallow_full_table_scans.enabled
booleanfalsesetting 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
Serverless/Dedicated/Self-Hosted
sql.defaults.distsql
enumerationautodefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.experimental_alter_column_type.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.experimental_distsql_planning
enumerationoffdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.experimental_enable_unique_without_index_constraints.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.experimental_implicit_column_partitioning.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.experimental_temporary_tables.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.foreign_key_cascades_limit
integer10000default 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
Serverless/Dedicated/Self-Hosted
sql.defaults.idle_in_session_timeout
duration0sdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.idle_in_transaction_session_timeout
duration0sdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.implicit_select_for_update.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.insert_fast_path.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.intervalstyle
enumerationpostgresdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.large_full_scan_rows
float1000default 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
Serverless/Dedicated/Self-Hosted
sql.defaults.locality_optimized_partitioned_index_scan.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.lock_timeout
duration0sdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.on_update_rehome_row.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.optimizer_use_histograms.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.optimizer_use_multicol_stats.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.override_alter_primary_region_in_super_region.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.override_multi_region_zone_config.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.prefer_lookup_joins_for_fks.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.primary_region
stringif 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
Serverless/Dedicated/Self-Hosted
sql.defaults.reorder_joins_limit
integer8default 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
Serverless/Dedicated/Self-Hosted
sql.defaults.require_explicit_primary_keys.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.results_buffer.size
byte size16 KiBdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.serial_normalization
enumerationrowiddefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.statement_timeout
duration0sdefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.stub_catalog_tables.enabled
booleantruedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.super_regions.enabled
booleanfalsedefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.transaction_rows_read_err
integer0the 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
Serverless/Dedicated/Self-Hosted
sql.defaults.transaction_rows_read_log
integer0the 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
Serverless/Dedicated/Self-Hosted
sql.defaults.transaction_rows_written_err
integer0the 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
Serverless/Dedicated/Self-Hosted
sql.defaults.transaction_rows_written_log
integer0the 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
Serverless/Dedicated/Self-Hosted
sql.defaults.use_declarative_schema_changer
enumerationondefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.vectorize
enumerationondefault 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
Serverless/Dedicated/Self-Hosted
sql.defaults.zigzag_join.enabled
booleantruedefault value for enable_zigzag_join session setting; allows 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
Serverless/Dedicated/Self-Hosted
sql.distsql.temp_storage.workmem
byte size64 MiBmaximum amount of memory in bytes a processor can use before falling back to temp storageServerless/Dedicated/Self-Hosted
sql.guardrails.max_row_size_err
byte size512 MiBmaximum 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 disableServerless/Dedicated/Self-Hosted
sql.guardrails.max_row_size_log
byte size64 MiBmaximum 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 disableServerless/Dedicated/Self-Hosted
sql.hash_sharded_range_pre_split.max
integer16max pre-split ranges to have when adding hash sharded index to an existing tableDedicated/Self-Hosted
sql.insights.anomaly_detection.enabled
booleantrueenable per-fingerprint latency recording and anomaly detectionServerless/Dedicated/Self-Hosted
sql.insights.anomaly_detection.latency_threshold
duration50msstatements must surpass this threshold to trigger anomaly detection and identificationServerless/Dedicated/Self-Hosted
sql.insights.anomaly_detection.memory_limit
byte size1.0 MiBthe maximum amount of memory allowed for tracking statement latenciesServerless/Dedicated/Self-Hosted
sql.insights.execution_insights_capacity
integer1000the size of the per-node store of execution insightsServerless/Dedicated/Self-Hosted
sql.insights.high_retry_count.threshold
integer10the number of retries a slow statement must have undergone for its high retry count to be highlighted as a potential problemServerless/Dedicated/Self-Hosted
sql.insights.latency_threshold
duration100msamount of time after which an executing statement is considered slow. Use 0 to disable.Serverless/Dedicated/Self-Hosted
sql.log.slow_query.experimental_full_table_scans.enabled
booleanfalsewhen 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.Serverless/Dedicated/Self-Hosted
sql.log.slow_query.internal_queries.enabled
booleanfalsewhen 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.Serverless/Dedicated/Self-Hosted
sql.log.slow_query.latency_threshold
duration0swhen set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each nodeServerless/Dedicated/Self-Hosted
sql.metrics.index_usage_stats.enabled
booleantruecollect per index usage statisticsServerless/Dedicated/Self-Hosted
sql.metrics.max_mem_reported_stmt_fingerprints
integer100000the maximum number of reported statement fingerprints stored in memoryServerless/Dedicated/Self-Hosted
sql.metrics.max_mem_reported_txn_fingerprints
integer100000the maximum number of reported transaction fingerprints stored in memoryServerless/Dedicated/Self-Hosted
sql.metrics.max_mem_stmt_fingerprints
integer100000the maximum number of statement fingerprints stored in memoryServerless/Dedicated/Self-Hosted
sql.metrics.max_mem_txn_fingerprints
integer100000the maximum number of transaction fingerprints stored in memoryServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.dump_to_logs
booleanfalsedump collected statement statistics to node logs when periodically clearedServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.enabled
booleantruecollect per-statement query statisticsServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.gateway_node.enabled
booleantruesave the gateway node for each statement fingerprint. If false, the value will be stored as 0.Serverless/Dedicated/Self-Hosted
sql.metrics.statement_details.index_recommendation_collection.enabled
booleantruegenerate an index recommendation for each fingerprint IDServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.max_mem_reported_idx_recommendations
integer5000the maximum number of reported index recommendation info stored in memoryServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.plan_collection.enabled
booleanfalseperiodically save a logical plan for each fingerprintServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.plan_collection.period
duration5m0sthe time until a new logical plan is collectedServerless/Dedicated/Self-Hosted
sql.metrics.statement_details.threshold
duration0sminimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected.Serverless/Dedicated/Self-Hosted
sql.metrics.transaction_details.enabled
booleantruecollect per-application transaction statisticsServerless/Dedicated/Self-Hosted
sql.multiple_modifications_of_table.enabled
booleanfalseif 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)Serverless/Dedicated/Self-Hosted
sql.multiregion.drop_primary_region.enabled
booleantrueallows dropping the PRIMARY REGION of a database if it is the last regionServerless/Dedicated/Self-Hosted
sql.notices.enabled
booleantrueenable notices in the server/client protocol being sentServerless/Dedicated/Self-Hosted
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled
booleanfalseif 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 probabilityServerless/Dedicated/Self-Hosted
sql.pgwire.multiple_active_portals.enabled
booleanfalseif true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution planServerless/Dedicated/Self-Hosted
sql.schema.telemetry.recurrence
string@weeklycron-tab recurrence for SQL schema telemetry jobServerless/Dedicated/Self-Hosted (read-only)
sql.show_ranges_deprecated_behavior.enabled
booleantrueif 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.Serverless/Dedicated/Self-Hosted
sql.spatial.experimental_box2d_comparison_operators.enabled
booleanfalseenables the use of certain experimental box2d comparison operatorsServerless/Dedicated/Self-Hosted
sql.stats.automatic_collection.enabled
booleantrueautomatic statistics collection modeServerless/Dedicated/Self-Hosted
sql.stats.automatic_collection.fraction_stale_rows
float0.2target fraction of stale rows per table that will trigger a statistics refreshServerless/Dedicated/Self-Hosted
sql.stats.automatic_collection.min_stale_rows
integer500target minimum number of stale rows per table that will trigger a statistics refreshServerless/Dedicated/Self-Hosted
sql.stats.cleanup.recurrence
string@hourlycron-tab recurrence for SQL Stats cleanup jobServerless/Dedicated/Self-Hosted
sql.stats.flush.enabled
booleantrueif set, SQL execution statistics are periodically flushed to diskServerless/Dedicated/Self-Hosted
sql.stats.flush.interval
duration10m0sthe interval at which SQL execution statistics are flushed to disk, this value must be less than or equal to 1 hourServerless/Dedicated/Self-Hosted
sql.stats.forecasts.enabled
booleantruewhen true, enables generation of statistics forecasts by default for all tablesServerless/Dedicated/Self-Hosted
sql.stats.histogram_collection.enabled
booleantruehistogram collection modeServerless/Dedicated/Self-Hosted
sql.stats.multi_column_collection.enabled
booleantruemulti-column statistics collection modeServerless/Dedicated/Self-Hosted
sql.stats.non_default_columns.min_retention_period
duration24h0m0sminimum retention period for table statistics collected on non-default columnsServerless/Dedicated/Self-Hosted
sql.stats.persisted_rows.max
integer1000000maximum number of rows of statement and transaction statistics that will be persisted in the system tablesServerless/Dedicated/Self-Hosted
sql.stats.post_events.enabled
booleanfalseif set, an event is logged for every CREATE STATISTICS jobServerless/Dedicated/Self-Hosted
sql.stats.response.max
integer20000the maximum number of statements and transaction stats returned in a CombinedStatements requestServerless/Dedicated/Self-Hosted
sql.stats.response.show_internal.enabled
booleanfalsecontrols 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 pagesServerless/Dedicated/Self-Hosted
sql.stats.system_tables.enabled
booleantruewhen true, enables use of statistics on system tables by the query optimizerServerless/Dedicated/Self-Hosted
sql.stats.system_tables_autostats.enabled
booleantruewhen true, enables automatic collection of statistics on system tablesServerless/Dedicated/Self-Hosted
sql.telemetry.query_sampling.enabled
booleanfalsewhen set to true, executed queries will emit an event on the telemetry logging channelServerless/Dedicated/Self-Hosted
sql.temp_object_cleaner.cleanup_interval
duration30m0show often to clean up orphaned temporary objectsServerless/Dedicated/Self-Hosted
sql.temp_object_cleaner.wait_interval
duration30m0show long after creation a temporary object will be cleaned upServerless/Dedicated/Self-Hosted
sql.trace.log_statement_execute
booleanfalseset to true to enable logging of executed statementsServerless/Dedicated/Self-Hosted
sql.trace.session_eventlog.enabled
booleanfalseset to true to enable session tracing; note that enabling this may have a negative performance impactServerless/Dedicated/Self-Hosted
sql.trace.stmt.enable_threshold
duration0senables 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_thresholdServerless/Dedicated/Self-Hosted
sql.trace.txn.enable_threshold
duration0senables 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)Serverless/Dedicated/Self-Hosted
sql.ttl.default_delete_batch_size
integer100default amount of rows to delete in a single query during a TTL jobServerless/Dedicated/Self-Hosted
sql.ttl.default_delete_rate_limit
integer0default delete rate limit for all TTL jobs. Use 0 to signify no rate limit.Serverless/Dedicated/Self-Hosted
sql.ttl.default_select_batch_size
integer500default amount of rows to select in a single query during a TTL jobServerless/Dedicated/Self-Hosted
sql.ttl.job.enabled
booleantruewhether the TTL job is enabledServerless/Dedicated/Self-Hosted
sql.txn_fingerprint_id_cache.capacity
integer100the maximum number of txn fingerprint IDs storedServerless/Dedicated/Self-Hosted
storage.value_blocks.enabled
booleantrueset to true to enable writing of value blocks in sstablesDedicated/Self-Hosted
timeseries.storage.enabled
booleantrueif set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhereServerless/Dedicated/Self-Hosted
timeseries.storage.resolution_10s.ttl
duration240h0m0sthe maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion.Serverless/Dedicated/Self-Hosted
timeseries.storage.resolution_30m.ttl
duration2160h0m0sthe maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion.Serverless/Dedicated/Self-Hosted
trace.debug.enable
booleanfalseif set, traces for recent requests can be seen at https://<ui>/debug/requestsServerless/Dedicated/Self-Hosted
trace.jaeger.agent
stringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.Serverless/Dedicated/Self-Hosted
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.Serverless/Dedicated/Self-Hosted
trace.snapshot.rate
duration0sif non-zero, interval at which background trace snapshots are capturedServerless/Dedicated/Self-Hosted
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracezServerless/Dedicated/Self-Hosted
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.Serverless/Dedicated/Self-Hosted
version
version1000023.1-4set the active cluster version in the format '<major>.<minor>'Serverless/Dedicated/Self-Hosted
diff --git a/pkg/cli/gen.go b/pkg/cli/gen.go index a895af8094a7..807d08d49687 100644 --- a/pkg/cli/gen.go +++ b/pkg/cli/gen.go @@ -28,7 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" "github.com/cockroachdb/errors/oserror" - "github.com/mozillazg/go-slugify" + slugify "github.com/mozillazg/go-slugify" "github.com/spf13/cobra" "github.com/spf13/cobra/doc" ) @@ -195,8 +195,11 @@ The resulting key file will be 32 bytes (random key ID) + key_size in bytes. }, } -var includeReservedSettings bool +var includeAllSettings bool var excludeSystemSettings bool +var showSettingClass bool +var classHeaderLabel string +var classLabels []string var genSettingsListCmd = &cobra.Command{ Use: "settings-list", @@ -234,8 +237,7 @@ Output the list of cluster settings known to this binary. continue } - if setting.Visibility() != settings.Public { - // We don't document non-public settings at this time. + if !includeAllSettings && setting.Visibility() != settings.Public { continue } @@ -269,11 +271,39 @@ This session variable default should now be configured using %s`, } row := []string{wrapDivSlug(name), typ, wrapCode(defaultVal), settingDesc} + if showSettingClass { + class := "unknown" + switch setting.Class() { + case settings.SystemOnly: + class = classLabels[0] + case settings.TenantReadOnly: + class = classLabels[1] + case settings.TenantWritable: + class = classLabels[2] + } + row = append(row, class) + } + if includeAllSettings { + if setting.Visibility() == settings.Public { + row = append(row, "public") + } else { + row = append(row, "reserved") + } + } rows = append(rows, row) } - sliceIter := clisqlexec.NewRowSliceIter(rows, "dddd") cols := []string{"Setting", "Type", "Default", "Description"} + align := "dddd" + if showSettingClass { + cols = append(cols, classHeaderLabel) + align += "d" + } + if includeAllSettings { + cols = append(cols, "Visibility") + align += "d" + } + sliceIter := clisqlexec.NewRowSliceIter(rows, align) return sqlExecCtx.PrintQueryOutput(os.Stdout, stderr, cols, sliceIter) }, } @@ -306,10 +336,19 @@ func init() { "AES key size for encryption at rest (one of: 128, 192, 256)") GenEncryptionKeyCmd.PersistentFlags().BoolVar(&overwriteKey, "overwrite", false, "Overwrite key if it exists") - genSettingsListCmd.PersistentFlags().BoolVar(&includeReservedSettings, "include-reserved", false, - "include undocumented 'reserved' settings") - genSettingsListCmd.PersistentFlags().BoolVar(&excludeSystemSettings, "without-system-only", false, + + f := genSettingsListCmd.PersistentFlags() + f.BoolVar(&includeAllSettings, "all-settings", false, + "include undocumented 'internal' settings") + f.BoolVar(&excludeSystemSettings, "without-system-only", false, "do not list settings only applicable to system tenant") + f.BoolVar(&showSettingClass, "show-class", false, + "show the setting class") + 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"}, + "label to use in the output for the various setting classes") genCmd.AddCommand(genCmds...) } diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 3004c1ee1f6b..e70a95c7b33c 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -327,6 +327,7 @@ var ReadableTypes = map[string]string{ "e": "enumeration", // This is named "m" (instead of "v") for backwards compatibility reasons. "m": "version", + "p": "protobuf", } // RedactedValue returns: