Skip to content

Commit

Permalink
spanconfig: disable infrastructure unless envvar is set
Browse files Browse the repository at this point in the history
Cluster settings are too easy a switch to reach for to enable the new
span configs machinery. Let's gate it behind a necessary envvar as
well and use cluster settings to selectively toggle individual
components.

This commit also fixes a mighty silly bug introduced in #69047; for the
two methods we intended to gate use
`spanconfig.experimental_kvaccessor.enabled`, we were checking the
opposite condition or not checking it at all. Oops.

Release note: None
Release justification: non-production code changes
  • Loading branch information
irfansharif committed Aug 31, 2021
1 parent 298cd21 commit ce9e3b7
Show file tree
Hide file tree
Showing 11 changed files with 69 additions and 14 deletions.
3 changes: 3 additions & 0 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,9 @@ type TestServerArgs struct {

// If set, a TraceDir is initialized at the provided path.
TraceDir string

// If set, the span configs infrastructure will be enabled.
EnableSpanConfigs bool
}

// TestClusterArgs contains the parameters one can set when creating a test
Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -729,6 +729,10 @@ type StoreConfig struct {
// KV Memory Monitor. Must be non-nil for production, and can be nil in some
// tests.
KVMemoryMonitor *mon.BytesMonitor

// SpanConfigsEnabled determines whether we're able to use the span configs
// infrastructure.
SpanConfigsEnabled bool
}

// ConsistencyTestingKnobs is a BatchEvalTestingKnobs struct used to control the
Expand Down
9 changes: 9 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,11 @@ type KVConfig struct {
// The following values can only be set via environment variables and are
// for testing only. They are not meant to be set by the end user.

// Enables the use of the (experimental) span configs infrastructure.
//
// Environment Variable: COCKROACH_EXPERIMENTAL_SPAN_CONFIGS
SpanConfigsEnabled bool

// Enables linearizable behavior of operations on this node by making sure
// that no commit timestamp is reported back to the client until all other
// node clocks have necessarily passed it.
Expand Down Expand Up @@ -417,6 +422,9 @@ func (cfg *Config) String() string {
if cfg.Linearizable {
fmt.Fprintln(w, "linearizable\t", cfg.Linearizable)
}
if cfg.SpanConfigsEnabled {
fmt.Fprintln(w, "span configs enabled\t", cfg.SpanConfigsEnabled)
}
_ = w.Flush()

return buf.String()
Expand Down Expand Up @@ -647,6 +655,7 @@ func (cfg *Config) RequireWebSession() bool {
// variable based. Note that this only happens when initializing a node and not
// when NewContext is called.
func (cfg *Config) readEnvironmentVariables() {
cfg.SpanConfigsEnabled = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS", cfg.SpanConfigsEnabled)
cfg.Linearizable = envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_LINEARIZABLE", cfg.Linearizable)
cfg.ScanInterval = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_INTERVAL", cfg.ScanInterval)
cfg.ScanMinIdleTime = envutil.EnvOrDefaultDuration("COCKROACH_SCAN_MIN_IDLE_TIME", cfg.ScanMinIdleTime)
Expand Down
8 changes: 8 additions & 0 deletions pkg/server/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,9 @@ func TestReadEnvironmentVariables(t *testing.T) {
if err := os.Unsetenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE"); err != nil {
t.Fatal(err)
}
if err := os.Unsetenv("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS"); err != nil {
t.Fatal(err)
}
if err := os.Unsetenv("COCKROACH_SCAN_INTERVAL"); err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -125,6 +128,10 @@ func TestReadEnvironmentVariables(t *testing.T) {

// Set all the environment variables to valid values and ensure they are set
// correctly.
if err := os.Setenv("COCKROACH_EXPERIMENTAL_SPAN_CONFIGS", "true"); err != nil {
t.Fatal(err)
}
cfgExpected.SpanConfigsEnabled = true
if err := os.Setenv("COCKROACH_EXPERIMENTAL_LINEARIZABLE", "true"); err != nil {
t.Fatal(err)
}
Expand All @@ -149,6 +156,7 @@ func TestReadEnvironmentVariables(t *testing.T) {
}

for _, envVar := range []string{
"COCKROACH_EXPERIMENTAL_SPAN_CONFIGS",
"COCKROACH_EXPERIMENTAL_LINEARIZABLE",
"COCKROACH_SCAN_INTERVAL",
"COCKROACH_SCAN_MIN_IDLE_TIME",
Expand Down
7 changes: 7 additions & 0 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -1447,6 +1447,10 @@ func (emptyMetricStruct) MetricStruct() {}
func (n *Node) GetSpanConfigs(
ctx context.Context, req *roachpb.GetSpanConfigsRequest,
) (*roachpb.GetSpanConfigsResponse, error) {
if !n.storeCfg.SpanConfigsEnabled {
return nil, errors.New("use of span configs disabled")
}

entries, err := n.spanConfigAccessor.GetSpanConfigEntriesFor(ctx, req.Spans)
if err != nil {
return nil, err
Expand All @@ -1459,6 +1463,9 @@ func (n *Node) GetSpanConfigs(
func (n *Node) UpdateSpanConfigs(
ctx context.Context, req *roachpb.UpdateSpanConfigsRequest,
) (*roachpb.UpdateSpanConfigsResponse, error) {
if !n.storeCfg.SpanConfigsEnabled {
return nil, errors.New("use of span configs disabled")
}
// TODO(irfansharif): We want to protect ourselves from tenants creating
// outlandishly large string buffers here and OOM-ing the host cluster. Is
// the maximum protobuf message size enough of a safeguard?
Expand Down
16 changes: 11 additions & 5 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
_ "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" // register jobs declared outside of pkg/sql
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -627,6 +628,16 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
ProtectedTimestampCache: protectedtsProvider,
KVMemoryMonitor: kvMemoryMonitor,
}

var spanConfigAccessor spanconfig.KVAccessor
if cfg.SpanConfigsEnabled {
storeCfg.SpanConfigsEnabled = true
spanConfigAccessor = spanconfigkvaccessor.New(
db, internalExecutor, cfg.Settings,
systemschema.SpanConfigurationsTableName.FQString(),
)
}

if storeTestingKnobs := cfg.TestingKnobs.Store; storeTestingKnobs != nil {
storeCfg.TestingKnobs = *storeTestingKnobs.(*kvserver.StoreTestingKnobs)
}
Expand Down Expand Up @@ -655,11 +666,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
tenantUsage := NewTenantUsageServer(db, internalExecutor)
registry.AddMetricStruct(tenantUsage.Metrics())

spanConfigAccessor := spanconfigkvaccessor.New(
db, internalExecutor, cfg.Settings,
systemschema.SpanConfigurationsTableName.FQString(),
)

node := NewNode(
storeCfg, recorder, registry, stopper,
txnMetrics, stores, nil /* execCfg */, &rpcContext.ClusterID,
Expand Down
10 changes: 5 additions & 5 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -832,17 +832,17 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
// Instantiate a span config manager; it exposes a hook to idempotently
// create the span config reconciliation job and captures all relevant job
// dependencies.
knobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs)
spanconfigMgr := spanconfigmanager.New(
spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs)
spanConfigMgr := spanconfigmanager.New(
cfg.db,
jobRegistry,
cfg.circularInternalExecutor,
cfg.stopper,
cfg.Settings,
cfg.spanConfigAccessor,
knobs,
spanConfigKnobs,
)
execCfg.SpanConfigReconciliationJobDeps = spanconfigMgr
execCfg.SpanConfigReconciliationJobDeps = spanConfigMgr

temporaryObjectCleaner := sql.NewTemporaryObjectCleaner(
cfg.Settings,
Expand Down Expand Up @@ -904,7 +904,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
sqlInstanceProvider: cfg.sqlInstanceProvider,
metricsRegistry: cfg.registry,
diagnosticsReporter: reporter,
spanconfigMgr: spanconfigMgr,
spanconfigMgr: spanConfigMgr,
settingsWatcher: settingsWatcher,
}, nil
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,9 @@ func makeTestConfigFromParams(params base.TestServerArgs) Config {
if params.EnableDemoLoginEndpoint {
cfg.EnableDemoLoginEndpoint = true
}
if params.EnableSpanConfigs {
cfg.SpanConfigsEnabled = true
}

// Ensure we have the correct number of engines. Add in-memory ones where
// needed. There must be at least one store/engine.
Expand Down
14 changes: 11 additions & 3 deletions pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,17 +52,21 @@ func New(
}
}

var kvAccessorEnabled = settings.RegisterBoolSetting(
"spanconfig.kvaccessor_experimental.enabled",
// enabledSetting is a hidden cluster setting that gates usage of the
// KVAccessor. It has no effect unless COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is
// also set.
var enabledSetting = settings.RegisterBoolSetting(
"spanconfig.experimental_kvaccessor.enabled",
"enable the use of the kv accessor", false).WithSystemOnly()

// GetSpanConfigEntriesFor is part of the KVAccessor interface.
func (k *KVAccessor) GetSpanConfigEntriesFor(
ctx context.Context, spans []roachpb.Span,
) (resp []roachpb.SpanConfigEntry, retErr error) {
if kvAccessorEnabled.Get(&k.settings.SV) {
if !enabledSetting.Get(&k.settings.SV) {
return nil, errors.New("use of span configs disabled")
}

if len(spans) == 0 {
return resp, nil
}
Expand Down Expand Up @@ -111,6 +115,10 @@ func (k *KVAccessor) GetSpanConfigEntriesFor(
func (k *KVAccessor) UpdateSpanConfigEntries(
ctx context.Context, toDelete []roachpb.Span, toUpsert []roachpb.SpanConfigEntry,
) error {
if !enabledSetting.Get(&k.settings.SV) {
return errors.New("use of span configs disabled")
}

if err := validateUpdateArgs(toDelete, toUpsert); err != nil {
return err
}
Expand Down
7 changes: 6 additions & 1 deletion pkg/spanconfig/spanconfigkvaccessor/kvaccessor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,11 @@ func TestKVAccessor(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
EnableSpanConfigs: true,
},
})
defer tc.Stopper().Stop(ctx)

span := func(start, end string) roachpb.Span {
Expand Down Expand Up @@ -72,6 +76,7 @@ func TestKVAccessor(t *testing.T) {

const dummySpanConfigurationsFQN = "defaultdb.public.dummy_span_configurations"
tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0))
tdb.Exec(t, `SET CLUSTER SETTING spanconfig.experimental_kvaccessor.enabled = true`)
tdb.Exec(t, fmt.Sprintf("CREATE TABLE %s (LIKE system.span_configurations INCLUDING ALL)", dummySpanConfigurationsFQN))
accessor := spanconfigkvaccessor.New(
tc.Server(0).DB(),
Expand Down
2 changes: 2 additions & 0 deletions pkg/spanconfig/spanconfigmanager/manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ func TestManagerConcurrentJobCreation(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
EnableSpanConfigs: true,
Knobs: base.TestingKnobs{
SpanConfig: &spanconfig.TestingKnobs{
ManagerDisableJobCreation: true, // disable the automatic job creation
Expand Down Expand Up @@ -136,6 +137,7 @@ func TestManagerStartsJobIfFailed(t *testing.T) {
ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
EnableSpanConfigs: true,
Knobs: base.TestingKnobs{
SpanConfig: &spanconfig.TestingKnobs{
ManagerDisableJobCreation: true, // disable the automatic job creation
Expand Down

0 comments on commit ce9e3b7

Please sign in to comment.