Skip to content

Commit

Permalink
tenantsettingswatcher: extend override store with alternate defaults
Browse files Browse the repository at this point in the history
As an end-goal, we want that when there is no override in
`system.tenant_settings`, i.e. the rangefeed over this table has
nothing to stay, we pick up the current value of `TenantReadOnly`
settings from `system.settings` instead to feed to virtual clusters.

To achieve this, we need to merge the current `TenantReadOnly` values
"underneath" the tenant overrides, i.e. use the current
`TenantReadOnly` as default override value when the rangefeed over
`tenant_settings` has nothing to say.

This commit updates the overrides store accordingly. It does so by
maintaining an `alternateDefaults` override slice, updated from
changes to `TenantReadOnly` settings from `system.settings`.

Then, updates to the overrides map (in `tenantOverrides`) for the
special tenant ID `allTenantOverridesID` are "merged" from
`alternateDefaults` with the following rule:

- when a change to `alternateDefaults` is made, the current
  `allTenantOverridesID` override slice is updated for each
  setting *not* currently set from the rangefeed (those missing from the
  `explicitKeys` map).

- when an *addition* or *modification* is observed from the
  rangefeed (for tenant ID `allTenantOverridesID`), the overrides slice
  is updated with the rangefeed value, and the rangefeed setting key is
  added to `explicitKeys`.

- when a *deletion* is observed from the rangefeed, the key is removed
  from `explicitKeys` and any override from `alternateDefaults` is
  inserted instead.

Note that entries are never deleted from `alternateDefaults`, even
when a TenantReadOnly customization is removed from
`system.settings` (i.e. the callback `setAlternateDefaults` is given a
smaller slice than the current `alternateDefaults`). This is on
purpose: to properly support mixed-version deployments, we want
that SQL servers observe the storage layer's idea of what the default
value of a TenantReadOnly setting should be. To achieve this, it's
important that the storage server always reports an override for every
TenantReadOnly setting.

Release note: None
  • Loading branch information
knz committed Sep 29, 2023
1 parent d632d44 commit 7a163cd
Show file tree
Hide file tree
Showing 3 changed files with 460 additions and 19 deletions.
2 changes: 2 additions & 0 deletions pkg/server/tenantsettingswatcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/buildutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/startup",
Expand Down Expand Up @@ -58,6 +59,7 @@ go_test(
"//pkg/settings",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
Expand Down
287 changes: 276 additions & 11 deletions pkg/server/tenantsettingswatcher/overrides_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -45,6 +46,16 @@ type overridesStore struct {
// this ever becomes a problem, we can periodically purge entries with no
// overrides.
tenants map[roachpb.TenantID]*tenantOverrides

// alternateDefaults defines values to use for the
// allTenantOverrides settings when there is no override set via
// tenant_settings.
//
// The slice is sorted by InternalKey.
//
// At the time of this writing, this is used for TenantReadOnly
// settings.
alternateDefaults []kvpb.TenantSetting
}
}

Expand All @@ -54,26 +65,44 @@ type tenantOverrides struct {
// overrides, ordered by InternalKey.
overrides []kvpb.TenantSetting

// explicitKeys contains override keys that got their value from the
// rangefeed over system.tenant_settings, i.e. NOT those inherited
// from alternateDefaults.
//
// This map is used when an alternate default is changed
// asynchronously after the overrides have been loaded from the
// rangefeed already, to detect which overrides need to be updated
// from alternate defaults vs those that need to stay as-is (because
// they come from an override in .tenant_settings).
explicitKeys map[settings.InternalKey]struct{}

// changeCh is a channel that is closed when the tenant overrides change (in
// which case a new tenantOverrides object will contain the updated settings).
changeCh chan struct{}
}

func newTenantOverrides(
ctx context.Context, tenID roachpb.TenantID, overrides []kvpb.TenantSetting,
ctx context.Context,
tenID roachpb.TenantID,
overrides []kvpb.TenantSetting,
explicitKeys map[settings.InternalKey]struct{},
) *tenantOverrides {
if log.V(1) {
var buf redact.StringBuilder
buf.Printf("loaded overrides for tenant %d (%s)\n", tenID.InternalValue, util.GetSmallTrace(2))
for _, v := range overrides {
buf.Printf("%v = %+v", v.InternalKey, v.Value)
if _, exists := explicitKeys[v.InternalKey]; exists {
buf.Printf(" (explicit)")
}
buf.SafeRune('\n')
}
log.VEventf(ctx, 1, "%v", buf)
}
return &tenantOverrides{
overrides: overrides,
changeCh: make(chan struct{}),
overrides: overrides,
explicitKeys: explicitKeys,
changeCh: make(chan struct{}),
}
}

Expand Down Expand Up @@ -106,13 +135,31 @@ func (s *overridesStore) setAll(
sort.Slice(overrides, func(i, j int) bool {
return overrides[i].InternalKey < overrides[j].InternalKey
})
// Sanity check.
for i := 1; i < len(overrides); i++ {
if overrides[i].InternalKey == overrides[i-1].InternalKey {
panic(errors.AssertionFailedf("duplicate setting: %s", overrides[i].InternalKey))
}

providedKeys := make(map[settings.InternalKey]struct{}, len(overrides))
for _, v := range overrides {
providedKeys[v.InternalKey] = struct{}{}
}
// If we are setting the all-tenant overrides, ensure there is a
// pseudo-override for every TenantReadOnly setting with an
// alternate default.
if tenantID == allTenantOverridesID && len(s.mu.alternateDefaults) > 0 {
// We can set copyOverrides==false because we took ownership
// over the incoming allOverrides map.
overrides = spliceOverrideDefaults(providedKeys, overrides, s.mu.alternateDefaults, false /* copyOverrides */)
}
s.mu.tenants[tenantID] = newTenantOverrides(ctx, tenantID, overrides, providedKeys)
}
}

func checkSortedByKey(a []kvpb.TenantSetting) {
if !buildutil.CrdbTestBuild {
return
}
for i := 1; i < len(a); i++ {
if a[i].InternalKey <= a[i-1].InternalKey {
panic(errors.AssertionFailedf("duplicate setting: %s", a[i].InternalKey))
}
s.mu.tenants[tenantID] = newTenantOverrides(ctx, tenantID, overrides)
}
}

Expand Down Expand Up @@ -140,7 +187,13 @@ func (s *overridesStore) getTenantOverrides(
if res, ok = s.mu.tenants[tenantID]; ok {
return res
}
res = newTenantOverrides(ctx, tenantID, nil /* overrides */)
var overrides []kvpb.TenantSetting
if tenantID == allTenantOverridesID {
// Inherit alternate defaults.
overrides = make([]kvpb.TenantSetting, len(s.mu.alternateDefaults))
copy(overrides, s.mu.alternateDefaults)
}
res = newTenantOverrides(ctx, tenantID, overrides, nil /* explicitKeys */)
s.mu.tenants[tenantID] = res
return res
}
Expand All @@ -154,10 +207,15 @@ func (s *overridesStore) setTenantOverride(
s.mu.Lock()
defer s.mu.Unlock()
var before []kvpb.TenantSetting
var providedKeys map[settings.InternalKey]struct{}
if existing, ok := s.mu.tenants[tenantID]; ok {
before = existing.overrides
providedKeys = existing.explicitKeys
close(existing.changeCh)
}
if providedKeys == nil {
providedKeys = make(map[settings.InternalKey]struct{})
}
after := make([]kvpb.TenantSetting, 0, len(before)+1)
// 1. Add all settings up to setting.InternalKey.
for len(before) > 0 && before[0].InternalKey < setting.InternalKey {
Expand All @@ -167,12 +225,219 @@ func (s *overridesStore) setTenantOverride(
// 2. Add the after setting, unless we are removing it.
if setting.Value != (settings.EncodedValue{}) {
after = append(after, setting)
providedKeys[setting.InternalKey] = struct{}{}
} else {
// The override is being removed. If we have an alternate default,
// use it instead.
delete(providedKeys, setting.InternalKey)
if tenantID == allTenantOverridesID {
if defaultVal, ok := findAlternateDefault(setting.InternalKey, s.mu.alternateDefaults); ok {
after = append(after, kvpb.TenantSetting{
InternalKey: setting.InternalKey,
Value: defaultVal,
})
}
}
}
// Skip any existing setting for this setting key.
if len(before) > 0 && before[0].InternalKey == setting.InternalKey {
before = before[1:]
}
// 3. Append all settings after setting.InternalKey.
after = append(after, before...)
s.mu.tenants[tenantID] = newTenantOverrides(ctx, tenantID, after)

// Sanity check.
checkSortedByKey(after)

s.mu.tenants[tenantID] = newTenantOverrides(ctx, tenantID, after, providedKeys)
}

// setAlternateDefaults defines alternate defaults, to use
// when there is no stored default in .tenant_settings.
//
// At the time of this writing, this is called when a change is made
// to a TenantReadOnly setting in the system tenant's system.settings
// table. Values set this way serve as default value if there is no
// override in system.tenant_settings.
//
// The input slice must be sorted by key already.
func (s *overridesStore) setAlternateDefaults(
ctx context.Context, alternateDefaultSlice []kvpb.TenantSetting,
) {
// Sanity check.
checkSortedByKey(alternateDefaultSlice)

s.mu.Lock()
defer s.mu.Unlock()

alternateDefaults := s.mu.alternateDefaults
alternateDefaults = updateDefaults(alternateDefaults, alternateDefaultSlice)
s.mu.alternateDefaults = alternateDefaults

// Inject the new read-only default into the all-tenants
// override map.
var overrides []kvpb.TenantSetting
var explicitKeys map[settings.InternalKey]struct{}
copyOverrides := false
if existing, ok := s.mu.tenants[allTenantOverridesID]; ok {
explicitKeys = existing.explicitKeys
overrides = existing.overrides
// Need to pass copyOverrides==true because we don't want to
// modify the overrides slice in-place -- there may be listeners
// that are consuming the slice asynchronously.
copyOverrides = true
close(existing.changeCh)
}

overrides = spliceOverrideDefaults(explicitKeys, overrides, alternateDefaults, copyOverrides)
s.mu.tenants[allTenantOverridesID] = newTenantOverrides(ctx, allTenantOverridesID, overrides, explicitKeys)
}

// findAlternateDefault searches the value associated with the given key
// in the alternate default slice, which is assumed to be sorted.
func findAlternateDefault(
key settings.InternalKey, defaults []kvpb.TenantSetting,
) (val settings.EncodedValue, found bool) {
idx := sort.Search(len(defaults), func(i int) bool {
return defaults[i].InternalKey >= key
})
if idx >= len(defaults) {
return val, false
}
if defaults[idx].InternalKey != key {
return val, false
}
return defaults[idx].Value, true
}

// spliceOverrideDefaults adds overrides into the 'overrides' slice
// for each key that doesn't have an override yet but is present in
// alternateDefaults.
//
// If the key already has an override, but wasn't set explicitly via
// the rangefeed from tenant_settings (as informed by explicitKeys),
// this means the override was already a fallback to the alternate
// default before; in which case we update it to the new value of the
// alternate default.
//
// The alternateDefaults slice must be sorted already.
//
// If the copyOverrides flag is set, the overrides slice is modified
// in-place. If false, it is copied first. This is necessary when
// updating the overrides slice that was already stored in the tenants
// map before (it can be accessed concurrently).
func spliceOverrideDefaults(
explicitKeys map[settings.InternalKey]struct{},
overrides []kvpb.TenantSetting,
alternateDefaults []kvpb.TenantSetting,
copyOverrides bool,
) []kvpb.TenantSetting {
if copyOverrides && len(overrides) > 0 {
dst := make([]kvpb.TenantSetting, len(overrides))
copy(dst, overrides)
overrides = dst
}
aIter, bIter := 0, 0
for aIter < len(overrides) && bIter < len(alternateDefaults) {
if overrides[aIter].InternalKey == alternateDefaults[bIter].InternalKey {
if _, ok := explicitKeys[overrides[aIter].InternalKey]; !ok {
// The key is not explicitly set (via the rangefeed), this means
// that we were relying on the default to start with.
// Update the override from the new value of the default.
overrides[aIter] = alternateDefaults[bIter]
}
aIter++
bIter++
} else if overrides[aIter].InternalKey < alternateDefaults[bIter].InternalKey {
if _, ok := explicitKeys[overrides[aIter].InternalKey]; !ok {
// The key is not explicitly set (via the rangefeed), this
// means that we were relying on the default to start with.
// But now, there is no default any more. Remove the override.
copy(overrides[aIter:], overrides[aIter+1:])
overrides = overrides[:len(overrides)-1]
} else {
aIter++
}
} else {
// The following is an optimization, to append as many missing elements
// from the alternateDefaults slice as possible in one go.
// This can be implemented also (albeit less efficiently) as:
// tail := overrides[aIter:]
// dst = append(overrides[:aIter:aIter], alternateDefaults[bIter])
// overrides = append(overrides, tail...)
// aIter++
// bIter++
// continue
numOverrides := 0
for ; (bIter+numOverrides) < len(alternateDefaults) &&
overrides[aIter].InternalKey > alternateDefaults[bIter+numOverrides].InternalKey; numOverrides++ {
}
tail := overrides[aIter:]
overrides = append(overrides[:aIter:aIter], alternateDefaults[bIter:bIter+numOverrides]...)
overrides = append(overrides, tail...)
aIter += numOverrides
bIter += numOverrides
}
}
for aIter < len(overrides) {
if _, ok := explicitKeys[overrides[aIter].InternalKey]; !ok {
// The key is not explicitly set (via the rangefeed), this
// means that we were relying on the default to start with.
// But now, there is no default any more. Remove the override.
copy(overrides[aIter:], overrides[aIter+1:])
overrides = overrides[:len(overrides)-1]
} else {
aIter++
}
}
if bIter < len(alternateDefaults) {
overrides = append(overrides, alternateDefaults[bIter:]...)
}
// Sanity check.
checkSortedByKey(overrides)
return overrides
}

// updateDefaults extends the slice in the first argument with the
// elements in the second argument. If the same setting key is present
// in both, the first slice is updated from the second.
//
// NB: the dst slice is modified in-place.
func updateDefaults(dst, src []kvpb.TenantSetting) []kvpb.TenantSetting {
aIter, bIter := 0, 0
for aIter < len(dst) && bIter < len(src) {
if dst[aIter].InternalKey == src[bIter].InternalKey {
dst[aIter] = src[bIter]
aIter++
bIter++
} else if dst[aIter].InternalKey < src[bIter].InternalKey {
aIter++
} else {
// The following is an optimization, to append as many missing elements
// from the src slice as possible in one go.
// This can be implemented also (albeit less efficiently) as:
// tail := dst[aIter:]
// dst = append(dst[:aIter:aIter], src[bIter])
// dst = append(dst, tail...)
// aIter++
// bIter++
// continue
numDst := 0
for ; (bIter+numDst) < len(src) &&
dst[aIter].InternalKey > src[bIter+numDst].InternalKey; numDst++ {
}
tail := dst[aIter:]
dst = append(dst[:aIter:aIter], src[bIter:bIter+numDst]...)
dst = append(dst, tail...)
aIter += numDst
bIter += numDst
}
}
if bIter < len(src) {
dst = append(dst, src[bIter:]...)
}

// Sanity check.
checkSortedByKey(dst)
return dst
}
Loading

0 comments on commit 7a163cd

Please sign in to comment.