Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98353: kvserver: make some cluster settings system only r=andrewbaptist a=kvoli

Update cluster settings in the `kv/kvserver` pkg to be `SystemOnly`.
Previously, there were many cluster settings which which were
`TenantWritable`  or `TenantReadOnly`. These settings, even if altered
by a tenant have no effect.

There are settings which are not updated, due due to tests relying on
modifying the setting value using a non-system tenant. We ignore these
in this commit and defer to #98723 for handling these.

These settings are updated to be `SystemOnly`:

```
kv.bulk_io_write.max_rate
kv.bulk_sst.max_allowed_overage
kv.bulk_sst.target_size
kv.closed_timestamp.follower_reads_enabled
kv.log_range_and_node_events.enabled
kv.range_split.by_load_enabled
kv.range_split.load_cpu_threshold
kv.range_split.load_qps_threshold
kv.replica_stats.addsst_request_size_factor
kv.replication_reports.interval
server.shutdown.lease_transfer_wait
```

Resolves: #98347

Release note (ops change): Some KV server cluster settings are now system
only. These settings could previously be written or read by
tenants, however writing to these settings had no effect.

99037: kvserver: skip `TestStoreRangeSplitRaceUninitializedRHS` under race/deadlock r=erikgrinaker a=erikgrinaker

The Raft groups are unable to maintain quorum when stressed under race/deadlock.

Resolves #98840.

Epic: none
Release note: None

99052: sql: add `switchToAnotherPortal` signal for result consumer r=ZhouXing19 a=ZhouXing19

This PR is part of the effort to implement the multiple active portals. (Extracted from #96358)

---

### sql/settings: add sql.pgwire.multiple_active_portals.enabled cluster setting


This commit is to add a non-public `sql.pgwire.multiple_active_portals.enabled`
setting. This setting is only for a PREVIEWABLE feature.
With it set to true, all non-internal portals with read-only queries without sub/post
queries can be paused and resumed in an interleaving manner but are executed with
local plan.

---
### sql: add switchToAnotherPortal signal for result consumer

Previously, after pushing the result to the consumer, we only accept the
following actions as the next step:
1. Pushing more data from the same source to the same consumer;
2. Drain or close the pipeline.

This commit is to add another option: pause the current execution, and switch
to the execution to another portal. I.e. when receiving an `ExecPortal` cmd but
for another portal, we do nothing and return the control to the connExecutor.
This allows us to execute different portals interleaving-ly.

Epic: CRDB-17622

Release note (sql change): add a non-public `sql.pgwire.multiple_active_portals.enabled` setting. This setting is only for a PREVIEWABLE feature. With it set to true, all non-internal portals with read-only queries without sub/post queries can be paused and resumed in an interleaving manner but are executed with local plan.


99062: sql: deflake `TestTrace` r=yuzefovich a=erikgrinaker

This has been seen to flake in CI:

```
=== RUN   TestTrace/ShowTraceForVectorized/TracingOff/node-1
    trace_test.go:386: expected span: "session recording", got: "pendingLeaseRequest: requesting lease"
    trace_test.go:397: remaining span: "session recording"
    trace_test.go:397: remaining span: "sql query"
    trace_test.go:397: remaining span: "sql txn"
    trace_test.go:397: remaining span: "txn coordinator send"
            --- FAIL: TestTrace/ShowTraceForVectorized/TracingOff/node-1 (0.02s)
```

There was already an exception for this span, but with a `storage.` prefix. This patch removes the prefix, and makes it match on substrings.

This flake has possibly been made worse with the introduction of a metamorphic setting to only use expiration-based leases in ecc931b.

Resolves #98971.

Epic: none
Release note: None

Co-authored-by: Austen McClernon <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Jane Xing <[email protected]>
  • Loading branch information
4 people committed Mar 20, 2023
5 parents 3427f11 + 7503b1a + a491e10 + cdaff50 + c7641af commit 3f960e1
Show file tree
Hide file tree
Showing 38 changed files with 202 additions and 83 deletions.
12 changes: 1 addition & 11 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -36,19 +36,9 @@ feature.restore.enabled boolean true set to true to enable restore, false to dis
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.bulk_io_write.max_rate byte size 1.0 TiB the rate limit (bytes/sec) to use for writes to disk on behalf of bulk io ops
kv.bulk_sst.max_allowed_overage byte size 64 MiB if 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 size 16 MiB target size for SSTs emitted from export requests; export requests (i.e. BACKUP) may buffer up to the sum of kv.bulk_sst.target_size and kv.bulk_sst.max_allowed_overage in memory
kv.closed_timestamp.follower_reads_enabled boolean true allow (all) replicas to serve consistent historical reads based on closed timestamp information
kv.log_range_and_node_events.enabled boolean true set 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 duration 5m0s the frequency for reconciling jobs with protected timestamp records
kv.range_split.by_load_enabled boolean true allow automatic splits of ranges based on where load is concentrated
kv.range_split.load_cpu_threshold duration 500ms the CPU use per second over which, the range becomes a candidate for load based splitting
kv.range_split.load_qps_threshold integer 2500 the QPS over which, the range becomes a candidate for load based splitting
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.replica_stats.addsst_request_size_factor integer 50000 the 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 duration 1m0s the frequency for generating the replication_constraint_stats, replication_stats_report and replication_critical_localities reports (set to 0 to disable)
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
Expand Down Expand Up @@ -82,7 +72,6 @@ server.oidc_authentication.scopes string openid sets OIDC scopes to include with
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.lease_transfer_wait duration 5s the timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
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
Expand Down Expand Up @@ -260,6 +249,7 @@ sql.multiple_modifications_of_table.enabled boolean false if true, allow stateme
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.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
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@
<tr><td><div id="setting-sql-multiregion-drop-primary-region-enabled" class="anchored"><code>sql.multiregion.drop_primary_region.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>allows dropping the PRIMARY REGION of a database if it is the last region</td></tr>
<tr><td><div id="setting-sql-notices-enabled" class="anchored"><code>sql.notices.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td></tr>
<tr><td><div id="setting-sql-optimizer-uniqueness-checks-for-gen-random-uuid-enabled" class="anchored"><code>sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>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</td></tr>
<tr><td><div id="setting-sql-pgwire-multiple-active-portals-enabled" class="anchored"><code>sql.pgwire.multiple_active_portals.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan</td></tr>
<tr><td><div id="setting-sql-schema-telemetry-recurrence" class="anchored"><code>sql.schema.telemetry.recurrence</code></div></td><td>string</td><td><code>@weekly</code></td><td>cron-tab recurrence for SQL schema telemetry job</td></tr>
<tr><td><div id="setting-sql-spatial-experimental-box2d-comparison-operators-enabled" class="anchored"><code>sql.spatial.experimental_box2d_comparison_operators.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables the use of certain experimental box2d comparison operators</td></tr>
<tr><td><div id="setting-sql-stats-automatic-collection-enabled" class="anchored"><code>sql.stats.automatic_collection.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>automatic statistics collection mode</td></tr>
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/allocator/storepool/store_pool.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ const (
// replicate queue will not consider stores which have failed a reservation a
// viable target.
var FailedReservationsTimeout = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"server.failed_reservation_timeout",
"the amount of time to consider the store throttled for up-replication after a failed reservation call",
5*time.Second,
Expand All @@ -59,7 +59,7 @@ const timeAfterStoreSuspectSettingName = "server.time_after_store_suspect"
// TimeAfterStoreSuspect measures how long we consider a store suspect since
// it's last failure.
var TimeAfterStoreSuspect = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
timeAfterStoreSuspectSettingName,
"the amount of time we consider a store suspect for after it fails a node liveness heartbeat."+
" A suspect node would not receive any new replicas or lease transfers, but will keep the replicas it has.",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ const SSTTargetSizeSetting = "kv.bulk_sst.target_size"
// ExportRequestTargetFileSize controls the target file size for SSTs created
// during backups.
var ExportRequestTargetFileSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
settings.SystemOnly,
SSTTargetSizeSetting,
fmt.Sprintf("target size for SSTs emitted from export requests; "+
"export requests (i.e. BACKUP) may buffer up to the sum of %s and %s in memory",
Expand All @@ -55,7 +55,7 @@ const MaxExportOverageSetting = "kv.bulk_sst.max_allowed_overage"
// and an SST would exceed this size (due to large rows or large numbers of
// versions), then the export will fail.
var ExportRequestMaxAllowedFileSizeOverage = settings.RegisterByteSizeSetting(
settings.TenantWritable,
settings.SystemOnly,
MaxExportOverageSetting,
fmt.Sprintf("if 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 %s and %s in memory",
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
// QueryResolvedTimestampIntentCleanupAge configures the minimum intent age that
// QueryResolvedTimestamp requests will consider for async intent cleanup.
var QueryResolvedTimestampIntentCleanupAge = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.query_resolved_timestamp.intent_cleanup_age",
"minimum intent age that QueryResolvedTimestamp requests will consider for async intent cleanup",
10*time.Second,
Expand Down
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/ts"
Expand Down Expand Up @@ -2027,6 +2028,11 @@ func TestStoreRangeSplitRaceUninitializedRHS(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// The aggressive Raft timeouts in this test prevents it from maintaining
// quorum and making progress when stressing under race or deadlock detection.
skip.UnderRace(t)
skip.UnderDeadlock(t)

currentTrigger := make(chan *roachpb.SplitTrigger, 1)
var seen struct {
syncutil.Mutex
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/closedts/setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ var SideTransportCloseInterval = settings.RegisterDurationSetting(
// (see TargetForPolicy), if it is set to a non-zero value. Meant as an escape
// hatch.
var LeadForGlobalReadsOverride = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.closed_timestamp.lead_for_global_reads_override",
"if nonzero, overrides the lead time that global_read ranges use to publish closed timestamps",
0,
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/concurrency/concurrency_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ import (
// utilization and runaway queuing for misbehaving clients, a role it is well
// positioned to serve.
var MaxLockWaitQueueLength = settings.RegisterIntSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.lock_table.maximum_lock_wait_queue_length",
"the maximum length of a lock wait-queue that read-write requests are willing "+
"to enter and wait in. The setting can be used to ensure some level of quality-of-service "+
Expand Down Expand Up @@ -93,7 +93,7 @@ var MaxLockWaitQueueLength = settings.RegisterIntSetting(
// discoveredCount > 100,000, caused by stats collection, where we definitely
// want to avoid adding these locks to the lock table, if possible.
var DiscoveredLocksThresholdToConsultFinalizedTxnCache = settings.RegisterIntSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.lock_table.discovered_locks_threshold_for_consulting_finalized_txn_cache",
"the maximum number of discovered locks by a waiter, above which the finalized txn cache"+
"is consulted and resolvable locks are not added to the lock table -- this should be a small"+
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/concurrency/lock_table_waiter.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ import (
// LockTableLivenessPushDelay sets the delay before pushing in order to detect
// coordinator failures of conflicting transactions.
var LockTableLivenessPushDelay = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.lock_table.coordinator_liveness_push_delay",
"the delay before pushing in order to detect coordinator failures of conflicting transactions",
// This is set to a short duration to ensure that we quickly detect failed
Expand Down Expand Up @@ -71,7 +71,7 @@ var LockTableLivenessPushDelay = settings.RegisterDurationSetting(
// LockTableDeadlockDetectionPushDelay sets the delay before pushing in order to
// detect dependency cycles between transactions.
var LockTableDeadlockDetectionPushDelay = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.lock_table.deadlock_detection_push_delay",
"the delay before pushing in order to detect dependency cycles between transactions",
// This is set to a medium duration to ensure that deadlock caused by
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/gc/gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ const (
// IntentAgeThreshold is the threshold after which an extant intent
// will be resolved.
var IntentAgeThreshold = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.gc.intent_age_threshold",
"intents older than this threshold will be resolved when encountered by the MVCC GC queue",
2*time.Hour,
Expand Down Expand Up @@ -106,7 +106,7 @@ var TxnCleanupThreshold = settings.RegisterDurationSetting(
// of writing. This value is subject to tuning in real environment as we have
// more data available.
var MaxIntentsPerCleanupBatch = settings.RegisterIntSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.gc.intent_cleanup_batch_size",
"if non zero, gc will split found intents into batches of this size when trying to resolve them",
5000,
Expand All @@ -125,7 +125,7 @@ var MaxIntentsPerCleanupBatch = settings.RegisterIntSetting(
// The default value is a conservative limit to prevent pending intent key sizes
// from ballooning.
var MaxIntentKeyBytesPerCleanupBatch = settings.RegisterIntSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.gc.intent_cleanup_batch_byte_size",
"if non zero, gc will split found intents into batches of this size when trying to resolve them",
1e6,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvserverbase/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ func IntersectSpan(

// SplitByLoadMergeDelay wraps "kv.range_split.by_load_merge_delay".
var SplitByLoadMergeDelay = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.range_split.by_load_merge_delay",
"the delay that range splits created due to load will wait before considering being merged away",
5*time.Minute,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvserverbase/syncing_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func LimitBulkIOWrite(ctx context.Context, limiter *rate.Limiter, cost int) erro

// sstWriteSyncRate wraps "kv.bulk_sst.sync_size". 0 disables syncing.
var sstWriteSyncRate = settings.RegisterByteSizeSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.bulk_sst.sync_size",
"threshold after which non-Rocks SST writes must fsync (0 disables)",
BulkIOWriteBurst,
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/logstore/logstore.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import (
)

var disableSyncRaftLog = settings.RegisterBoolSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.raft_log.disable_synchronization_unsafe",
"set to true to disable synchronization on Raft log writes to persistent storage. "+
"Setting to true risks data loss or data corruption on server crashes. "+
Expand All @@ -47,7 +47,7 @@ var disableSyncRaftLog = settings.RegisterBoolSetting(
)

var enableNonBlockingRaftLogSync = settings.RegisterBoolSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.raft_log.non_blocking_synchronization.enabled",
"set to true to enable non-blocking synchronization on Raft log writes to "+
"persistent storage. Setting to true does not risk data loss or data corruption "+
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/protectedts/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ var MaxBytes = settings.RegisterIntSetting(
// MaxSpans controls the maximum number of spans which can be protected
// by all protected timestamp records.
var MaxSpans = settings.RegisterIntSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.protectedts.max_spans",
"if non-zero the limit of the number of spans which can be protected",
32768,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ const (

// targetRaftOutgoingBatchSize wraps "kv.raft.command.target_batch_size".
var targetRaftOutgoingBatchSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.raft.command.target_batch_size",
"size of a batch of raft commands after which it will be sent without further batching",
64<<20, // 64 MB
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_backpressure.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ var backpressureLogLimiter = log.Every(500 * time.Millisecond)
// range's size must grow to before backpressure will be applied on writes. Set
// to 0 to disable backpressure altogether.
var backpressureRangeSizeMultiplier = settings.RegisterFloatSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.range.backpressure_range_size_multiplier",
"multiple of range_max_bytes that a range is allowed to grow to without "+
"splitting before writes to that range are blocked, or 0 to disable",
Expand Down Expand Up @@ -66,7 +66,7 @@ var backpressureRangeSizeMultiplier = settings.RegisterFloatSetting(
// currently backpressuring than ranges which are larger but are not
// applying backpressure.
var backpressureByteTolerance = settings.RegisterByteSizeSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.range.backpressure_byte_tolerance",
"defines the number of bytes above the product of "+
"backpressure_range_size_multiplier and the range_max_size at which "+
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_follower_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
// information is collected and passed around, regardless of the value of this
// setting.
var FollowerReadsEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.closed_timestamp.follower_reads_enabled",
"allow (all) replicas to serve consistent historical reads based on closed timestamp information",
true,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ var RangefeedEnabled = settings.RegisterBoolSetting(
// RangeFeedRefreshInterval controls the frequency with which we deliver closed
// timestamp updates to rangefeeds.
var RangeFeedRefreshInterval = settings.RegisterDurationSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.rangefeed.closed_timestamp_refresh_interval",
"the interval at which closed-timestamp updates"+
"are delivered to rangefeeds; set to 0 to use kv.closed_timestamp.side_transport_interval",
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import (
)

var optimisticEvalLimitedScans = settings.RegisterBoolSetting(
settings.TenantWritable,
settings.SystemOnly,
"kv.concurrency.optimistic_eval_limited_scans.enabled",
"when true, limited scans are optimistically evaluated in the sense of not checking for "+
"conflicting latches or locks up front for the full key range of the scan, and instead "+
Expand Down
Loading

0 comments on commit 3f960e1

Please sign in to comment.