diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 7118806fa7b4..1cc40859bfe5 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -343,7 +343,11 @@ type slotGranter struct { workKind WorkKind requester requester usedSlots int - totalSlots int + usedSoftSlots int + totalHighLoadSlots int + totalModerateLoadSlots int + failedSoftSlotsGet bool + runnableEWMA float64 skipSlotEnforcement bool // Optional. Nil for a slotGranter used for KVWork since the slots for that @@ -375,7 +379,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 +394,31 @@ 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 { + sg.failedSoftSlotsGet = true + return 0 + } + allocatedSlots := count + if allocatedSlots > spareModerateLoadSlots { + allocatedSlots = spareModerateLoadSlots + } + sg.usedSoftSlots += allocatedSlots + return allocatedSlots +} + +func (sg *slotGranter) 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") + } +} + func (sg *slotGranter) returnGrantLocked(count int64) { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) @@ -486,9 +515,54 @@ func (tg *tokenGranter) continueGrantChain(grantChainID grantChainID) { tg.coord.continueGrantChain(tg.workKind, grantChainID) } -// kvStoreTokenGranter implements granterWithLockedCalls. It is used for -// grants to KVWork to a store, that is limited by IO tokens. -type kvStoreTokenGranter struct { + +// 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. + + // kvStoreTokenGranter implements granterWithLockedCalls. It is used for + // grants to KVWork to a store, that is limited by IO tokens. + type kvStoreTokenGranter struct { coord *GrantCoordinator requester requester // There is no rate limiting in granting these tokens. That is, they are all @@ -496,7 +570,7 @@ type kvStoreTokenGranter struct { availableIOTokens int64 ioTokensExhaustedDurationMetric *metric.Counter exhaustedStart time.Time -} + } var _ granterWithLockedCalls = &kvStoreTokenGranter{} @@ -514,12 +588,13 @@ func (sg *kvStoreTokenGranter) tryGet(count int64) bool { func (sg *kvStoreTokenGranter) tryGetLocked(count int64) grantResult { if sg.availableIOTokens > 0 { - sg.subtractTokens(count, false) - return grantSuccess + sg.subtractTokens(count, false) + return grantSuccess } return grantFailLocal } + func (sg *kvStoreTokenGranter) returnGrant(count int64) { sg.coord.returnGrant(KVWork, count) } @@ -737,11 +812,13 @@ 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, } + kvSlotAdjuster.granter = kvg coord.queues[KVWork] = makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork)) kvg.requester = coord.queues[KVWork] @@ -774,7 +851,7 @@ func NewGrantCoordinators( sg := &slotGranter{ coord: coord, workKind: SQLStatementLeafStartWork, - totalSlots: opts.SQLStatementLeafStartWorkSlots, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, cpuOverload: kvSlotAdjuster, usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } @@ -786,7 +863,7 @@ func NewGrantCoordinators( sg = &slotGranter{ coord: coord, workKind: SQLStatementRootStartWork, - totalSlots: opts.SQLStatementRootStartWorkSlots, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, cpuOverload: kvSlotAdjuster, usedSlotsMetric: metrics.SQLRootStartUsedSlots, } @@ -865,7 +942,7 @@ func NewGrantCoordinatorSQL( sg := &slotGranter{ coord: coord, workKind: SQLStatementLeafStartWork, - totalSlots: opts.SQLStatementLeafStartWorkSlots, + totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, cpuOverload: sqlNodeCPU, usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } @@ -877,7 +954,7 @@ func NewGrantCoordinatorSQL( sg = &slotGranter{ coord: coord, workKind: SQLStatementRootStartWork, - totalSlots: opts.SQLStatementRootStartWorkSlots, + totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, cpuOverload: sqlNodeCPU, usedSlotsMetric: metrics.SQLRootStartUsedSlots, } @@ -1212,16 +1289,20 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { switch kind { 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) - case *kvStoreTokenGranter: - s.Printf(" io-avail: %d", g.availableIOTokens) - } + case *slotGranter: + s.Printf( + "%s%s: used: %d, high(moderate)-total: %d(%d)", curSep, workKindString(kind), + g.usedSlots, g.totalHighLoadSlots, g.totalModerateLoadSlots) + 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,8 +1434,9 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo useGrantChains: false, numProcs: 1, } + kvg := &kvStoreTokenGranter{ - coord: coord, + coord: coord, ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration, } opts := makeWorkQueueOptions(KVWork) @@ -1466,7 +1548,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(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 +1562,81 @@ 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 { + // 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 (usedSlots >= total || (usedSlots >= 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 } - kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalSlots)) + + // TODO: the fractions below are arbitrary and subject to tuning. + 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 + } + kvsa.granter.failedSoftSlotsGet = false + kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalHighLoadSlots)) } 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 @@ -2067,3 +2202,33 @@ 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). +// 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 72ae85572118..4e06e1fdc7e9 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -110,6 +110,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 { @@ -156,6 +157,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 "init-store-grant-coordinator": @@ -254,6 +258,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..9b4f12e62bb1 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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) 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(2) 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(3) 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(2) 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(2) 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(2) 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(2) 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(2) 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(2) 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(2) 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) 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) 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) 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) 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(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. @@ -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(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. @@ -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(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. @@ -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(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. @@ -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(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. @@ -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(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 ##################################################################### @@ -461,3 +461,250 @@ 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) 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) 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) 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) 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) 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, high(moderate)-total: 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, high(moderate)-total: 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(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 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(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 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(1) returned false +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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: 0 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 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, high(moderate)-total: 4(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=2 +---- +requested: 2, granted: 1 +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 4(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 + +cpu-load runnable=3 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 4(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 + +return-grant work=kv +---- +kv: returnGrant(1) +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(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 + +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) 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, high(moderate)-total: 4(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=2 procs=8 +---- +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(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 + +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) 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, high(moderate)-total: 5(5) 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, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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 + +##################################################################### +# 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) 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) 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, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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, high(moderate)-total: 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(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 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(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 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(1) returned true +GrantCoordinator: +(chain: id: 1 active: false index: 5) kv: used: 3, high(moderate)-total: 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: 3, high(moderate)-total: 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