Skip to content

Commit

Permalink
spanconfig: integrate SpanConfigBounds with the Store and KVSubscriber
Browse files Browse the repository at this point in the history
This patch integrates `SpanConfigBounds` with the `KVSubscriber` and
`spanconfig.Store`. The `spanconfig.Store` used by the `KVSubscriber`
now has a handle to the global tenant capability state. It uses this to
clamp any secondary tenant span configs that are not in conformance
before returning them.

By default, clamping of secondary tenant span configurations is turned
off. It can be enabled using the `spanconfig.bounds.enabled` cluster
setting. The setting is hidden.

Fixes: #99689

Release note: None
  • Loading branch information
arulajmani committed Mar 27, 2023
1 parent 72d207b commit 0664f06
Show file tree
Hide file tree
Showing 22 changed files with 329 additions and 44 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,7 @@ go_test(
"//pkg/kv/kvserver/tscache",
"//pkg/kv/kvserver/txnwait",
"//pkg/kv/kvserver/uncertainty",
"//pkg/multitenant/tenantcapabilities",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer",
"//pkg/roachpb",
"//pkg/rpc",
Expand Down
6 changes: 5 additions & 1 deletion pkg/kv/kvserver/client_spanconfigs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand All @@ -41,6 +42,7 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) {
spanConfigStore := spanconfigstore.New(
roachpb.TestingDefaultSpanConfig(),
cluster.MakeTestingClusterSettings(),
tenantcapabilities.NewEmptyReader(),
nil,
)
var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
Expand Down Expand Up @@ -107,7 +109,9 @@ func TestFallbackSpanConfigOverride(t *testing.T) {
defer leaktest.AfterTest(t)()

st := cluster.MakeTestingClusterSettings()
spanConfigStore := spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, nil)
spanConfigStore := spanconfigstore.New(
roachpb.TestingDefaultSpanConfig(), st, tenantcapabilities.NewEmptyReader(), nil,
)
var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore)

Expand Down
18 changes: 18 additions & 0 deletions pkg/multitenant/tenantcapabilities/capabilities.go
Original file line number Diff line number Diff line change
Expand Up @@ -249,3 +249,21 @@ func (c CapabilityID) CapabilityType() Type {
panic(errors.AssertionFailedf("missing case: %q", c))
}
}

// NewEmptyReader returns a new Reader which corresponds to an empty global
// capability state.
func NewEmptyReader() Reader {
return emptyReader(true)
}

type emptyReader bool

// GetCapabilities implements the Reader interface.
func (emptyReader) GetCapabilities(roachpb.TenantID) (TenantCapabilities, bool) {
return nil, false
}

// GetGlobalCapabilityState implements the Reader interface.
func (emptyReader) GetGlobalCapabilityState() map[roachpb.TenantID]TenantCapabilities {
return nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,9 +29,15 @@ func (b boolCapValue) SafeFormat(p redact.SafePrinter, verb rune) {
p.Print(redact.Safe(bool(b)))
}

// Unwrap implements the tenantcapabilities.Value interface.
func (b boolCapValue) Unwrap() interface{} { return bool(b) }

// Unwrap implements the tenantcapabilities.Value interface.
func (m *SpanConfigBounds) Unwrap() interface{} { return m }

func (m *SpanConfigBounds) SafeFormat(p redact.SafePrinter, verb rune) {
p.Print(redact.SafeString(m.String()))
}

// boolCap is an accessor struct for boolean capabilities.
type boolCap struct {
cap *bool
Expand Down Expand Up @@ -73,6 +79,26 @@ func (i invertedBoolCap) Set(val interface{}) {
*i.cap = !bval
}

// spanConfigBoundsCap is an accessor struct for SpanConfigBounds that are
// stored on the underlying TenantCapabilities proto.
type spanConfigBoundsCap struct {
cap *SpanConfigBounds
}

// Get implements the tenantcapabilities.Capability interface.
func (s spanConfigBoundsCap) Get() tenantcapabilities.Value {
return s.cap
}

// Set implements the tenantcapabilities.Capability interface.
func (s spanConfigBoundsCap) Set(val interface{}) {
scfgBoundsVal, ok := val.(SpanConfigBounds)
if !ok {
panic(errors.AssertionFailedf("invalid value type: %T", val))
}
*s.cap = scfgBoundsVal
}

// Cap implements the tenantcapabilities.TenantCapabilities interface.
func (t *TenantCapabilities) Cap(
capabilityID tenantcapabilities.CapabilityID,
Expand All @@ -92,6 +118,8 @@ func (t *TenantCapabilities) Cap(
return boolCap{&t.CanViewTSDBMetrics}
case tenantcapabilities.ExemptFromRateLimiting:
return boolCap{&t.ExemptFromRateLimiting}
case tenantcapabilities.TenantSpanConfigBounds:
return spanConfigBoundsCap{t.SpanConfigBounds}

default:
panic(errors.AssertionFailedf("unknown capability: %q", capabilityID.String()))
Expand Down
19 changes: 10 additions & 9 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -623,6 +623,15 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
keys.SystemSQLCodec, clock, rangeFeedFactory, &cfg.DefaultZoneConfig,
)

tenantCapabilitiesWatcher := tenantcapabilitieswatcher.New(
clock,
rangeFeedFactory,
keys.TenantsTableID,
stopper,
1<<20, /* 1 MB */
tenantCapabilitiesTestingKnobs,
)

var spanConfig struct {
// kvAccessor powers the span configuration RPCs and the host tenant's
// reconciliation job.
Expand Down Expand Up @@ -668,6 +677,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
1<<20, /* 1 MB */
fallbackConf,
cfg.Settings,
tenantCapabilitiesWatcher,
spanConfigKnobs,
registry,
)
Expand Down Expand Up @@ -791,15 +801,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
clock, rangeFeedFactory, stopper, st,
)

tenantCapabilitiesWatcher := tenantcapabilitieswatcher.New(
clock,
rangeFeedFactory,
keys.TenantsTableID,
stopper,
1<<20, /* 1 MB */
tenantCapabilitiesTestingKnobs,
)

node := NewNode(
storeCfg,
recorder,
Expand Down
2 changes: 1 addition & 1 deletion pkg/spanconfig/spanconfigbounds/span_config_bounds.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ func (b Bounds) Check(c *roachpb.SpanConfig) Violations {

func (b Bounds) clamp(c *roachpb.SpanConfig, reporter func(Field)) (changed bool) {
for _, f := range fields {
if b := f.FieldBound(b); !b.clamp(c, f) {
if bb := f.FieldBound(b); !bb.clamp(c, f) {
continue
}
changed = true
Expand Down
2 changes: 2 additions & 0 deletions pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"//pkg/kv/kvclient/rangefeed/rangefeedbuffer",
"//pkg/kv/kvclient/rangefeed/rangefeedcache",
"//pkg/kv/kvpb",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down Expand Up @@ -57,6 +58,7 @@ go_test(
"//pkg/kv/kvclient/rangefeed/rangefeedbuffer",
"//pkg/kv/kvclient/rangefeed/rangefeedcache",
"//pkg/kv/kvserver",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
Expand Down
2 changes: 2 additions & 0 deletions pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor"
Expand Down Expand Up @@ -147,6 +148,7 @@ func TestDataDriven(t *testing.T) {
10<<20, /* 10 MB */
spanconfigtestutils.ParseConfig(t, "FALLBACK"),
tc.Server(0).ClusterSettings(),
tenantcapabilities.NewEmptyReader(),
&spanconfig.TestingKnobs{
KVSubscriberRangeFeedKnobs: &rangefeedcache.TestingKnobs{
OnTimestampAdvance: func(ts hlc.Timestamp) {
Expand Down
18 changes: 12 additions & 6 deletions pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -139,6 +140,9 @@ type KVSubscriber struct {

clock *hlc.Clock
metrics *Metrics

// capabilitiesReader provides a handle to the global tenant capability state.
capabilitiesReader tenantcapabilities.Reader
}

var _ spanconfig.KVSubscriber = &KVSubscriber{}
Expand Down Expand Up @@ -186,6 +190,7 @@ func New(
bufferMemLimit int64,
fallback roachpb.SpanConfig,
settings *cluster.Settings,
capabilitiesReader tenantcapabilities.Reader,
knobs *spanconfig.TestingKnobs,
registry *metric.Registry,
) *KVSubscriber {
Expand All @@ -200,12 +205,13 @@ func New(
Key: spanConfigTableStart,
EndKey: spanConfigTableStart.PrefixEnd(),
}
spanConfigStore := spanconfigstore.New(fallback, settings, knobs)
spanConfigStore := spanconfigstore.New(fallback, settings, capabilitiesReader, knobs)
s := &KVSubscriber{
fallback: fallback,
knobs: knobs,
settings: settings,
clock: clock,
fallback: fallback,
knobs: knobs,
settings: settings,
clock: clock,
capabilitiesReader: capabilitiesReader,
}
var rfCacheKnobs *rangefeedcache.TestingKnobs
if knobs != nil {
Expand Down Expand Up @@ -395,7 +401,7 @@ func (s *KVSubscriber) handleUpdate(ctx context.Context, u rangefeedcache.Update
func (s *KVSubscriber) handleCompleteUpdate(
ctx context.Context, ts hlc.Timestamp, events []rangefeedbuffer.Event,
) {
freshStore := spanconfigstore.New(s.fallback, s.settings, s.knobs)
freshStore := spanconfigstore.New(s.fallback, s.settings, s.capabilitiesReader, s.knobs)
for _, ev := range events {
freshStore.Apply(ctx, false /* dryrun */, ev.(*bufferEvent).Update)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand Down Expand Up @@ -91,6 +92,7 @@ func TestGetProtectionTimestamps(t *testing.T) {
1<<20, /* 1 MB */
roachpb.SpanConfig{},
cluster.MakeTestingClusterSettings(),
tenantcapabilities.NewEmptyReader(),
nil,
nil,
)
Expand Down
1 change: 1 addition & 0 deletions pkg/spanconfig/spanconfigreconciler/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
deps = [
"//pkg/keys",
"//pkg/kv",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/settings/cluster",
"//pkg/spanconfig",
Expand Down
5 changes: 4 additions & 1 deletion pkg/spanconfig/spanconfigreconciler/reconciler.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand Down Expand Up @@ -384,7 +385,9 @@ func (f *fullReconciler) fetchExistingSpanConfigs(
targets = append(targets,
spanconfig.MakeTargetFromSystemTarget(spanconfig.MakeAllTenantKeyspaceTargetsSet(f.tenID)))
}
store := spanconfigstore.New(roachpb.SpanConfig{}, f.settings, f.knobs)
// The reconciler doesn't do any bounds checks or clamping, so it shouldn't
// need access to tenant capabilities (and by extension span config bounds).
store := spanconfigstore.New(roachpb.SpanConfig{}, f.settings, tenantcapabilities.NewEmptyReader(), f.knobs)
{
// Fully populate the store with KVAccessor contents.
records, err := f.kvAccessor.GetSpanConfigRecords(ctx, targets)
Expand Down
1 change: 1 addition & 0 deletions pkg/spanconfig/spanconfigreporter/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ go_test(
"//pkg/keys",
"//pkg/kv/kvserver/constraint",
"//pkg/kv/kvserver/liveness/livenesspb",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
Expand Down
5 changes: 4 additions & 1 deletion pkg/spanconfig/spanconfigreporter/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/constraint"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand Down Expand Up @@ -233,7 +234,9 @@ func newMockCluster(
nodes: make(map[roachpb.NodeID]roachpb.NodeDescriptor),
ranges: make(map[roachpb.RangeID]roachpb.RangeDescriptor),
liveness: make(map[roachpb.NodeID]bool),
store: spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, scKnobs),
store: spanconfigstore.New(
roachpb.TestingDefaultSpanConfig(), st, tenantcapabilities.NewEmptyReader(), scKnobs,
),
}
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/spanconfig/spanconfigstore/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,13 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/keys",
"//pkg/multitenant/tenantcapabilities",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/spanconfig/spanconfigbounds",
"//pkg/util/iterutil",
"//pkg/util/log",
"//pkg/util/protoutil",
Expand All @@ -41,6 +44,7 @@ go_test(
embed = [":spanconfigstore"],
deps = [
"//pkg/keys",
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/settings/cluster",
"//pkg/spanconfig",
Expand Down
Loading

0 comments on commit 0664f06

Please sign in to comment.