diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index ac54363b1496..532c7bd5a0a2 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -213,30 +213,36 @@ func formatViewQueryTypesForDisplay( desc catalog.TableDescriptor, ) (string, error) { replaceFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { + // We need to resolve the type to check if it's user-defined. If not, + // no other work is needed. + var typRef tree.ResolvableTypeReference switch n := expr.(type) { - case *tree.AnnotateTypeExpr, *tree.CastExpr: - texpr, err := tree.TypeCheck(ctx, n, semaCtx, types.Any) - if err != nil { - return false, expr, err - } - if !texpr.ResolvedType().UserDefined() { - return true, expr, nil - } - - formattedExpr, err := schemaexpr.FormatExprForDisplay( - ctx, desc, expr.String(), semaCtx, sessionData, tree.FmtParsable, - ) - if err != nil { - return false, expr, err - } - newExpr, err = parser.ParseExpr(formattedExpr) - if err != nil { - return false, expr, err - } - return false, newExpr, nil + case *tree.CastExpr: + typRef = n.Type + case *tree.AnnotateTypeExpr: + typRef = n.Type default: return true, expr, nil } + var typ *types.T + typ, err = tree.ResolveType(ctx, typRef, semaCtx.TypeResolver) + if err != nil { + return false, expr, err + } + if !typ.UserDefined() { + return true, expr, nil + } + formattedExpr, err := schemaexpr.FormatExprForDisplay( + ctx, desc, expr.String(), semaCtx, sessionData, tree.FmtParsable, + ) + if err != nil { + return false, expr, err + } + newExpr, err = parser.ParseExpr(formattedExpr) + if err != nil { + return false, expr, err + } + return false, newExpr, nil } viewQuery := desc.GetViewQuery() diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 79f05ea4868d..30b8e9278272 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -3292,6 +3292,8 @@ 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", }, diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 7118806fa7b4..cc6f6b6ef5ee 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -337,21 +337,68 @@ type granterWithLockedCalls interface { getPairedRequester() requester } +// For the cpu-bound slot case we have background activities (like Pebble +// compactions) that would like to utilize additional slots if available (e.g. +// to do concurrent compression of ssblocks). These activities do not want to +// wait for a slot, since they can proceed without the slot at their usual +// slower pace (e.g. without doing concurrent compression). They also are +// sensitive to small overheads in their tight loops, and cannot afford the +// overhead of interacting with admission control at a fine granularity (like +// asking for a slot when compressing each ssblock). A coarse granularity +// interaction causes a delay in returning slots to admission control, and we +// don't want that delay to cause admission delay for normal work. Hence, we +// model slots granted to background activities as "soft-slots". Think of +// regular used slots as "hard-slots", in that we assume that the holder of +// the slot is still "using" it, while a soft-slot is "squishy" and in some +// cases we can pretend that it is not being used. Say we are allowed +// to allocate up to M slots. In this scheme, when allocating a soft-slot +// one must conform to usedSoftSlots+usedSlots <= M, and when allocating +// a regular (hard) slot one must conform to usedSlots <= M. +// +// That is, soft-slots allow for over-commitment until the soft-slots are +// returned, which may mean some additional queueing in the goroutine +// scheduler. +// +// We have another wrinkle in that we do not want to maintain a single M. For +// these optional background activities we desire to do them only when the +// load is low enough. This is because at high load, all work suffers from +// additional queueing in the goroutine scheduler. So we want to make sure +// regular work does not suffer such goroutine scheduler queueing because we +// granted too many soft-slots and caused CPU utilization to be high. So we +// maintain two kinds of M, totalHighLoadSlots and totalModerateLoadSlots. +// totalHighLoadSlots are estimated so as to allow CPU utilization to be high, +// while totalModerateLoadSlots are trying to keep queuing in the goroutine +// scheduler to a lower level. So the revised equations for allocation are: +// - Allocating a soft-slot: usedSoftSlots+usedSlots <= totalModerateLoadSlots +// - Allocating a regular slot: usedSlots <= totalHighLoadSlots +// +// NB: we may in the future add other kinds of background activities that do +// not have a lag in interacting with admission control, but want to schedule +// them only under moderate load. Those activities will be counted in +// usedSlots but when granting a slot to such an activity, the equation will +// be usedSoftSlots+usedSlots <= totalModerateLoadSlots. +// +// That is, let us not confuse that moderate load slot allocation is only for +// soft-slots. Soft-slots are introduced only for squishiness. +// // slotGranter implements granterWithLockedCalls. type slotGranter struct { - coord *GrantCoordinator - workKind WorkKind - requester requester - usedSlots int - totalSlots int - skipSlotEnforcement bool + coord *GrantCoordinator + workKind WorkKind + requester requester + usedSlots int + usedSoftSlots int + totalHighLoadSlots int + totalModerateLoadSlots int + skipSlotEnforcement bool // Optional. Nil for a slotGranter used for KVWork since the slots for that // slotGranter are directly adjusted by the kvSlotAdjuster (using the // kvSlotAdjuster here would provide a redundant identical signal). cpuOverload cpuOverloadIndicator - usedSlotsMetric *metric.Gauge + usedSlotsMetric *metric.Gauge + usedSoftSlotsMetric *metric.Gauge } var _ granterWithLockedCalls = &slotGranter{} @@ -375,7 +422,7 @@ func (sg *slotGranter) tryGetLocked(count int64) grantResult { if sg.cpuOverload != nil && sg.cpuOverload.isOverloaded() { return grantFailDueToSharedResource } - if sg.usedSlots < sg.totalSlots || sg.skipSlotEnforcement { + if sg.usedSlots < sg.totalHighLoadSlots || sg.skipSlotEnforcement { sg.usedSlots++ sg.usedSlotsMetric.Update(int64(sg.usedSlots)) return grantSuccess @@ -390,6 +437,32 @@ func (sg *slotGranter) returnGrant(count int64) { sg.coord.returnGrant(sg.workKind, count) } +func (sg *slotGranter) tryGetSoftSlots(count int) int { + sg.coord.mu.Lock() + defer sg.coord.mu.Unlock() + spareModerateLoadSlots := sg.totalModerateLoadSlots - sg.usedSoftSlots - sg.usedSlots + if spareModerateLoadSlots <= 0 { + return 0 + } + allocatedSlots := count + if allocatedSlots > spareModerateLoadSlots { + allocatedSlots = spareModerateLoadSlots + } + sg.usedSoftSlots += allocatedSlots + sg.usedSoftSlotsMetric.Update(int64(sg.usedSoftSlots)) + return allocatedSlots +} + +func (sg *slotGranter) returnSoftSlots(count int) { + sg.coord.mu.Lock() + defer sg.coord.mu.Unlock() + sg.usedSoftSlots -= count + sg.usedSoftSlotsMetric.Update(int64(sg.usedSoftSlots)) + if sg.usedSoftSlots < 0 { + panic("used soft slots is negative") + } +} + func (sg *slotGranter) returnGrantLocked(count int64) { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) @@ -633,8 +706,12 @@ var _ CPULoadListener = &GrantCoordinator{} // Options for constructing GrantCoordinators. type Options struct { - MinCPUSlots int - MaxCPUSlots int + 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 SQLKVResponseBurstTokens int64 SQLSQLResponseBurstTokens int64 SQLStatementLeafStartWorkSlots int @@ -720,10 +797,13 @@ func NewGrantCoordinators( metrics := makeGranterMetrics() metricStructs := append([]metric.Struct(nil), metrics) kvSlotAdjuster := &kvSlotAdjuster{ - settings: st, - minCPUSlots: opts.MinCPUSlots, - maxCPUSlots: opts.MaxCPUSlots, - totalSlotsMetric: metrics.KVTotalSlots, + settings: st, + minCPUSlots: opts.MinCPUSlots, + maxCPUSlots: opts.MaxCPUSlots, + totalSlotsMetric: metrics.KVTotalSlots, + totalModerateSlotsMetric: metrics.KVTotalModerateSlots, + moderateSlotsClamp: opts.MaxCPUSlots, + runnableAlphaOverride: opts.RunnableAlphaOverride, } coord := &GrantCoordinator{ ambientCtx: ambientCtx, @@ -737,11 +817,14 @@ func NewGrantCoordinators( } kvg := &slotGranter{ - coord: coord, - workKind: KVWork, - totalSlots: opts.MinCPUSlots, - usedSlotsMetric: metrics.KVUsedSlots, + coord: coord, + workKind: KVWork, + totalHighLoadSlots: opts.MinCPUSlots, + totalModerateLoadSlots: opts.MinCPUSlots, + usedSlotsMetric: metrics.KVUsedSlots, + usedSoftSlotsMetric: metrics.KVUsedSoftSlots, } + kvSlotAdjuster.granter = kvg coord.queues[KVWork] = makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) kvg.requester = coord.queues[KVWork] @@ -772,11 +855,11 @@ func NewGrantCoordinators( coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + coord: coord, + workKind: SQLStatementLeafStartWork, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx, SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) @@ -784,11 +867,11 @@ func NewGrantCoordinators( coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, + coord: coord, + workKind: SQLStatementRootStartWork, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, } coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx, SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) @@ -863,11 +946,11 @@ func NewGrantCoordinatorSQL( coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + coord: coord, + workKind: SQLStatementLeafStartWork, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx, SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork)) @@ -875,11 +958,11 @@ func NewGrantCoordinatorSQL( coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, + coord: coord, + workKind: SQLStatementRootStartWork, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, } coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx, SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork)) @@ -1213,15 +1296,20 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { case KVWork: switch g := coord.granters[i].(type) { case *slotGranter: - s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, - g.totalSlots) + 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) + } case *kvStoreTokenGranter: s.Printf(" io-avail: %d", g.availableIOTokens) } 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.totalSlots) + s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalHighLoadSlots) } case SQLKVResponseWork, SQLSQLResponseWork: if coord.granters[i] != nil { @@ -1353,6 +1441,7 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo useGrantChains: false, numProcs: 1, } + kvg := &kvStoreTokenGranter{ coord: coord, ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration, @@ -1454,19 +1543,48 @@ type kvSlotAdjuster struct { granter *slotGranter minCPUSlots int maxCPUSlots int - - totalSlotsMetric *metric.Gauge + // moderateSlotsClamp is the most recent value which may have been used to + // clamp down on slotGranter.totalModerateLoadSlots. Justification for + // clamping down on totalModerateLoadSlots is given where the moderateSlotsClamp + // value is written to. + moderateSlotsClamp int + // moderateSlotsClampOverride is used during testing to override the value of the + // moderateSlotsClamp. Its purpose is to make it easier to write tests. A default + // value of 0 implies no override. + moderateSlotsClampOverride int + // runnableEWMA is a weighted average of the most recent runnable goroutine counts. + // runnableEWMA is used to tune the slotGranter.totalModerateLoadSlots. + runnableEWMA float64 + // runnableAlphaOverride is used to override the value of runnable alpha during testing. + // A 0 value indicates that there is no override. + runnableAlphaOverride float64 + + totalSlotsMetric *metric.Gauge + totalModerateSlotsMetric *metric.Gauge } var _ cpuOverloadIndicator = &kvSlotAdjuster{} var _ CPULoadListener = &kvSlotAdjuster{} -func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, _ time.Duration) { +func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, samplePeriod time.Duration) { threshold := int(KVSlotAdjusterOverloadThreshold.Get(&kvsa.settings.SV)) + // 0.009 gives weight to at least a few hundred samples at a 1ms sampling rate. + alpha := 0.009 * float64(samplePeriod/time.Millisecond) + if alpha > 0.5 { + alpha = 0.5 + } else if alpha < 0.001 { + alpha = 0.001 + } + if kvsa.runnableAlphaOverride > 0 { + alpha = kvsa.runnableAlphaOverride + } + kvsa.runnableEWMA = kvsa.runnableEWMA*(1-alpha) + float64(runnable)*alpha + // Simple heuristic, which worked ok in experiments. More sophisticated ones // could be devised. - if runnable >= threshold*procs { + usedSlots := kvsa.granter.usedSlots + kvsa.granter.usedSoftSlots + tryDecreaseSlots := func(total int) int { // Overload. // If using some slots, and the used slots is less than the total slots, // and total slots hasn't bottomed out at the min, decrease the total @@ -1479,29 +1597,114 @@ func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, _ time.Duration) { // so it is suggests that the drop in slots should not be causing cpu // under-utilization, but one cannot be sure. Experiment with a smoothed // signal or other ways to prevent a fast drop. - if kvsa.granter.usedSlots > 0 && kvsa.granter.totalSlots > kvsa.minCPUSlots && - kvsa.granter.usedSlots <= kvsa.granter.totalSlots { - kvsa.granter.totalSlots-- + if usedSlots > 0 && total > kvsa.minCPUSlots && usedSlots <= total { + total-- } - } else if float64(runnable) <= float64((threshold*procs)/2) { + return total + } + tryIncreaseSlots := func(total int) int { // Underload. - // Used all its slots and can increase further, so additive increase. - if kvsa.granter.usedSlots >= kvsa.granter.totalSlots && - kvsa.granter.totalSlots < kvsa.maxCPUSlots { + // Used all its slots and can increase further, so additive increase. We + // also handle the case where the used slots are a bit less than total + // slots, since callers for soft slots don't block. + if usedSlots >= total && total < kvsa.maxCPUSlots { // NB: If the workload is IO bound, the slot count here will keep // incrementing until these slots are no longer the bottleneck for // admission. So it is not unreasonable to see this slot count go into // the 1000s. If the workload switches to being CPU bound, we can // decrease by 1000 slots every second (because the CPULoad ticks are at // 1ms intervals, and we do additive decrease). - kvsa.granter.totalSlots++ + total++ } + return total + } + + if runnable >= threshold*procs { + // Very overloaded. + kvsa.granter.totalHighLoadSlots = tryDecreaseSlots(kvsa.granter.totalHighLoadSlots) + kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) + } else if float64(runnable) <= float64((threshold*procs)/4) { + // Very underloaded. + kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) + kvsa.granter.totalModerateLoadSlots = tryIncreaseSlots(kvsa.granter.totalModerateLoadSlots) + } else if float64(runnable) <= float64((threshold*procs)/2) { + // Moderately underloaded -- can afford to increase regular slots. + kvsa.granter.totalHighLoadSlots = tryIncreaseSlots(kvsa.granter.totalHighLoadSlots) + } else if runnable >= 3*threshold*procs/4 { + // Moderately overloaded -- should decrease moderate load slots. + // + // NB: decreasing moderate load slots may not halt the runnable growth + // since the regular traffic may be high and can use up to the high load + // slots. When usedSlots>totalModerateLoadSlots, we won't actually + // decrease totalModerateLoadSlots (see the logic in tryDecreaseSlots). + // However, that doesn't mean that totalModerateLoadSlots is accurate. + // This inaccuracy is fine since we have chosen to be in a high load + // regime, since all the work we are doing is non-optional regular work + // (not background work). + // + // Where this will help is when what is pushing us over moderate load is + // optional background work, so by decreasing totalModerateLoadSlots we will + // contain the load due to that work. + kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots(kvsa.granter.totalModerateLoadSlots) + } + // Consider the following cases, when we started this method with + // totalHighLoadSlots==totalModerateLoadSlots. + // - underload such that we are able to increase totalModerateLoadSlots: in + // this case we will also be able to increase totalHighLoadSlots (since + // the used and total comparisons gating the increase in tryIncreaseSlots + // will also be true for totalHighLoadSlots). + // - overload such that we are able to decrease totalHighLoadSlots: in this + // case the logic in tryDecreaseSlots will also be able to decrease + // totalModerateLoadSlots. + // So the natural behavior of the slot adjustment itself guarantees + // totalHighLoadSlots >= totalModerateLoadSlots. But as a defensive measure + // we clamp totalModerateLoadSlots to not exceed totalHighLoadSlots. + if kvsa.granter.totalHighLoadSlots < kvsa.granter.totalModerateLoadSlots { + kvsa.granter.totalModerateLoadSlots = kvsa.granter.totalHighLoadSlots + } + + // During a kv50 workload, we noticed soft slots grants succeeding despite + // high cpu utilization, and high runnable goroutine counts. + // + // Consider the following log lines from the kv50 experiment: + // [runnable count 372 threshold*procs 256] + // [totalHighLoadSlots 254 totalModerateLoadSlots 164 usedSlots 0 usedSoftSlots 1] + // + // Note that even though the runnable count is high, of the (254, 164), + // (totalHighLoad, totalModerateLoad) slots respectively, only 1 slot is + // being used. The slot mechanism behaves in a bi-modal manner in nodes that + // do both KV and SQL processing. While there is backlogged KV work, the slot + // usage is high, and blocks all SQL work, but eventually all callers have done + // their KV processing and are queued up for SQL work. The latter causes bursts + // of grants (because it uses tokens), gated only by the grant-chain mechanism, + // during which runnable count is high but used (KV) slots are low. This is exactly + // the case where we have low slot usage, but high CPU utilization. + // + // We can afford to be more conservative in calculating totalModerateLoadSlots + // since we don't care about saturating CPU for the less important work that is + // controlled by these slots. So we could use a slow reacting and conservative + // signal to decide on the value of totalModerateLoadSlots. + // + // To account for the increased CPU utilization and runnable counts when the used + // slots are low, we clamp down on the totalModerateSlots value by keeping track + // of a historical runnable goroutine average. + kvsa.moderateSlotsClamp = int(float64(threshold*procs)/2 - kvsa.runnableEWMA) + if kvsa.moderateSlotsClampOverride != 0 { + kvsa.moderateSlotsClamp = kvsa.moderateSlotsClampOverride } - kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalSlots)) + if kvsa.granter.totalModerateLoadSlots > kvsa.moderateSlotsClamp { + kvsa.granter.totalModerateLoadSlots = kvsa.moderateSlotsClamp + } + if kvsa.granter.totalModerateLoadSlots < 0 { + kvsa.granter.totalModerateLoadSlots = 0 + } + + kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalHighLoadSlots)) + kvsa.totalModerateSlotsMetric.Update(int64(kvsa.granter.totalModerateLoadSlots)) } func (kvsa *kvSlotAdjuster) isOverloaded() bool { - return kvsa.granter.usedSlots >= kvsa.granter.totalSlots && !kvsa.granter.skipSlotEnforcement + return kvsa.granter.usedSlots >= kvsa.granter.totalHighLoadSlots && !kvsa.granter.skipSlotEnforcement } // sqlNodeCPUOverloadIndicator is the implementation of cpuOverloadIndicator @@ -2015,6 +2218,12 @@ var ( Measurement: "Slots", Unit: metric.Unit_COUNT, } + totalModerateSlots = metric.Metadata{ + Name: "admission.granter.total_moderate_slots.kv", + Help: "Total moderate load slots for low priority work", + Measurement: "Slots", + Unit: metric.Unit_COUNT, + } usedSlots = metric.Metadata{ // Note: we append a WorkKind string to this name. Name: "admission.granter.used_slots.", @@ -2022,6 +2231,12 @@ var ( Measurement: "Slots", Unit: metric.Unit_COUNT, } + usedSoftSlots = metric.Metadata{ + Name: "admission.granter.used_soft_slots.kv", + Help: "Used soft slots", + Measurement: "Slots", + Unit: metric.Unit_COUNT, + } kvIOTokensExhaustedDuration = metric.Metadata{ Name: "admission.granter.io_tokens_exhausted_duration.kv", Help: "Total duration when IO tokens were exhausted, in micros", @@ -2034,6 +2249,8 @@ var ( type GranterMetrics struct { KVTotalSlots *metric.Gauge KVUsedSlots *metric.Gauge + KVTotalModerateSlots *metric.Gauge + KVUsedSoftSlots *metric.Gauge KVIOTokensExhaustedDuration *metric.Counter SQLLeafStartUsedSlots *metric.Gauge SQLRootStartUsedSlots *metric.Gauge @@ -2046,6 +2263,8 @@ func makeGranterMetrics() GranterMetrics { m := GranterMetrics{ KVTotalSlots: metric.NewGauge(totalSlots), KVUsedSlots: metric.NewGauge(addName(string(workKindString(KVWork)), usedSlots)), + KVTotalModerateSlots: metric.NewGauge(totalModerateSlots), + KVUsedSoftSlots: metric.NewGauge(usedSoftSlots), KVIOTokensExhaustedDuration: metric.NewCounter(kvIOTokensExhaustedDuration), SQLLeafStartUsedSlots: metric.NewGauge( addName(string(workKindString(SQLStatementLeafStartWork)), usedSlots)), @@ -2067,3 +2286,32 @@ var _ = NewGrantCoordinatorSQL // uses the term "slot" for these is that we have a completion indicator, and // when we do have such an indicator it can be beneficial to be able to keep // track of how many ongoing work items we have. + +// SoftSlotGranter grants soft slots without queueing. See the comment with +// kvGranter. +type SoftSlotGranter struct { + kvGranter *slotGranter +} + +// MakeSoftSlotGranter constructs a SoftSlotGranter given a GrantCoordinator +// that is responsible for KV and lower layers. +func MakeSoftSlotGranter(gc *GrantCoordinator) (*SoftSlotGranter, error) { + kvGranter, ok := gc.granters[KVWork].(*slotGranter) + if !ok { + return nil, errors.Errorf("GrantCoordinator does not support soft slots") + } + return &SoftSlotGranter{ + kvGranter: kvGranter, + }, nil +} + +// TryGetSlots attempts to acquire count slots and returns what was acquired +// (possibly 0). +func (ssg *SoftSlotGranter) TryGetSlots(count int) int { + return ssg.kvGranter.tryGetSoftSlots(count) +} + +// ReturnSlots returns count slots (count must be >= 0). +func (ssg *SoftSlotGranter) ReturnSlots(count int) { + ssg.kvGranter.returnSoftSlots(count) +} diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go index 72ae85572118..61e7782c1409 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -90,19 +90,27 @@ func (tr *testRequester) setStoreRequestEstimates(estimates storeRequestEstimate // Only used by ioLoadListener, so don't bother. } +// setModerateSlotsClamp is used in testing to force a value for kvsa.moderateSlotsClamp. +func (kvsa *kvSlotAdjuster) setModerateSlotsClamp(val int) { + kvsa.moderateSlotsClampOverride = val + kvsa.moderateSlotsClamp = val +} + // TestGranterBasic is a datadriven test with the following commands: // // init-grant-coordinator min-cpu= max-cpu= sql-kv-tokens= -// sql-sql-tokens= sql-leaf= sql-root= +// sql-sql-tokens= sql-leaf= sql-root= // set-has-waiting-requests work= v= // set-return-value-from-granted work= v= // try-get work= [v=] // return-grant work= [v=] // took-without-permission work= [v=] // continue-grant-chain work= -// cpu-load runnable= procs= [infrequent=] +// cpu-load runnable= procs= [infrequent=] [clamp=] // init-store-grant-coordinator // set-io-tokens tokens= +// try-get-soft-slots slots= +// return-soft-slots slots= func TestGranterBasic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -110,6 +118,7 @@ func TestGranterBasic(t *testing.T) { var ambientCtx log.AmbientContext var requesters [numWorkKinds]*testRequester var coord *GrantCoordinator + var ssg *SoftSlotGranter clearRequesterAndCoord := func() { coord = nil for i := range requesters { @@ -154,8 +163,12 @@ func TestGranterBasic(t *testing.T) { return req } delayForGrantChainTermination = 0 + opts.RunnableAlphaOverride = 1 // This gives weight to only the most recent sample. coords, _ := NewGrantCoordinators(ambientCtx, opts) coord = coords.Regular + var err error + ssg, err = MakeSoftSlotGranter(coord) + require.NoError(t, err) return flushAndReset() case "init-store-grant-coordinator": @@ -236,6 +249,13 @@ func TestGranterBasic(t *testing.T) { if d.HasArg("infrequent") { d.ScanArgs(t, "infrequent", &infrequent) } + if d.HasArg("clamp") { + var clamp int + d.ScanArgs(t, "clamp", &clamp) + kvsa := coord.cpuLoadListener.(*kvSlotAdjuster) + kvsa.setModerateSlotsClamp(clamp) + } + samplePeriod := time.Millisecond if infrequent { samplePeriod = 250 * time.Millisecond @@ -254,6 +274,19 @@ func TestGranterBasic(t *testing.T) { coord.testingTryGrant() return flushAndReset() + case "try-get-soft-slots": + var slots int + d.ScanArgs(t, "slots", &slots) + granted := ssg.TryGetSlots(slots) + fmt.Fprintf(&buf, "requested: %d, granted: %d\n", slots, granted) + return flushAndReset() + + case "return-soft-slots": + var slots int + d.ScanArgs(t, "slots", &slots) + ssg.ReturnSlots(slots) + return flushAndReset() + default: return fmt.Sprintf("unknown command: %s", d.Cmd) } diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index 763422c4d3c1..6667eeb6aa3a 100644 --- a/pkg/util/admission/testdata/granter +++ b/pkg/util/admission/testdata/granter @@ -1,14 +1,14 @@ init-grant-coordinator min-cpu=1 max-cpu=3 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # No more slots. @@ -16,13 +16,13 @@ try-get work=kv ---- kv: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-has-waiting-requests work=kv v=true ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Since no more KV slots, couldn't get. @@ -30,13 +30,13 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-has-waiting-requests work=sql-kv-response v=true ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Since no more KV slots, couldn't get. @@ -44,13 +44,13 @@ try-get work=sql-leaf-start ---- sql-leaf-start: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-has-waiting-requests work=sql-leaf-start v=true ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Since no more KV slots, couldn't get. @@ -58,13 +58,13 @@ try-get work=sql-root-start ---- sql-root-start: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-has-waiting-requests work=sql-root-start v=true ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 return-grant work=kv @@ -72,19 +72,19 @@ return-grant work=kv kv: returnGrant(1) kv: granted in chain 1, and returning 1 GrantCoordinator: -(chain: id: 1 active: true index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-has-waiting-requests work=kv v=false ---- GrantCoordinator: -(chain: id: 1 active: true index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 set-return-value-from-granted work=kv v=0 ---- GrantCoordinator: -(chain: id: 1 active: true index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # The grant chain dies out since kv slots are fully used. @@ -92,7 +92,7 @@ continue-grant-chain work=kv ---- kv: continueGrantChain GrantCoordinator: -(chain: id: 2 active: false index: 1) kv: used: 1, total: 1 sql-kv-response: avail: 2 +(chain: id: 2 active: false index: 1) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Grant to sql-kv-response consumes a token. @@ -101,7 +101,7 @@ return-grant work=kv kv: returnGrant(1) sql-kv-response: granted in chain 2, and returning 1 GrantCoordinator: -(chain: id: 2 active: true index: 1) kv: used: 0, total: 1 sql-kv-response: avail: 1 +(chain: id: 2 active: true index: 1) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Grant to sql-kv-response consumes another token. None left. @@ -110,7 +110,7 @@ continue-grant-chain work=sql-kv-response sql-kv-response: continueGrantChain sql-kv-response: granted in chain 2, and returning 1 GrantCoordinator: -(chain: id: 2 active: true index: 1) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 1) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 # Even though there are still waiting requests for sql-kv-response, no more @@ -120,7 +120,7 @@ continue-grant-chain work=sql-kv-response sql-kv-response: continueGrantChain sql-leaf-start: granted in chain 2, and returning 1 GrantCoordinator: -(chain: id: 2 active: true index: 3) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 3) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 0, total: 1 continue-grant-chain work=sql-leaf-start @@ -128,7 +128,7 @@ continue-grant-chain work=sql-leaf-start sql-leaf-start: continueGrantChain sql-leaf-start: granted in chain 2, and returning 1 GrantCoordinator: -(chain: id: 2 active: true index: 3) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 3) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 0, total: 1 # Even though there are still waiting requests for sql-leaf-start, no more @@ -138,7 +138,7 @@ continue-grant-chain work=sql-leaf-start sql-leaf-start: continueGrantChain sql-root-start: granted in chain 2, and returning 1 GrantCoordinator: -(chain: id: 2 active: true index: 4) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 4) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # sql-root-start ran out of tokens. Grant chain dies out. @@ -146,7 +146,7 @@ continue-grant-chain work=sql-root-start ---- sql-root-start: continueGrantChain GrantCoordinator: -(chain: id: 3 active: false index: 5) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 3 active: false index: 5) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # Return sql-leaf-start slot. This will cause another grant chain to start @@ -156,7 +156,7 @@ return-grant work=sql-leaf-start sql-leaf-start: returnGrant(1) sql-leaf-start: granted in chain 3, and returning 1 GrantCoordinator: -(chain: id: 3 active: true index: 3) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # Return another sql-leaf-start slot. The grant chain is already active and @@ -165,7 +165,7 @@ return-grant work=sql-leaf-start ---- sql-leaf-start: returnGrant(1) GrantCoordinator: -(chain: id: 3 active: true index: 3) kv: used: 0, total: 1 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # The kv slots are fully used after this tryGet, which succeeds. @@ -173,7 +173,7 @@ try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 3 active: true index: 3) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # This tryGet for kv fails and forces termination of the grant chain. @@ -181,13 +181,13 @@ try-get work=kv ---- kv: tryGet(1) returned false GrantCoordinator: -(chain: id: 4 active: false index: 3) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 set-has-waiting-requests work=kv v=true ---- GrantCoordinator: -(chain: id: 4 active: false index: 3) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # The grant chain cannot continue since it was force terminated, and a new one @@ -196,14 +196,14 @@ continue-grant-chain work=sql-leaf-start ---- sql-leaf-start: continueGrantChain GrantCoordinator: -(chain: id: 4 active: false index: 3) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Pretend that the kv work that was waiting is gone. set-has-waiting-requests work=kv v=false ---- GrantCoordinator: -(chain: id: 4 active: false index: 3) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Some other kv work takes without permission. @@ -211,14 +211,14 @@ took-without-permission work=kv ---- kv: tookWithoutPermission(1) GrantCoordinator: -(chain: id: 4 active: false index: 3) kv: used: 2, total: 1 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 2, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Refill the tokens and increase the kv slots to 2. cpu-load runnable=0 procs=1 ---- GrantCoordinator: -(chain: id: 4 active: false index: 1) kv: used: 2, total: 2 sql-kv-response: avail: 2 +(chain: id: 4 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: 0 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Tokens don't get overfull. And kv slots increased to 3. This causes a grant @@ -227,14 +227,14 @@ cpu-load runnable=0 procs=1 ---- sql-kv-response: granted in chain 4, and returning 1 GrantCoordinator: -(chain: id: 4 active: true index: 1) kv: used: 2, total: 3 sql-kv-response: avail: 1 +(chain: id: 4 active: true index: 1) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Overload and kv slots decreased. Forces termination of grant chain 4. cpu-load runnable=2 procs=1 ---- GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, total: 2 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Grant chain 4 terminates. @@ -242,7 +242,7 @@ continue-grant-chain work=sql-kv-response ---- sql-kv-response: continueGrantChain GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, total: 2 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 1, total: 2 sql-root-start: used: 1, total: 1 # Return a slot for sql-leaf-start. Grant chain cannot start since KV slots @@ -251,7 +251,7 @@ return-grant work=sql-leaf-start ---- sql-leaf-start: returnGrant(1) GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, total: 2 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 1, total: 1 # Underload and kv slots increased. The number of procs=4, so can grant 4 at @@ -264,7 +264,7 @@ sql-kv-response: granted in chain 0, and returning 1 sql-leaf-start: granted in chain 0, and returning 1 sql-leaf-start: granted in chain 5, and returning 1 GrantCoordinator: -(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # There is now a free sql-root-start slot, which the grant chain will get to. @@ -272,7 +272,7 @@ return-grant work=sql-root-start ---- sql-root-start: returnGrant(1) GrantCoordinator: -(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 0, total: 1 # Continuing with chain-id=0 has no effect. @@ -280,7 +280,7 @@ continue-grant-chain work=sql-kv-response ---- sql-kv-response: continueGrantChain GrantCoordinator: -(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 0, total: 1 # Continuing chain-id=5 causes a grant to sql-root-start and the chain dies @@ -290,7 +290,7 @@ continue-grant-chain work=sql-leaf-start sql-leaf-start: continueGrantChain sql-root-start: granted in chain 0, and returning 1 GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 2, total: 3 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 ##################################################################### @@ -298,7 +298,7 @@ sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: use init-grant-coordinator min-cpu=1 max-cpu=3 sql-kv-tokens=1 sql-sql-tokens=1 sql-leaf=2 sql-root=2 ---- GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, total: 1 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # No more slots after this slot is granted. @@ -306,7 +306,7 @@ try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # Since no more KV slots, cannot grant token to sql-kv-response. @@ -314,7 +314,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # Since no more KV slots, cannot grant token to sql-sql-response. @@ -322,7 +322,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet(1) returned false GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, total: 1 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # CPULoad shows overload, so cannot increase KV slots, but since it is @@ -330,7 +330,7 @@ sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: use cpu-load runnable=20 procs=1 infrequent=true ---- GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, total: 1 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: 1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # sql-kv-response can get a token. @@ -338,7 +338,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, total: 1 sql-kv-response: avail: 0 +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # sql-kv-response can get another token, even though tokens are exhausted. @@ -346,7 +346,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, total: 1 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # sql-sql-response can get a token. @@ -354,7 +354,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, total: 1 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 sql-sql-response: avail: 0 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # sql-sql-response can get another token, even though tokens are exhausted. @@ -362,7 +362,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, total: 1 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 # KV can get another slot even though slots are exhausted. @@ -370,7 +370,7 @@ try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, total: 1 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 ##################################################################### @@ -461,3 +461,320 @@ kv: returnGrant(1) kv: granted in chain 0, and returning 100 GrantCoordinator: (chain: id: 0 active: false index: 5) io-avail: -99 + +##################################################################### +# Test soft slots +init-grant-coordinator min-cpu=1 max-cpu=6 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 0 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +return-soft-slots slots=1 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=0 procs=4 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# Over-commitment. +try-get work=kv +---- +kv: tryGet(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# Over-commitment. +try-get work=kv +---- +kv: tryGet(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get work=kv +---- +kv: tryGet(1) returned false +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# No longer over-committed. +return-soft-slots slots=2 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 0 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=4 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(2) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 0 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(2) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=1 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=3 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +return-grant work=kv +---- +kv: returnGrant(1) +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=2 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=2 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=2 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(5) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=6 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(4) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=6 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(3) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=6 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(3) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +##################################################################### +# Test soft slots is not higher than regular slots. +init-grant-coordinator min-cpu=1 max-cpu=6 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=1 +---- +requested: 1, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=2 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=1 +---- +requested: 1, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=2 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +return-soft-slots slots=2 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=10 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get work=kv +---- +kv: tryGet(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get work=kv +---- +kv: tryGet(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get work=kv +---- +kv: tryGet(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 3, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=10 procs=8 clamp=100 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 3, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +##################################################################### +# Test clamping down on total moderate slots. +init-grant-coordinator min-cpu=1 max-cpu=100 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 100 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=1 +---- +requested: 1, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=0 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 4 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=1 +---- +requested: 1, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 4 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +cpu-load runnable=0 procs=2 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(1) moderate-clamp: 1 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# The moderate slots clamp is set to -1, because 10/2-6=-1. +cpu-load runnable=6 procs=10 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(0) moderate-clamp: -1 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# Making sure total moderate slots is set to 0, and not a negative value. +cpu-load runnable=6 procs=10 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(0) moderate-clamp: -1 used-soft: 2 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +##################################################################### +# Make sure the moderate slots clamp has no effect when it is higher than the total moderate slots. +init-grant-coordinator min-cpu=1 max-cpu=3 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +try-get-soft-slots slots=1 +---- +requested: 1, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 + +# Should increase total moderate slots to 2, since the moderate clamp was 3 which is higher. +cpu-load runnable=0 procs=2 clamp=3 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 3 used-soft: 1 sql-kv-response: avail: 2 +sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1