Skip to content

Commit d627a86

Browse files
committed
admission: move yielding into ElasticCPUWorkHandle
ElasticCPUWorkHandle.Overlimit is expected to be called in a tight loop, so yielding there is conceptually the right place. More importantly, this will allow in the future for KV work that is not holding latches to also yield. As part of this change, elastic work that does not wish to wait in admission control queues (due to cluster settings), is now accounted for in the elastic tokens, and in the admission.elastic_cpu_bypassed.utilization metric. One side-effect of this accounting is that work that needs to wait in admission queues may have fewer tokens available to it, and may wait longer. This is considered acceptable since: - Elastic work that bypasses queueing is still elastic work, and our overarching goal is to reduce impact to foreground work. - Due to the default on use of runtime.Yield, all elastic work yields, which allows the system to run at higher elastic CPU utilization without impacting the latency of foreground work. Epic: none Release note: None
1 parent 0331673 commit d627a86

File tree

21 files changed

+193
-164
lines changed

21 files changed

+193
-164
lines changed

docs/generated/metrics/metrics.yaml

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10841,6 +10841,14 @@ layers:
1084110841
unit: PERCENT
1084210842
aggregation: AVG
1084310843
derivative: NONE
10844+
- name: admission.elastic_cpu_bypassed.utilization
10845+
exported_name: admission_elastic_cpu_bypassed_utilization
10846+
description: CPU utilization by elastic work that bypassed admission
10847+
y_axis_label: CPU Time
10848+
type: GAUGE
10849+
unit: PERCENT
10850+
aggregation: AVG
10851+
derivative: NONE
1084410852
- name: admission.errored.elastic-cpu
1084510853
exported_name: admission_errored_elastic_cpu
1084610854
description: Number of requests not admitted due to error

pkg/backup/datadriven_test.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ import (
2727
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
2828
"github.com/cockroachdb/cockroach/pkg/keys"
2929
"github.com/cockroachdb/cockroach/pkg/kv"
30-
"github.com/cockroachdb/cockroach/pkg/kv/bulk"
3130
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
3231
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
3332
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
@@ -200,8 +199,7 @@ func (d *datadrivenTestState) addCluster(t *testing.T, cfg clusterCfg) error {
200199
// and overload the host. Generally overload would mean bulk work, which only
201200
// uses strictly spare capacitym gets starved, but these tests expect it to
202201
// still run (just slowly, along with everything else).
203-
bulk.YieldIfNoPacer.Override(context.Background(), &settings.SV, false)
204-
admission.YieldInPacer.Override(context.Background(), &settings.SV, false)
202+
admission.YieldForElasticCPU.Override(context.Background(), &settings.SV, false)
205203
params.ServerArgs.Settings = settings
206204

207205
clusterSize := cfg.nodes

pkg/kv/bulk/cpu_pacer.go

Lines changed: 10 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -25,26 +25,18 @@ var cpuPacerRequestDuration = settings.RegisterDurationSetting(
2525
50*time.Millisecond,
2626
)
2727

28-
// YieldIfNoPacer is exported so it can be overridden in tests.
29-
var YieldIfNoPacer = settings.RegisterBoolSetting(
30-
settings.ApplicationLevel,
31-
"bulkio.elastic_cpu_control.always_yield.enabled",
32-
"if true, yield the CPU as needed even when time-based elastic pacing is not enabled",
33-
true,
34-
)
35-
36-
// NewCPUPacer creates a new AC pacer for SST batcher. It may return an empty
37-
// Pacer which noops if pacing is disabled or its arguments are nil.
28+
// NewCPUPacer creates a new AC pacer for SST batcher. It will return an empty
29+
// Pacer which noops if db or db.AdmissionPacerFactory is nil.
30+
//
31+
// The setting specifies where waiting in the elastic admission control queue
32+
// is enabled. If disabled, the CPU consumed will be accounted for in
33+
// admission control, but pacing will not wait in admission control.
3834
func NewCPUPacer(ctx context.Context, db *kv.DB, setting *settings.BoolSetting) *admission.Pacer {
39-
if db == nil || db.AdmissionPacerFactory == nil || !setting.Get(db.SettingsValues()) {
40-
log.Dev.Infof(ctx, "admission control is not configured to pace bulk ingestion")
41-
42-
if db != nil && YieldIfNoPacer.Get(db.SettingsValues()) {
43-
// Return a Pacer that just yields.
44-
return &admission.Pacer{Yield: true}
45-
}
35+
if db == nil || db.AdmissionPacerFactory == nil {
36+
log.Dev.Infof(ctx, "admission control is not configured to pace this bulk work")
4637
return nil
4738
}
39+
bypassACQueue := !setting.Get(db.SettingsValues())
4840
tenantID, ok := roachpb.ClientTenantFromContext(ctx)
4941
if !ok {
5042
tenantID = roachpb.SystemTenantID
@@ -55,6 +47,6 @@ func NewCPUPacer(ctx context.Context, db *kv.DB, setting *settings.BoolSetting)
5547
TenantID: tenantID,
5648
Priority: admissionpb.BulkNormalPri,
5749
CreateTime: timeutil.Now().UnixNano(),
58-
BypassAdmission: false,
50+
BypassAdmission: bypassACQueue,
5951
})
6052
}

pkg/kv/kvserver/kvadmission/kvadmission.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -396,7 +396,7 @@ func (n *controllerImpl) AdmitKVWork(
396396
// reads, so in some sense, the integration is incomplete. This is
397397
// probably harmless.
398398
elasticWorkHandle, err := n.elasticCPUGrantCoordinator.ElasticCPUWorkQueue.Admit(
399-
ctx, admitDuration, admissionInfo,
399+
ctx, admitDuration, admissionInfo, false,
400400
)
401401
if err != nil {
402402
return Handle{}, err

pkg/roachprod/agents/opentelemetry/files/cockroachdb_metrics.yaml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,7 @@ admission_admitted_sql_sql_response_locking_normal_pri: admission.admitted.sql_s
6565
admission_admitted_sql_sql_response_normal_pri: admission.admitted.sql_sql_response.normal_pri
6666
admission_elastic_cpu_acquired_nanos: admission.elastic_cpu.acquired_nanos
6767
admission_elastic_cpu_available_nanos: admission.elastic_cpu.available_nanos
68+
admission_elastic_cpu_bypassed_utilization: admission.elastic_cpu_bypassed.utilization
6869
admission_elastic_cpu_max_available_nanos: admission.elastic_cpu.max_available_nanos
6970
admission_elastic_cpu_nanos_exhausted_duration: admission.elastic_cpu.nanos_exhausted_duration
7071
admission_elastic_cpu_over_limit_durations: admission.elastic_cpu.over_limit_durations

pkg/server/testserver.go

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/jobs"
2727
"github.com/cockroachdb/cockroach/pkg/keys"
2828
"github.com/cockroachdb/cockroach/pkg/kv"
29-
"github.com/cockroachdb/cockroach/pkg/kv/bulk"
3029
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
3130
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
3231
"github.com/cockroachdb/cockroach/pkg/kv/kvprober"
@@ -1486,8 +1485,7 @@ func (ts *testServer) StartSharedProcessTenant(
14861485

14871486
// Disable yield AC for tenant servers in tests, for the same reason as the
14881487
// system tenant (see comment in serverutils.NewServer).
1489-
admission.YieldInPacer.Override(ctx, &sqlServer.cfg.Settings.SV, false)
1490-
bulk.YieldIfNoPacer.Override(ctx, &sqlServer.cfg.Settings.SV, false)
1488+
admission.YieldForElasticCPU.Override(ctx, &sqlServer.cfg.Settings.SV, false)
14911489

14921490
hts := &httpTestServer{}
14931491
hts.t.authentication = sqlServerWrapper.authentication
@@ -1879,8 +1877,7 @@ func (ts *testServer) StartTenant(
18791877

18801878
// Disable yield AC for tenant servers in tests, for the same reason as the
18811879
// system tenant (see comment in serverutils.NewServer).
1882-
admission.YieldInPacer.Override(ctx, &st.SV, false)
1883-
bulk.YieldIfNoPacer.Override(ctx, &st.SV, false)
1880+
admission.YieldForElasticCPU.Override(ctx, &st.SV, false)
18841881

18851882
hts := &httpTestServer{}
18861883
hts.t.authentication = sw.authentication

0 commit comments

Comments
 (0)