Skip to content

Commit

Permalink
Merge #95590
Browse files Browse the repository at this point in the history
95590: admission: remove soft/moderate load slots r=irfansharif a=irfansharif

We originally introduced these notions in admission control (#78519) for additional threads for Pebble compaction compression. We envisioned granting these "squishy" slots to background activities and permit work only under periods of low load. In working through #86638 (as part of \#75066), we observed experimentally that the moderate-slots count was not sensitive enough to scheduling latency, and consequently latency observed by foreground traffic. Elastic CPU tokens, the kind now being used for backups, offer an alternative to soft slots. We've since replaced uses of soft slots with elastic CPU tokens.

This PR just removes the now dead-code code around soft/moderate load slots (it's better to minimize the number of mechanisms in the admission package). Fixes #88032.

Release note: None

---

First commit is from #95007.

Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
craig[bot] and irfansharif committed Jan 23, 2023
2 parents 1a4bcbf + f39eefb commit 479e5aa
Show file tree
Hide file tree
Showing 8 changed files with 104 additions and 776 deletions.
23 changes: 0 additions & 23 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/ts"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -343,10 +342,6 @@ type KVConfig struct {
// The value is split evenly between the stores if there are more than one.
CacheSize int64

// SoftSlotGranter can be optionally passed into a store to allow the store
// to perform additional CPU bound work.
SoftSlotGranter *admission.SoftSlotGranter

// TimeSeriesServerConfig contains configuration specific to the time series
// server.
TimeSeriesServerConfig ts.ServerConfig
Expand Down Expand Up @@ -632,19 +627,6 @@ func (e *Engines) Close() {
*e = nil
}

// cpuWorkPermissionGranter implements the pebble.CPUWorkPermissionGranter
// interface.
//type cpuWorkPermissionGranter struct {
//*admission.SoftSlotGranter
//}

//func (c *cpuWorkPermissionGranter) TryGetProcs(count int) int {
//return c.TryGetSlots(count)
//}
//func (c *cpuWorkPermissionGranter) ReturnProcs(count int) {
//c.ReturnSlots(count)
//}

// CreateEngines creates Engines based on the specs in cfg.Stores.
func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
engines := Engines(nil)
Expand Down Expand Up @@ -775,11 +757,6 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
pebbleConfig.Opts.TableCache = tableCache
pebbleConfig.Opts.MaxOpenFiles = int(openFileLimitPerStore)
pebbleConfig.Opts.Experimental.MaxWriterConcurrency = 2
// TODO(jackson): Implement the new pebble.CPUWorkPermissionGranter
// interface.
//pebbleConfig.Opts.Experimental.CPUWorkPermissionGranter = &cpuWorkPermissionGranter{
//cfg.SoftSlotGranter,
//}
if storeKnobs.SmallEngineBlocks {
for i := range pebbleConfig.Opts.Levels {
pebbleConfig.Opts.Levels[i].BlockSize = 1
Expand Down
5 changes: 0 additions & 5 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,11 +246,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
admissionOptions.Override(opts)
}
gcoords := admission.NewGrantCoordinators(cfg.AmbientCtx, st, admissionOptions, registry)
ssg, err := admission.MakeSoftSlotGranter(gcoords.Regular)
if err != nil {
return nil, errors.Wrap(err, "failed to soft slot granter")
}
cfg.SoftSlotGranter = ssg

engines, err := cfg.CreateEngines(ctx)
if err != nil {
Expand Down
2 changes: 0 additions & 2 deletions pkg/ts/catalog/chart_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -3748,8 +3748,6 @@ var charts = []sectionDescription{
Metrics: []string{
"admission.granter.total_slots.kv",
"admission.granter.used_slots.kv",
"admission.granter.total_moderate_slots.kv",
"admission.granter.used_soft_slots.kv",
"admission.granter.used_slots.sql-leaf-start",
"admission.granter.used_slots.sql-root-start",
},
Expand Down
75 changes: 27 additions & 48 deletions pkg/util/admission/grant_coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,12 +272,8 @@ var _ CPULoadListener = &GrantCoordinator{}

// Options for constructing GrantCoordinators.
type Options struct {
MinCPUSlots int
MaxCPUSlots int
// RunnableAlphaOverride is used to override the alpha value used to
// compute the ewma of the runnable goroutine counts. It is only used
// during testing. A 0 value indicates that there is no override.
RunnableAlphaOverride float64
MinCPUSlots int
MaxCPUSlots int
SQLKVResponseBurstTokens int64
SQLSQLResponseBurstTokens int64
SQLStatementLeafStartWorkSlots int
Expand Down Expand Up @@ -434,10 +430,7 @@ func makeRegularGrantCoordinator(
settings: st,
minCPUSlots: opts.MinCPUSlots,
maxCPUSlots: opts.MaxCPUSlots,
moderateSlotsClamp: opts.MaxCPUSlots,
runnableAlphaOverride: opts.RunnableAlphaOverride,
totalSlotsMetric: metrics.KVTotalSlots,
totalModerateSlotsMetric: metrics.KVTotalModerateSlots,
cpuLoadShortPeriodDurationMetric: metrics.KVCPULoadShortPeriodDuration,
cpuLoadLongPeriodDurationMetric: metrics.KVCPULoadLongPeriodDuration,
slotAdjusterIncrementsMetric: metrics.KVSlotAdjusterIncrements,
Expand All @@ -457,10 +450,8 @@ func makeRegularGrantCoordinator(
kvg := &slotGranter{
coord: coord,
workKind: KVWork,
totalHighLoadSlots: opts.MinCPUSlots,
totalModerateLoadSlots: opts.MinCPUSlots,
totalSlots: opts.MinCPUSlots,
usedSlotsMetric: metrics.KVUsedSlots,
usedSoftSlotsMetric: metrics.KVUsedSoftSlots,
slotsExhaustedDurationMetric: metrics.KVSlotsExhaustedDuration,
}

Expand Down Expand Up @@ -500,11 +491,11 @@ func makeRegularGrantCoordinator(
coord.granters[SQLSQLResponseWork] = tg

sg := &slotGranter{
coord: coord,
workKind: SQLStatementLeafStartWork,
totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots,
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
coord: coord,
workKind: SQLStatementLeafStartWork,
totalSlots: opts.SQLStatementLeafStartWorkSlots,
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry, admissionpb.NormalPri, admissionpb.LockingPri)
req = makeRequester(ambientCtx,
Expand All @@ -514,11 +505,11 @@ func makeRegularGrantCoordinator(
coord.granters[SQLStatementLeafStartWork] = sg

sg = &slotGranter{
coord: coord,
workKind: SQLStatementRootStartWork,
totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots,
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
coord: coord,
workKind: SQLStatementRootStartWork,
totalSlots: opts.SQLStatementRootStartWorkSlots,
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry, admissionpb.NormalPri, admissionpb.LockingPri)
req = makeRequester(ambientCtx,
Expand Down Expand Up @@ -585,11 +576,11 @@ func NewGrantCoordinatorSQL(
coord.granters[SQLSQLResponseWork] = tg

sg := &slotGranter{
coord: coord,
workKind: SQLStatementLeafStartWork,
totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots,
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
coord: coord,
workKind: SQLStatementLeafStartWork,
totalSlots: opts.SQLStatementLeafStartWorkSlots,
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry)
req = makeRequester(ambientCtx,
Expand All @@ -599,11 +590,11 @@ func NewGrantCoordinatorSQL(
coord.granters[SQLStatementLeafStartWork] = sg

sg = &slotGranter{
coord: coord,
workKind: SQLStatementRootStartWork,
totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots,
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
coord: coord,
workKind: SQLStatementRootStartWork,
totalSlots: opts.SQLStatementRootStartWorkSlots,
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry)
req = makeRequester(ambientCtx,
Expand Down Expand Up @@ -919,21 +910,15 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) {
case KVWork:
switch g := coord.granters[i].(type) {
case *slotGranter:
kvsa := coord.cpuLoadListener.(*kvSlotAdjuster)
s.Printf(
"%s%s: used: %d, high(moderate)-total: %d(%d) moderate-clamp: %d", curSep, workKindString(kind),
g.usedSlots, g.totalHighLoadSlots, g.totalModerateLoadSlots, kvsa.moderateSlotsClamp)
if g.usedSoftSlots > 0 {
s.Printf(" used-soft: %d", g.usedSoftSlots)
}
s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots)
case *kvStoreTokenGranter:
s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens,
g.elasticDiskBWTokensAvailable)
}
case SQLStatementLeafStartWork, SQLStatementRootStartWork:
if coord.granters[i] != nil {
g := coord.granters[i].(*slotGranter)
s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalHighLoadSlots)
s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots)
}
case SQLKVResponseWork, SQLSQLResponseWork:
if coord.granters[i] != nil {
Expand All @@ -953,8 +938,6 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) {
type GrantCoordinatorMetrics struct {
KVTotalSlots *metric.Gauge
KVUsedSlots *metric.Gauge
KVTotalModerateSlots *metric.Gauge
KVUsedSoftSlots *metric.Gauge
KVSlotsExhaustedDuration *metric.Counter
KVCPULoadShortPeriodDuration *metric.Counter
KVCPULoadLongPeriodDuration *metric.Counter
Expand All @@ -970,12 +953,8 @@ func (GrantCoordinatorMetrics) MetricStruct() {}

func makeGrantCoordinatorMetrics() GrantCoordinatorMetrics {
m := GrantCoordinatorMetrics{
KVTotalSlots: metric.NewGauge(totalSlots),
KVUsedSlots: metric.NewGauge(addName(workKindString(KVWork), usedSlots)),
// TODO(sumeer): remove moderate load slots and soft slots code and
// metrics #88032.
KVTotalModerateSlots: metric.NewGauge(totalModerateSlots),
KVUsedSoftSlots: metric.NewGauge(usedSoftSlots),
KVTotalSlots: metric.NewGauge(totalSlots),
KVUsedSlots: metric.NewGauge(addName(workKindString(KVWork), usedSlots)),
KVSlotsExhaustedDuration: metric.NewCounter(kvSlotsExhaustedDuration),
KVCPULoadShortPeriodDuration: metric.NewCounter(kvCPULoadShortPeriodDuration),
KVCPULoadLongPeriodDuration: metric.NewCounter(kvCPULoadLongPeriodDuration),
Expand Down
Loading

0 comments on commit 479e5aa

Please sign in to comment.