diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 40d2a098b435..e1cb8e6953b2 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -451,12 +451,36 @@ func (tg *tokenGranter) continueGrantChain(grantChainID grantChainID) { // kvGranter implements granterWithLockedCalls. It is used for grants to // KVWork, that are limited by slots (CPU bound work) and/or tokens (IO // bound work). +// +// In production, a kvGranter is doing either slots (ioTokensEnabled=false), +// or tokens (totalSlots=MaxInt). The former is used for (typically) cpu-bound +// work (KV and storage layer) across the whole node, and latter for the +// per-store write admission control (see StoreGrantCoordinators). +// +// +// 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. They also are performance sensitive, and can't afford to +// interact 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". Granting a soft-slot has +// to conform to usedSoftSlots+usedSlots <= totalSlots. Granting a regular +// slot only has to conform to usedSlots <= totalSlots. That is, soft-slots +// allow for over-commitment until the soft-slots are returned, which may mean +// some additional queueing in the goroutine scheduler. type kvGranter struct { coord *GrantCoordinator requester requester usedSlots int + usedSoftSlots int totalSlots int + totalSoftSlots int skipSlotEnforcement bool + failedSoftSlotsGet bool ioTokensEnabled bool // There is no rate limiting in granting these tokens. That is, they are all @@ -555,6 +579,35 @@ func (sg *kvGranter) setAvailableIOTokensLocked(tokens int64) { } } +func (sg *kvGranter) tryGetSoftSlots(count int) int { + sg.coord.mu.Lock() + defer sg.coord.mu.Unlock() + spareSoftSlots := sg.totalSoftSlots - sg.usedSoftSlots + spareSlots := sg.totalSlots - (sg.usedSlots + sg.usedSoftSlots) + if spareSlots < spareSoftSlots { + spareSoftSlots = spareSlots + } + if spareSoftSlots <= 0 { + sg.failedSoftSlotsGet = true + return 0 + } + allocatedSlots := count + if allocatedSlots > spareSoftSlots { + allocatedSlots = spareSoftSlots + } + sg.usedSoftSlots += allocatedSlots + return allocatedSlots +} + +func (sg *kvGranter) returnSoftSlots(count int) { + sg.coord.mu.Lock() + defer sg.coord.mu.Unlock() + sg.usedSoftSlots -= count + if sg.usedSoftSlots < 0 { + panic("used soft slots is negative") + } +} + // GrantCoordinator is the top-level object that coordinates grants across // different WorkKinds (for more context see the comment in doc.go, and the // comment where WorkKind is declared). Typically there will one @@ -717,6 +770,7 @@ func NewGrantCoordinators( kvg := &kvGranter{ coord: coord, totalSlots: opts.MinCPUSlots, + totalSoftSlots: opts.MinCPUSlots, usedSlotsMetric: metrics.KVUsedSlots, } kvSlotAdjuster.granter = kvg @@ -1175,7 +1229,11 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { switch kind { case KVWork: g := coord.granters[i].(*kvGranter) - s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots) + s.Printf("%s%s: used: %d, total(soft): %d(%d)", curSep, workKindString(kind), + g.usedSlots, g.totalSlots, g.totalSoftSlots) + if g.usedSoftSlots > 0 { + s.Printf(" used-soft: %d", g.usedSoftSlots) + } if g.ioTokensEnabled { s.Printf(" io-avail: %d", g.availableIOTokens) } @@ -1398,7 +1456,8 @@ func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, _ time.Duration) { // 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(used int, 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 @@ -1411,24 +1470,71 @@ 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 used > 0 && total > kvsa.minCPUSlots && used <= total { + total-- } - } else if float64(runnable) <= float64((threshold*procs)/2) { + return total + } + tryIncreaseSlots := func(used int, total int) int { + // TODO: 0.8 is arbitrary. + closeToTotalSlots := int(float64(total) * 0.8) // 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 (used >= total || (used >= closeToTotalSlots && kvsa.granter.failedSoftSlotsGet)) && + 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 + } + // NB: usedSlots >= kvGranter.usedSoftSlots. Consider the following cases, when + // totalSlots == totalSoftSlots. + // - underload such that we are able to increase totalSoftSlots: in this + // case we will also be able to increase totalSlots (since the used and + // total comparisons gating the increase in tryIncreaseSlots will also be + // true for totalSlots). + // - overload such that we are able to decrease totalSlots: in this case it + // is possible that we don't decrease the totalSoftSlots because of the + // used > 0 condition in tryDecreaseSlots. + + // So the natural behavior of the slot adjustments does not guarantee + // totalSlots >= totalSoftSlots. We add logic to impose this on top of the + // natural adjustment. + // + // TODO: the fractions below are arbitrary and subject to tuning. + if runnable >= threshold*procs { + // Very overloaded. + kvsa.granter.totalSlots = tryDecreaseSlots(usedSlots, kvsa.granter.totalSlots) + kvsa.granter.totalSoftSlots = tryDecreaseSlots( + kvsa.granter.usedSoftSlots, kvsa.granter.totalSoftSlots) + } else if float64(runnable) <= float64((threshold*procs)/4) { + // Very underloaded. + kvsa.granter.totalSlots = tryIncreaseSlots(usedSlots, kvsa.granter.totalSlots) + kvsa.granter.totalSoftSlots = tryIncreaseSlots( + kvsa.granter.usedSoftSlots, kvsa.granter.totalSoftSlots) + } else if float64(runnable) <= float64((threshold*procs)/2) { + // Moderately underloaded -- can afford to increase regular slots. + kvsa.granter.totalSlots = tryIncreaseSlots(usedSlots, kvsa.granter.totalSlots) + } else if runnable >= 3*threshold*procs/4 { + // Moderately overloaded -- should decrease soft slots. + // NB: decreasing soft slots may not halt the runnable growth since the + // regular slot traffic may be high. Which means we will keep decreasing + // soft slots and undershoot. This is acceptable since soft slots are + // strictly best-effort. + kvsa.granter.totalSoftSlots = tryDecreaseSlots( + kvsa.granter.usedSoftSlots, kvsa.granter.totalSoftSlots) + } + if kvsa.granter.totalSlots < kvsa.granter.totalSoftSlots { + kvsa.granter.totalSoftSlots = kvsa.granter.totalSlots + } + kvsa.granter.failedSoftSlotsGet = false kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalSlots)) } @@ -1755,3 +1861,33 @@ var _ = (*GrantCoordinator)(nil).GetWorkQueue // 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 *kvGranter +} + +// 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].(*kvGranter) + 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). +// TODO: experiment with calling this when opening an sstable.Writer. +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 fd533a013363..5b0b79428520 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -96,6 +96,7 @@ func TestGranterBasic(t *testing.T) { var ambientCtx log.AmbientContext var requesters [numWorkKinds]*testRequester var coord *GrantCoordinator + var ssg *SoftSlotGranter var buf strings.Builder flushAndReset := func() string { fmt.Fprintf(&buf, "GrantCoordinator:\n%s\n", coord.String()) @@ -131,6 +132,9 @@ func TestGranterBasic(t *testing.T) { delayForGrantChainTermination = 0 coords, _ := NewGrantCoordinators(ambientCtx, opts) coord = coords.Regular + var err error + ssg, err = MakeSoftSlotGranter(coord) + require.NoError(t, err) return flushAndReset() case "set-has-waiting-requests": @@ -187,6 +191,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 c68edae0ceb8..783e97ec7ccd 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, total(soft): 1(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 work=kv ---- kv: tryGet 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, total(soft): 1(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 # No more slots. @@ -16,13 +16,13 @@ try-get work=kv ---- kv: tryGet 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, total(soft): 1(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 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, total(soft): 1(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 # Since no more KV slots, couldn't get. @@ -30,13 +30,13 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet 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, total(soft): 1(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 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, total(soft): 1(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 # Since no more KV slots, couldn't get. @@ -44,13 +44,13 @@ try-get work=sql-leaf-start ---- sql-leaf-start: tryGet 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, total(soft): 1(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 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, total(soft): 1(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 # Since no more KV slots, couldn't get. @@ -58,13 +58,13 @@ try-get work=sql-root-start ---- sql-root-start: tryGet 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, total(soft): 1(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 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, total(soft): 1(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 @@ -72,19 +72,19 @@ return-grant work=kv kv: returnGrant kv: granted in chain 1, and returning true 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, total(soft): 1(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 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, total(soft): 1(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 set-return-value-from-granted 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, total(soft): 1(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 # 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, total(soft): 1(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 # Grant to sql-kv-response consumes a token. @@ -101,7 +101,7 @@ return-grant work=kv kv: returnGrant sql-kv-response: granted in chain 2, and returning true 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, total(soft): 1(1) 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 true 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, total(soft): 1(1) 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 true 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, total(soft): 1(1) 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 true 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, total(soft): 1(1) 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 true 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, total(soft): 1(1) 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, total(soft): 1(1) 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 sql-leaf-start: granted in chain 3, and returning true 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, total(soft): 1(1) 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 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, total(soft): 1(1) 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 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, total(soft): 1(1) 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 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, total(soft): 1(1) 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, total(soft): 1(1) 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, total(soft): 1(1) 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, total(soft): 1(1) 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 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, total(soft): 1(1) 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, total(soft): 2(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 # 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 true 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, total(soft): 3(1) 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, total(soft): 2(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, total(soft): 2(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 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, total(soft): 2(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 true sql-leaf-start: granted in chain 0, and returning true sql-leaf-start: granted in chain 5, and returning true 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, total(soft): 3(1) 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 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, total(soft): 3(1) 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, total(soft): 3(1) 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,14 +290,14 @@ continue-grant-chain work=sql-leaf-start sql-leaf-start: continueGrantChain sql-root-start: granted in chain 0, and returning true 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, total(soft): 3(1) sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # Start restricting IO tokens for KV. set-io-tokens tokens=1 ---- GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 2, total: 3 io-avail: 1 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 2, total(soft): 3(1) io-avail: 1 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 both slots currently used by KV, so that 3 slots are free, but there @@ -306,14 +306,14 @@ return-grant work=kv ---- kv: returnGrant GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 1, total: 3 io-avail: 1 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 1, total(soft): 3(1) io-avail: 1 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-grant work=kv ---- kv: returnGrant GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 0, total: 3 io-avail: 1 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 0, total(soft): 3(1) io-avail: 1 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # Takes 1 slot and 1 token. @@ -321,7 +321,7 @@ try-get work=kv ---- kv: tryGet returned true GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 1, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 1, total(soft): 3(1) io-avail: 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 are 2 slots available, but no tokens, so fails. @@ -329,19 +329,19 @@ try-get work=kv ---- kv: tryGet returned false GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 1, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 1, total(soft): 3(1) io-avail: 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 set-has-waiting-requests work=kv v=true ---- GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 1, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 1, total(soft): 3(1) io-avail: 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 set-return-value-from-granted work=kv v=true ---- GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 1, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 1, total(soft): 3(1) io-avail: 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 # Tokens become negative. @@ -349,14 +349,14 @@ took-without-permission work=kv ---- kv: tookWithoutPermission GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 2, total: 3 io-avail: -1 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 2, total(soft): 3(1) io-avail: -1 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 # Tokens are refilled, but since was at -1, the result is 0 tokens. set-io-tokens tokens=1 ---- GrantCoordinator: -(chain: id: 6 active: false index: 5) kv: used: 2, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 2, total(soft): 3(1) io-avail: 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 # Refill again. The waiting KV work is granted. @@ -364,7 +364,7 @@ set-io-tokens tokens=1 ---- kv: granted in chain 0, and returning true GrantCoordinator: -(chain: id: 6 active: false index: 0) kv: used: 3, total: 3 io-avail: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 0) kv: used: 3, total(soft): 3(1) io-avail: 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 ##################################################################### @@ -372,7 +372,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, total(soft): 1(1) 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. @@ -380,7 +380,7 @@ try-get work=kv ---- kv: tryGet 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, total(soft): 1(1) 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. @@ -388,7 +388,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet 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, total(soft): 1(1) 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. @@ -396,7 +396,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet 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, total(soft): 1(1) 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 @@ -404,7 +404,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, total(soft): 1(1) 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. @@ -412,7 +412,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet 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, total(soft): 1(1) 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. @@ -420,7 +420,7 @@ try-get work=sql-kv-response ---- sql-kv-response: tryGet 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, total(soft): 1(1) 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. @@ -428,7 +428,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet 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, total(soft): 1(1) 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. @@ -436,7 +436,7 @@ try-get work=sql-sql-response ---- sql-sql-response: tryGet 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, total(soft): 1(1) 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. @@ -444,5 +444,248 @@ try-get work=kv ---- kv: tryGet 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, total(soft): 1(1) sql-kv-response: avail: -1 sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 2 + +##################################################################### +# 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, total(soft): 1(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: 0) kv: used: 0, total(soft): 1(1) 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, total(soft): 1(1) 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, total(soft): 1(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: 0) kv: used: 0, total(soft): 1(1) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, total(soft): 2(2) 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, total(soft): 2(2) 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 returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 2(2) 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 returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 2(2) 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 returned false +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 2(2) 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, total(soft): 2(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: 0 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 2(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=4 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 3(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=1 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 3(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: 2, total(soft): 3(2) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 4(2) 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 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 4(2) 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, total(soft): 4(2) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 4(3) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 4(3) 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, total(soft): 4(3) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 5(4) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 5(3) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 5(2) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 5(2) 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, total(soft): 1(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: 0) kv: used: 0, total(soft): 1(1) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, total(soft): 2(2) 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, total(soft): 2(2) 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 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, total(soft): 3(3) 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, total(soft): 3(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=10 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 0, total(soft): 3(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 returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, total(soft): 3(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 returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, total(soft): 3(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 returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 3, total(soft): 3(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 + +# This is where the natural decrease logic will only decrease regular slots to +# 2 and not soft slots, since no soft slots are being used. But the code +# enforces the invariant that regular slots >= soft slots. +cpu-load runnable=10 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 3, total(soft): 2(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