Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
69658: spanconfig: disable infrastructure unless envvar is set r=irfansharif a=irfansharif

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 cockroachdb#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

69809: kv/kvserver: use proper formatting when debug printing intents r=AlexTalks a=AlexTalks

This commit changes the formatting used when printing intents via the
CLI debug command from the default generated Protobuf formatter to our
custom `MVCCMetadata` formatter implementation.  Additionally, the
`MergeTimestamp` and `TxnDidNotUpdateMetadata` fields have been added to
the output.  This changes the debug formatting from the former
example:
```
0,0 /Local/RangeID/203/r/RangePriorReadSummary (0x0169f6cb727270727300): {Txn:<nil> Timestamp:0,0 Deleted:false KeyBytes:0 ValBytes:0 RawBytes:[230 123 85 161 3 10 12 10 10 8 146 229 195 204 139 135 186 208 22 18 12 10 10 8 146 229 195 204 139 135 186 208 22] IntentHistory:[] Me
rgeTimestamp:<nil> TxnDidNotUpdateMeta:<nil>}
/Local/Lock/Intent/Table/56/1/1319/6/3055/0 0361fea07d3f0d40ba8f44dc4ee46cbdc2 (0x017a6b12c089f705278ef70bef880001000361fea07d3f0d40ba8f44dc4ee46cbdc212): 1630559399.176502568,0 {Txn:id=61fea07d key=/Table/57/1/1319/6/0 pri=0.01718258 epo=0 ts=1630559399.176502568,0 min=1630559399.176502568,0 seq=4 Timestamp:1630559399.176502568,0 Deleted:false KeyBytes:12 ValBytes:5 RawBytes:[] IntentHistory:[] MergeTimestamp:<nil> TxnDidNotUpdateMeta:0xc0016059b0}
```
to the following example:
```
0,0 /Local/RangeID/203/r/RangePriorReadSummary (0x0169f6cb727270727300): txn={<nil>} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/0x0a0c0a0a0892e5c3cc8b87bad016120c0a0a0892e5c3cc8b87bad016 mergeTs=<nil> txnDidNotUpdateMeta=false
/Local/Lock/Intent/Table/56/1/1319/6/3055/0 0361fea07d3f0d40ba8f44dc4ee46cbdc2 (0x017a6b12c089f705278ef70bef880001000361fea07d3f0d40ba8f44dc4ee46cbdc212): 1630559399.176502568,0 txn={id=61fea07d key=/Table/57/1/1319/6/0 pri=0.01718258 epo=0 ts=1630559399.176502568,0 min=1630559399.176502568,0 seq=4} ts=1630559399.176502568,0 del=false klen=12 vlen=5 mergeTs=<nil> txnDidNotUpdateMeta=true
```

Related to cockroachdb#69414

Release justification: Bug fix
Release note: None

Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: Alex Sarkesian <sarkesian@cockroachlabs.com>
  • Loading branch information
3 people committed Sep 14, 2021
3 parents b367897 + f3c94ef + a278eee commit 1f98510
Show file tree
Hide file tree
Showing 55 changed files with 565 additions and 340 deletions.
4 changes: 4 additions & 0 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,10 @@ type TestServerArgs struct {

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

// If set, the span configs infrastructure will be enabled. This is
// equivalent to setting COCKROACH_EXPERIMENTAL_SPAN_CONFIGS.
EnableSpanConfigs bool
}

// TestClusterArgs contains the parameters one can set when creating a test
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/doctor/test_examine_cluster
Original file line number Diff line number Diff line change
Expand Up @@ -3,5 +3,5 @@ debug doctor examine cluster
debug doctor examine cluster
Examining 41 descriptors and 42 namespace entries...
ParentID 50, ParentSchemaID 29: relation "foo" (53): expected matching namespace entry, found none
Examining 4 jobs...
Examining 3 jobs...
ERROR: validation failed
11 changes: 11 additions & 0 deletions pkg/clusterversion/clusterversion.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,12 @@ type Handle interface {
// node restarts when initializing the cluster version, as seen by this
// node.
SetActiveVersion(context.Context, ClusterVersion) error

// SetOnChange installs a callback that's invoked when the active cluster
// version changes. The callback should avoid doing long-running or blocking
// work; it's called on the same goroutine handling all cluster setting
// updates.
SetOnChange(fn func(context.Context))
}

// handleImpl is a concrete implementation of Handle. It mostly relegates to the
Expand Down Expand Up @@ -205,6 +211,11 @@ func (v *handleImpl) SetActiveVersion(ctx context.Context, cv ClusterVersion) er
return nil
}

// SetOnChange implements the Handle interface.
func (v *handleImpl) SetOnChange(fn func(ctx context.Context)) {
version.SetOnChange(v.sv, fn)
}

// IsActive implements the Handle interface.
func (v *handleImpl) IsActive(ctx context.Context, key Key) bool {
return version.isActive(ctx, v.sv, key)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/debug_print.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ func tryIntent(kv storage.MVCCKeyValue) (string, error) {
if err := protoutil.Unmarshal(kv.Value, &meta); err != nil {
return "", err
}
s := fmt.Sprintf("%+v", meta)
s := fmt.Sprintf("%+v", &meta)
if meta.Txn != nil {
s = meta.Txn.WriteTimestamp.String() + " " + s
}
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 @@ -753,6 +753,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 @@ -149,6 +149,11 @@ type BaseConfig struct {
// instantiate stores.
StorageEngine enginepb.EngineType

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

// TestingKnobs is used for internal test controls only.
TestingKnobs base.TestingKnobs
}
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 @@ -648,6 +656,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
18 changes: 13 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 @@ -602,6 +603,18 @@ 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(),
)
} else {
spanConfigAccessor = spanconfigkvaccessor.DisabledAccessor{}
}

if storeTestingKnobs := cfg.TestingKnobs.Store; storeTestingKnobs != nil {
storeCfg.TestingKnobs = *storeTestingKnobs.(*kvserver.StoreTestingKnobs)
}
Expand Down Expand Up @@ -630,11 +643,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
38 changes: 21 additions & 17 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -831,20 +831,22 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
execCfg.MigrationTestingKnobs = knobs
}

// 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(
cfg.db,
jobRegistry,
cfg.circularInternalExecutor,
cfg.stopper,
cfg.Settings,
cfg.spanConfigAccessor,
knobs,
)
execCfg.SpanConfigReconciliationJobDeps = spanconfigMgr
var spanConfigMgr *spanconfigmanager.Manager
if !codec.ForSystemTenant() || cfg.SpanConfigsEnabled {
// Instantiate a span config manager. If we're the host tenant we'll
// only do it if COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is set.
spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs)
spanConfigMgr = spanconfigmanager.New(
cfg.db,
jobRegistry,
cfg.circularInternalExecutor,
cfg.stopper,
cfg.Settings,
cfg.spanConfigAccessor,
spanConfigKnobs,
)
execCfg.SpanConfigReconciliationJobDeps = spanConfigMgr
}

temporaryObjectCleaner := sql.NewTemporaryObjectCleaner(
cfg.Settings,
Expand Down Expand Up @@ -906,7 +908,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 Expand Up @@ -1045,8 +1047,10 @@ func (s *SQLServer) preStart(
return err
}

if err := s.spanconfigMgr.Start(ctx); err != nil {
return err
if s.spanconfigMgr != nil {
if err := s.spanconfigMgr.Start(ctx); err != nil {
return err
}
}

var bootstrapVersion roachpb.Version
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
12 changes: 0 additions & 12 deletions pkg/settings/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,15 +176,3 @@ func TestingRegisterVersionSetting(key, desc string, impl VersionSettingImpl) *V
register(key, desc, &setting)
return &setting
}

// SetOnChange is part of the Setting interface, and is discouraged for use in
// VersionSetting (we're implementing it here to not fall back on the embedded
// `common` type definition).
//
// NB: VersionSetting is unique in more ways than one, and we might want to move
// it out of the settings package before long (see TODO on the type itself). In
// our current usage we don't rely on attaching pre-change triggers, so let's
// not add it needlessly.
func (v *VersionSetting) SetOnChange(sv *Values, fn func(ctx context.Context)) {
panic("unimplemented")
}
5 changes: 4 additions & 1 deletion pkg/spanconfig/spanconfigkvaccessor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "spanconfigkvaccessor",
srcs = ["kvaccessor.go"],
srcs = [
"disabled.go",
"kvaccessor.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor",
visibility = ["//visibility:public"],
deps = [
Expand Down
39 changes: 39 additions & 0 deletions pkg/spanconfig/spanconfigkvaccessor/disabled.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package spanconfigkvaccessor

import (
"context"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/errors"
)

// DisabledAccessor provides a implementation of the KVAccessor interface that
// simply errors out with a disabled error.
type DisabledAccessor struct{}

// GetSpanConfigEntriesFor is part of the KVAccessor interface.
func (n DisabledAccessor) GetSpanConfigEntriesFor(
context.Context, []roachpb.Span,
) ([]roachpb.SpanConfigEntry, error) {
return nil, errors.New("span configs disabled")
}

// UpdateSpanConfigEntries is part of the KVAccessor interface.
func (n DisabledAccessor) UpdateSpanConfigEntries(
context.Context, []roachpb.Span, []roachpb.SpanConfigEntry,
) error {
return errors.New("span configs disabled")
}

var _ spanconfig.KVAccessor = &DisabledAccessor{}
19 changes: 15 additions & 4 deletions pkg/spanconfig/spanconfigkvaccessor/kvaccessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,17 +52,24 @@ func New(
}
}

var kvAccessorEnabled = settings.RegisterBoolSetting(
"spanconfig.kvaccessor_experimental.enabled",
// enabledSetting 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()

// errDisabled is returned if the setting gating usage of the KVAccessor is
// disabled.
var errDisabled = errors.New("span config kv accessor disabled")

// 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) {
return nil, errors.New("use of span configs disabled")
if !enabledSetting.Get(&k.settings.SV) {
return nil, errDisabled
}

if len(spans) == 0 {
return resp, nil
}
Expand Down Expand Up @@ -111,6 +118,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 errDisabled
}

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
6 changes: 4 additions & 2 deletions pkg/spanconfig/spanconfigmanager/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@ go_library(
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
"//pkg/util/log",
"//pkg/util/stop",
Expand All @@ -35,14 +33,18 @@ go_test(
deps = [
":spanconfigmanager",
"//pkg/base",
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/protoutil",
Expand Down
Loading

0 comments on commit 1f98510

Please sign in to comment.