Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
110758: server,settings: properly cascade defaults for TenantReadOnly r=stevendanna,yuzefovich a=knz

Previous in sequence:
- [x] #110789 and #110947 
- [x] #110676
- [x] #111008
- [x] #111145
- [x] #111147
- [x]  #111149
- [x] #111150
- [x] #111153
- [x] #111475
- [x] #111210
- [x] #111212

Fixes #108677.
Fixes #85729.
Fixes #91825.
Completes the work described in the settings RFC.
Epic: CRDB-6671

TLDR: this patch ensures that virtual cluster servers observe changes
made to TenantReadOnly settings via SET CLUSTER SETTING in the system
interface, even when there is no override set via ALTER VIRTUAL
CLUSTER SET CLUSTER SETTING.

For example, after `SET CLUSTER SETTING
kv.closed_timestamp.target_duration = '10s'` in the system interface,
this value will show up via `SHOW CLUSTER SETTING` in a virtual
cluster SQL session.

This changes the way that settings are picked up in virtual cluster,
as follows:

1. if there is an override specifically for this tenant's ID
   (in `tenant_settings`), use that.
2. otherwise, if there is an override for the pseudo-ID 0
   (in `tenant_settings` still, set via `ALTER TENANT ALL SET CLUSTER
   SETTING`), then use that.
3. **NEW**  otherwise, if the class is TenantReadOnly and there
   is a custom value in `system.settings`, set via a regular
   `SET CLUSTER SETTING` in the system tenant, then use that.
4. otherwise, use the global default set via the setting's
   `Register()` call.

----

Prior to this patch, TenantReadOnly settings as observed from virtual
clusters were defined as the following priority order:

1. if there is an override specifically for this tenant's ID
   (in `tenant_settings`), use that.
2. otherwise, if there is an override for the pseudo-ID 0
   (in `tenant_settings` still, set via `ALTER TENANT ALL SET CLUSTER
   SETTING`), then use that.
3. otherwise, use the global default set via the setting's
   `Register()` call.

Remarkably, this did not pick up any changes made via a plain `SET
CLUSTER SETTING` statement via the system interface, which only
modifies this setting's value in `system.settings` (thus not
`tenant_settings`).

This situation was problematic in two ways.

To start, settings like `kv.closed_timestamp.target_duration` cannot
be set solely in `system.tenant_settings`; they are also used in the
storage layer and so must also be picked up from changes in
`system.settings`.

For these settings, it is common for operators to just issue the plain
`SET CLUSTER SETTING` statement (to update `system.settings`) and
simply forget to _also_ run `ALTER TENANT ALL SET CLUSTER SETTING`.

This mistake is nearly unavoidable and would result in incoherent
behavior, where the storage layer would use the customized value
and virtual clusters would use the registered global default.

The second problem is in mixed-version configurations, where
the storage layer runs version N+1 and the SQL service runs version N
of the executable. If the registered global default changes from
version N to N+1, the SQL service would not properly pick up
the new default defined in version N+1 of the storage layer.

This patch fixes both problems as follows:

- it integrates changes to TenantReadOnly settings observed
  in `system.settings`, to the watcher that tracks changes
  to `system.tenant_settings`. When a TenantReadOnly setting
  is present in the former but not the latter, a synthetic
  override is added.

- it also initializes synthetic overrides for all the
  TenantReadOnly settings upon server initialization,
  from the registered global default, so that virtual
  cluster servers always pick up the storage layer's
  default as override.


111383: *: simplify tests r=yuzefovich a=knz

All commits except the last are from #110758.
Epic: CRDB-6671

Now that "tenant-ro" settings take their default from the system tenant's value, we do not need `ALTER TENANT ALL` for them any more.

This patch simplifies test code accordingly.

Rcommended by `@yuzefovich` in the review for #110758. 

111440: cluster-ui: fix db page stories r=THardy98 a=THardy98

Epic: none

This change fixes the stories for the database pages.

Release note: None

111512: kv: correctly handle shared lock replays in KVNemesis r=nvanbenschoten a=arulajmani

Previously, we'd return an AssertionFailed error if a SKIP LOCKED request discovered another request from its own transaction waiting in a lock's wait queue. In SQL's use of KV, this can only happen if the SKIP LOCKED request is being replayed -- so returning an error here is fine. However, this tripped KVNemesis up.

This patch marks such errors, for the benefit of KVNemesis, and doesn't call them assertion failed errors.

Fixes #111426
Fixes #111506
Fixes #111513

Release note: None

Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
  • Loading branch information
4 people committed Sep 29, 2023
5 parents 8d45ee3 + 654bd80 + efdf0db + 1796236 + 0859e83 commit 1787e21
Show file tree
Hide file tree
Showing 35 changed files with 1,188 additions and 125 deletions.
2 changes: 1 addition & 1 deletion pkg/bench/foreachdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func benchmarkSepProcessTenantCockroach(b *testing.B, f BenchmarkFn) {
// The benchmarks sometime hit the default span limit, so we increase it.
// NOTE(andrei): Benchmarks drop the tables they're creating, so I'm not sure
// if hitting this limit is expected.
_, err := db.Exec(`ALTER TENANT ALL SET CLUSTER SETTING "spanconfig.virtual_cluster.max_spans" = 10000000`)
_, err := db.Exec(`SET CLUSTER SETTING spanconfig.virtual_cluster.max_spans = 10000000`)
require.NoError(b, err)

_, err = tenantDB.Exec(`CREATE DATABASE bench`)
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/alter_backup_schedule_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,6 @@ func newAlterSchedulesTestHelper(t *testing.T) (*alterSchedulesTestHelper, func(
th.sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.merge_file_buffer_size = '1MiB'`)
sysDB := sqlutils.MakeSQLRunner(s.SystemLayer().SQLConn(t, ""))
sysDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`) // speeds up test
sysDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)

return th, func() {
dirCleanupFn()
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5839,6 +5839,8 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

skip.UnderRace(t) // very slow test

ctx, cancel := context.WithCancel(context.Background())
defer cancel()

Expand Down Expand Up @@ -5890,7 +5892,6 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) {
// Speeds up the test.
systemTenantRunner.Exec(t, "SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms';")
systemTenantRunner.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'")
systemTenantRunner.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'")

// Run a full backup.
baseBackupURI := "userfile:///foo"
Expand Down
7 changes: 0 additions & 7 deletions pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3596,7 +3596,6 @@ func TestChangefeedFailOnTableOffline(t *testing.T) {
sqlDB := sqlutils.MakeSQLRunner(s.DB)
sysDB := sqlutils.MakeSQLRunner(s.SystemServer.SQLConn(t, ""))
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sqlDB.Exec(t, `CREATE TABLE for_import (a INT PRIMARY KEY, b INT)`)
defer sqlDB.Exec(t, `DROP TABLE for_import`)
sqlDB.Exec(t, `INSERT INTO for_import VALUES (0, NULL)`)
Expand Down Expand Up @@ -3762,7 +3761,6 @@ func TestChangefeedWorksOnRBRChange(t *testing.T) {
sqlDB := sqlutils.MakeSQLRunner(s.DB)
sysDB := sqlutils.MakeSQLRunner(s.SystemServer.SQLConn(t, ""))
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
t.Run("regional by row change works", func(t *testing.T) {
sqlDB.Exec(t, `CREATE TABLE rbr (a INT PRIMARY KEY, b INT)`)
defer sqlDB.Exec(t, `DROP TABLE rbr`)
Expand Down Expand Up @@ -3912,9 +3910,7 @@ func TestChangefeedStopOnSchemaChange(t *testing.T) {
sysDB.Exec(t, "SET CLUSTER SETTING changefeed.experimental_poll_interval = '200ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING changefeed.experimental_poll_interval = '200ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '50ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '50ms'")

Expand Down Expand Up @@ -4049,9 +4045,7 @@ func TestChangefeedNoBackfill(t *testing.T) {
sysDB.Exec(t, "SET CLUSTER SETTING changefeed.experimental_poll_interval = '200ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING changefeed.experimental_poll_interval = '200ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '10ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '10ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '10ms'")
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '10ms'")

Expand Down Expand Up @@ -4441,7 +4435,6 @@ func TestChangefeedMonitoring(t *testing.T) {
// Check that two changefeeds add correctly.
// Set cluster settings back so we don't interfere with schema changes.
sysDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s'`)
sysDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s'`)
fooCopy := feed(t, f, `CREATE CHANGEFEED FOR foo`)
_, _ = fooCopy.Next()
_, _ = fooCopy.Next()
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/changefeedccl/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,8 +381,6 @@ SET CLUSTER SETTING kv.rangefeed.enabled = true;
SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s';
SET CLUSTER SETTING changefeed.experimental_poll_interval = '10ms';
SET CLUSTER SETTING sql.defaults.vectorize=on;
ALTER TENANT ALL SET CLUSTER SETTING kv.rangefeed.enabled = true;
ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s';
ALTER TENANT ALL SET CLUSTER SETTING changefeed.experimental_poll_interval = '10ms';
ALTER TENANT ALL SET CLUSTER SETTING sql.defaults.vectorize=on;
CREATE DATABASE d;
Expand Down
3 changes: 0 additions & 3 deletions pkg/ccl/changefeedccl/protected_timestamps_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,6 @@ func TestChangefeedUpdateProtectedTimestamp(t *testing.T) {
sysDB := sqlutils.MakeSQLRunner(s.SystemServer.SQLConn(t, ""))
sysDB.Exec(t, "SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms'")
sysDB.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'") // speeds up the test
sysDB.Exec(t, "ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'")
sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY)`)
foo := feed(t, f, `CREATE CHANGEFEED FOR foo WITH resolved = '20ms'`)
defer closeFeed(t, foo)
Expand Down Expand Up @@ -237,7 +236,6 @@ func TestChangefeedProtectedTimestamps(t *testing.T) {
sysDB := sqlutils.MakeSQLRunner(s.SystemServer.SQLConn(t, ""))
sysDB.Exec(t, `SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms'`)
sysDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
sysDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
sqlDB.Exec(t, `ALTER RANGE default CONFIGURE ZONE USING gc.ttlseconds = 100`)
sqlDB.Exec(t, `ALTER RANGE system CONFIGURE ZONE USING gc.ttlseconds = 100`)
sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING)`)
Expand Down Expand Up @@ -406,7 +404,6 @@ func TestChangefeedCanceledWhenPTSIsOld(t *testing.T) {
sqlDB := sqlutils.MakeSQLRunner(s.DB)
sysDB := sqlutils.MakeSQLRunner(s.SystemServer.SQLConn(t, ""))
sysDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
sysDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
sqlDB.Exec(t, `SET CLUSTER SETTING jobs.metrics.interval.poll = '100ms'`) // speed up metrics poller
// Create the data table; it will only contain a
// single row with multiple versions.
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -964,8 +964,6 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) {
systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '0.1s'`)
systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '0.1s'`)
systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.propagation_slack = '0.1s'`)
systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '0.1s'`)
systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '0.1s'`)
systemSQL.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.propagation_slack = '0.1s'`)
// We're making assertions on traces collected by the tenant using log lines
// in KV so we must ensure they're not redacted.
Expand Down
3 changes: 0 additions & 3 deletions pkg/ccl/multitenantccl/tenantcostclient/tenant_side_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1311,9 +1311,6 @@ func BenchmarkExternalIOAccounting(b *testing.B) {
setRUAccountingMode := func(b *testing.B, mode string) {
_, err := hostSQL.Exec(fmt.Sprintf("SET CLUSTER SETTING tenant_cost_control.external_io.ru_accounting_mode = '%s'", mode))
require.NoError(b, err)

_, err = hostSQL.Exec(fmt.Sprintf("ALTER TENANT ALL SET CLUSTER SETTING tenant_cost_control.external_io.ru_accounting_mode = '%s'", mode))
require.NoError(b, err)
}

concurrently := func(b *testing.B, ctx context.Context, concurrency int, op func(context.Context) error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/serverccl/shared_process_tenant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func TestSharedProcessTenantNoSpanLimit(t *testing.T) {
_, err := db.Exec("CREATE TENANT hello; ALTER TENANT hello START SERVICE SHARED")
require.NoError(t, err)

_, err = db.Exec("ALTER TENANT ALL SET CLUSTER SETTING spanconfig.virtual_cluster.max_spans = 1000")
_, err = db.Exec("SET CLUSTER SETTING spanconfig.virtual_cluster.max_spans = 1000")
require.NoError(t, err)

var tenantDB *gosql.DB
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,6 @@ func TestDataDriven(t *testing.T) {
{
sysDB := sqlutils.MakeSQLRunner(tc.SystemLayer(0).SQLConn(t, ""))
sysDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '20ms'`)
sysDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '20ms'`)
}

spanConfigTestCluster := spanconfigtestcluster.NewHandle(t, tc, scKnobs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -302,9 +302,7 @@ func TestSQLWatcherMultiple(t *testing.T) {
tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0 /* idx */))
sdb := sqlutils.MakeSQLRunner(tc.SystemLayer(0).SQLConn(t, ""))
sdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`)
sdb.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.rangefeed.enabled = true`)
sdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
sdb.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)

noopCheckpointDuration := 100 * time.Millisecond
sqlWatcher := spanconfigsqlwatcher.New(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,11 +227,8 @@ func NewReplicationHelper(

// Speeds up the tests a bit.
`SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '200ms'`,
`ALTER TENANT ALL SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '200ms'`,
`SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'`,
`ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.target_duration = '50ms'`,
`SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '10ms'`,
`ALTER TENANT ALL SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '10ms'`,
`SET CLUSTER SETTING stream_replication.min_checkpoint_frequency = '10ms'`)

// Sink to read data from.
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvnemesis/applier.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,10 @@ func exceptSharedLockPromotionError(err error) bool { // true if lock promotion
return errors.Is(err, &concurrency.LockPromotionError{})
}

func exceptSkipLockedReplayError(err error) bool { // true if skip locked replay error
return errors.Is(err, &concurrency.SkipLockedReplayError{})
}

func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) {
switch o := op.GetValue().(type) {
case *GetOperation,
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvnemesis/validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -705,7 +705,10 @@ func (v *validator) processOp(op Operation) {
//
// So we ignore the results of failIfError, calling it only for its side
// effect of perhaps registering a failure with the validator.
v.failIfError(op, t.Result, exceptRollback, exceptAmbiguous, exceptSharedLockPromotionError)
v.failIfError(
op, t.Result,
exceptRollback, exceptAmbiguous, exceptSharedLockPromotionError, exceptSkipLockedReplayError,
)

ops := t.Ops
if t.CommitInBatch != nil {
Expand Down Expand Up @@ -1208,6 +1211,7 @@ func (v *validator) checkError(
exceptAmbiguous, exceptOmitted, exceptRetry,
exceptDelRangeUsingTombstoneStraddlesRangeBoundary,
exceptSharedLockPromotionError,
exceptSkipLockedReplayError,
}
sl = append(sl, extraExceptions...)
return v.failIfError(op, r, sl...)
Expand Down
37 changes: 34 additions & 3 deletions pkg/kv/kvserver/concurrency/lock_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -747,9 +747,22 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn(
break
}
if g.isSameTxn(qqg.guard.txnMeta()) {
return false, nil, errors.AssertionFailedf(
"SKIP LOCKED request should not find another waiting request from the same transaction",
)
// A SKIP LOCKED request should not find another waiting request from its
// own transaction, at least not in the way that SQL uses KV. The only way
// we can end up finding another request in the lock's wait queue from our
// own transaction is if we're a replay. We could handle this case by
// treating it as a non-conflict, but doing so expands the testing surface
// area -- we would want to include tests for:
// 1. Just our own request in the wait queue, treated as a non-conflict.
// 2. A request from a different transaction in the wait queue, with a
// lower sequence number, that conflicts.
// 3. A request from a different transaction in the wait queue, with a
// higher sequence number, that conflicts.
// 4. A request from a different transaction in the wait queue, with a
// lower sequence number, that does not conflict.
// For now, we simply return an error, and mark it for the benefit of
// KVNemesis.
return false, nil, MarkSkipLockedReplayError(errors.Errorf("SKIP LOCKED request should not find another waiting request from the same transaction"))
}
if lock.Conflicts(qqg.mode, makeLockMode(str, g.txnMeta(), g.ts), &g.lt.settings.SV) {
return true, nil, nil // the conflict isn't with a lock holder, nil is returned
Expand Down Expand Up @@ -4278,3 +4291,21 @@ func MarkLockPromotionError(cause error) error {
}
return errors.Mark(cause, &LockPromotionError{})
}

// SkipLockedReplayError is used to mark errors resulting from replayed SKIP
// LOCKED requests that discover other requests from their own transactions in
// a lock's wait queue. We mark such errors for the benefit of KVNemesis.
type SkipLockedReplayError struct{}

func (e *SkipLockedReplayError) Error() string {
return "skip locked replay error"
}

// MarkSkipLockedReplayError wraps the given error, if non-nil, as a skip locked
// replay error.
func MarkSkipLockedReplayError(cause error) error {
if cause == nil {
return nil
}
return errors.Mark(cause, &SkipLockedReplayError{})
}
2 changes: 2 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1112,6 +1112,8 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf
spanConfigKVAccessor: spanConfig.kvAccessorForTenantRecords,
kvStoresIterator: kvserver.MakeStoresIterator(node.stores),
inspectzServer: inspectzServer,

notifyChangeToTenantReadOnlySettings: tenantSettingsWatcher.SetAlternateDefaults,
},
SQLConfig: &cfg.SQLConfig,
BaseConfig: &cfg.BaseConfig,
Expand Down
12 changes: 10 additions & 2 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangestats"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
Expand Down Expand Up @@ -257,6 +258,13 @@ type sqlServerOptionalKVArgs struct {
// inspectzServer is used to power various crdb_internal vtables, exposing
// the equivalent of /inspectz but through SQL.
inspectzServer inspectzpb.InspectzServer

// notifyChangeToTenantReadOnlySettings is called by the settings
// watcher when one or more TenandReadOnly setting is updated via
// SET CLUSTER SETTING (i.e. updated in system.settings).
//
// The second argument must be sorted by setting key already.
notifyChangeToTenantReadOnlySettings func(context.Context, []kvpb.TenantSetting)
}

// sqlServerOptionalTenantArgs are the arguments supplied to newSQLServer which
Expand Down Expand Up @@ -568,8 +576,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {

var settingsWatcher *settingswatcher.SettingsWatcher
if codec.ForSystemTenant() {
settingsWatcher = settingswatcher.New(
cfg.clock, codec, cfg.Settings, cfg.rangeFeedFactory, cfg.stopper, cfg.settingsStorage,
settingsWatcher = settingswatcher.NewWithNotifier(ctx,
cfg.clock, codec, cfg.Settings, cfg.rangeFeedFactory, cfg.stopper, cfg.notifyChangeToTenantReadOnlySettings, cfg.settingsStorage,
)
} else {
// Create the tenant settings watcher, using the tenant connector as the
Expand Down
Loading

0 comments on commit 1787e21

Please sign in to comment.