-admission.kv.enabled | boolean | false | when true, work performed by the KV layer is subject to admission control |
-admission.sql_kv_response.enabled | boolean | false | when true, work performed by the SQL layer when receiving a KV response is subject to admission control |
-admission.sql_sql_response.enabled | boolean | false | when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control |
+admission.kv.enabled | boolean | true | when true, work performed by the KV layer is subject to admission control |
+admission.sql_kv_response.enabled | boolean | true | when true, work performed by the SQL layer when receiving a KV response is subject to admission control |
+admission.sql_sql_response.enabled | boolean | true | when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control |
bulkio.backup.file_size | byte size | 128 MiB | target size for individual data files produced during BACKUP |
bulkio.backup.read_timeout | duration | 5m0s | amount of time after which a read attempt is considered timed out, which causes the backup to fail |
bulkio.backup.read_with_priority_after | duration | 1m0s | amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads |
diff --git a/pkg/cmd/roachtest/tests/kv.go b/pkg/cmd/roachtest/tests/kv.go
index 77790420a44e..1fc59cdfb4fb 100644
--- a/pkg/cmd/roachtest/tests/kv.go
+++ b/pkg/cmd/roachtest/tests/kv.go
@@ -48,15 +48,15 @@ func registerKV(r registry.Registry) {
blockSize int
splits int // 0 implies default, negative implies 0
// If true, load-based splitting will be disabled.
- disableLoadSplits bool
- encryption bool
- sequential bool
- admissionControlEnabled bool
- concMultiplier int
- duration time.Duration
- tracing bool // `trace.debug.enable`
- tags []string
- owner registry.Owner // defaults to KV
+ disableLoadSplits bool
+ encryption bool
+ sequential bool
+ admissionControlDisabled bool
+ concMultiplier int
+ duration time.Duration
+ tracing bool // `trace.debug.enable`
+ tags []string
+ owner registry.Owner // defaults to KV
}
computeNumSplits := func(opts kvOptions) int {
// TODO(ajwerner): set this default to a more sane value or remove it and
@@ -89,9 +89,7 @@ func registerKV(r registry.Registry) {
t.Fatalf("failed to enable tracing: %v", err)
}
}
- if opts.admissionControlEnabled {
- EnableAdmissionControl(ctx, t, c)
- }
+ SetAdmissionControl(ctx, t, c, !opts.admissionControlDisabled)
t.Status("running workload")
m := c.NewMonitor(ctx, c.Range(1, nodes))
@@ -150,13 +148,8 @@ func registerKV(r registry.Registry) {
// CPU overload test, to stress admission control.
{nodes: 1, cpus: 8, readPercent: 50, concMultiplier: 8192, duration: 20 * time.Minute},
// IO write overload test, to stress admission control.
- //
- // TODO(sumeerbhola): re-enable when admission control is enabled by default. This
- // test crashes the cluster every now and then. See:
- // https://github.com/cockroachdb/cockroach/issues/70247
- //
- // {nodes: 1, cpus: 8, readPercent: 0, concMultiplier: 4096, blockSize: 1 << 16, /* 64 KB */
- // duration: 20 * time.Minute},
+ {nodes: 1, cpus: 8, readPercent: 0, concMultiplier: 4096, blockSize: 1 << 16, /* 64 KB */
+ duration: 20 * time.Minute},
{nodes: 1, cpus: 8, readPercent: 95},
{nodes: 1, cpus: 32, readPercent: 0},
{nodes: 1, cpus: 32, readPercent: 95},
@@ -167,8 +160,8 @@ func registerKV(r registry.Registry) {
{nodes: 3, cpus: 8, readPercent: 95, splits: -1 /* no splits */},
{nodes: 3, cpus: 32, readPercent: 0},
{nodes: 3, cpus: 32, readPercent: 95},
- {nodes: 3, cpus: 32, readPercent: 0, admissionControlEnabled: true},
- {nodes: 3, cpus: 32, readPercent: 95, admissionControlEnabled: true},
+ {nodes: 3, cpus: 32, readPercent: 0, admissionControlDisabled: true},
+ {nodes: 3, cpus: 32, readPercent: 95, admissionControlDisabled: true},
{nodes: 3, cpus: 32, readPercent: 0, splits: -1 /* no splits */},
{nodes: 3, cpus: 32, readPercent: 95, splits: -1 /* no splits */},
@@ -182,9 +175,9 @@ func registerKV(r registry.Registry) {
{nodes: 3, cpus: 32, readPercent: 0, blockSize: 1 << 16 /* 64 KB */},
{nodes: 3, cpus: 32, readPercent: 95, blockSize: 1 << 16 /* 64 KB */},
{nodes: 3, cpus: 32, readPercent: 0, blockSize: 1 << 16, /* 64 KB */
- admissionControlEnabled: true},
+ admissionControlDisabled: true},
{nodes: 3, cpus: 32, readPercent: 95, blockSize: 1 << 16, /* 64 KB */
- admissionControlEnabled: true},
+ admissionControlDisabled: true},
// Configs with large batch sizes.
{nodes: 3, cpus: 8, readPercent: 0, batchSize: 16},
@@ -241,8 +234,8 @@ func registerKV(r registry.Registry) {
if opts.sequential {
nameParts = append(nameParts, "seq")
}
- if opts.admissionControlEnabled {
- nameParts = append(nameParts, "admission")
+ if opts.admissionControlDisabled {
+ nameParts = append(nameParts, "no-admission")
}
if opts.concMultiplier != 0 { // support legacy test name which didn't include this multiplier
nameParts = append(nameParts, fmt.Sprintf("conc=%d", opts.concMultiplier))
@@ -873,7 +866,9 @@ func registerKVMultiStoreWithOverload(r registry.Registry) {
t.Fatalf("failed to configure zone for %s: %v", name, err)
}
}
- EnableAdmissionControl(ctx, t, c)
+ // Defensive, since admission control is enabled by default. This test can
+ // fail if admission control is disabled.
+ SetAdmissionControl(ctx, t, c, true)
if _, err := db.ExecContext(ctx,
"SET CLUSTER SETTING kv.range_split.by_load_enabled = 'false'"); err != nil {
t.Fatalf("failed to disable load based splitting: %v", err)
diff --git a/pkg/cmd/roachtest/tests/tpcc.go b/pkg/cmd/roachtest/tests/tpcc.go
index 5ffc88fe7e4e..d915d3b54146 100644
--- a/pkg/cmd/roachtest/tests/tpcc.go
+++ b/pkg/cmd/roachtest/tests/tpcc.go
@@ -693,9 +693,9 @@ func registerTPCC(r registry.Registry) {
EstimatedMax: gceOrAws(cloud, 2400, 3000),
})
registerTPCCBenchSpec(r, tpccBenchSpec{
- Nodes: 3,
- CPUs: 16,
- AdmissionControlEnabled: true,
+ Nodes: 3,
+ CPUs: 16,
+ AdmissionControlDisabled: true,
LoadWarehouses: gceOrAws(cloud, 3000, 3500),
EstimatedMax: gceOrAws(cloud, 2400, 3000),
@@ -801,12 +801,12 @@ func (l tpccBenchLoadConfig) numLoadNodes(d tpccBenchDistribution) int {
}
type tpccBenchSpec struct {
- Nodes int
- CPUs int
- Chaos bool
- AdmissionControlEnabled bool
- Distribution tpccBenchDistribution
- LoadConfig tpccBenchLoadConfig
+ Nodes int
+ CPUs int
+ Chaos bool
+ AdmissionControlDisabled bool
+ Distribution tpccBenchDistribution
+ LoadConfig tpccBenchLoadConfig
// The number of warehouses to load into the cluster before beginning
// benchmarking. Should be larger than EstimatedMax and should be a
@@ -857,8 +857,8 @@ func registerTPCCBenchSpec(r registry.Registry, b tpccBenchSpec) {
if b.Chaos {
nameParts = append(nameParts, "chaos")
}
- if b.AdmissionControlEnabled {
- nameParts = append(nameParts, "admission")
+ if b.AdmissionControlDisabled {
+ nameParts = append(nameParts, "no-admission")
}
opts := []spec.Option{spec.CPU(b.CPUs)}
@@ -1034,9 +1034,7 @@ func runTPCCBench(ctx context.Context, t test.Test, c cluster.Cluster, b tpccBen
c.EncryptDefault(false)
c.EncryptAtRandom(false)
c.Start(ctx, append(b.startOpts(), roachNodes)...)
- if b.AdmissionControlEnabled {
- EnableAdmissionControl(ctx, t, c)
- }
+ SetAdmissionControl(ctx, t, c, !b.AdmissionControlDisabled)
useHAProxy := b.Chaos
const restartWait = 15 * time.Second
{
@@ -1125,9 +1123,7 @@ func runTPCCBench(ctx context.Context, t test.Test, c cluster.Cluster, b tpccBen
}
c.Start(ctx, append(b.startOpts(), roachNodes)...)
- if b.AdmissionControlEnabled {
- EnableAdmissionControl(ctx, t, c)
- }
+ SetAdmissionControl(ctx, t, c, !b.AdmissionControlDisabled)
}
s := search.NewLineSearcher(1, b.LoadWarehouses, b.EstimatedMax, initStepSize, precision)
diff --git a/pkg/cmd/roachtest/tests/util.go b/pkg/cmd/roachtest/tests/util.go
index 7cdb6e1684d7..5e177b217a37 100644
--- a/pkg/cmd/roachtest/tests/util.go
+++ b/pkg/cmd/roachtest/tests/util.go
@@ -94,16 +94,20 @@ func WaitForUpdatedReplicationReport(ctx context.Context, t test.Test, db *gosql
}
}
-// EnableAdmissionControl enables the admission control cluster settings on
-// the given cluster.
-func EnableAdmissionControl(ctx context.Context, t test.Test, c cluster.Cluster) {
+// SetAdmissionControl sets the admission control cluster settings on the
+// given cluster.
+func SetAdmissionControl(ctx context.Context, t test.Test, c cluster.Cluster, enabled bool) {
db := c.Conn(ctx, 1)
defer db.Close()
+ val := "true"
+ if !enabled {
+ val = "false"
+ }
for _, setting := range []string{"admission.kv.enabled", "admission.sql_kv_response.enabled",
"admission.sql_sql_response.enabled"} {
if _, err := db.ExecContext(
- ctx, "SET CLUSTER SETTING "+setting+" = 'true'"); err != nil {
- t.Fatalf("failed to enable admission control: %v", err)
+ ctx, "SET CLUSTER SETTING "+setting+" = '"+val+"'"); err != nil {
+ t.Fatalf("failed to set admission control to %t: %v", enabled, err)
}
}
}
diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go
index 5b7089da8fce..6a112ccf9b18 100644
--- a/pkg/util/admission/work_queue.go
+++ b/pkg/util/admission/work_queue.go
@@ -36,7 +36,7 @@ import (
var KVAdmissionControlEnabled = settings.RegisterBoolSetting(
"admission.kv.enabled",
"when true, work performed by the KV layer is subject to admission control",
- false).WithPublic()
+ true).WithPublic()
// SQLKVResponseAdmissionControlEnabled controls whether response processing
// in SQL, for KV requests, is enabled.
@@ -44,7 +44,7 @@ var SQLKVResponseAdmissionControlEnabled = settings.RegisterBoolSetting(
"admission.sql_kv_response.enabled",
"when true, work performed by the SQL layer when receiving a KV response is subject to "+
"admission control",
- false).WithPublic()
+ true).WithPublic()
// SQLSQLResponseAdmissionControlEnabled controls whether response processing
// in SQL, for DistSQL requests, is enabled.
@@ -52,7 +52,7 @@ var SQLSQLResponseAdmissionControlEnabled = settings.RegisterBoolSetting(
"admission.sql_sql_response.enabled",
"when true, work performed by the SQL layer when receiving a DistSQL response is subject "+
"to admission control",
- false).WithPublic()
+ true).WithPublic()
var admissionControlEnabledSettings = [numWorkKinds]*settings.BoolSetting{
KVWork: KVAdmissionControlEnabled,