diff --git a/pkg/server/config.go b/pkg/server/config.go index 873c0c26b094..714ff7bf4545 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -36,7 +36,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/ts" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -343,10 +342,6 @@ type KVConfig struct { // The value is split evenly between the stores if there are more than one. CacheSize int64 - // SoftSlotGranter can be optionally passed into a store to allow the store - // to perform additional CPU bound work. - SoftSlotGranter *admission.SoftSlotGranter - // TimeSeriesServerConfig contains configuration specific to the time series // server. TimeSeriesServerConfig ts.ServerConfig @@ -632,19 +627,6 @@ func (e *Engines) Close() { *e = nil } -// cpuWorkPermissionGranter implements the pebble.CPUWorkPermissionGranter -// interface. -//type cpuWorkPermissionGranter struct { -//*admission.SoftSlotGranter -//} - -//func (c *cpuWorkPermissionGranter) TryGetProcs(count int) int { -//return c.TryGetSlots(count) -//} -//func (c *cpuWorkPermissionGranter) ReturnProcs(count int) { -//c.ReturnSlots(count) -//} - // CreateEngines creates Engines based on the specs in cfg.Stores. func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { engines := Engines(nil) @@ -775,11 +757,6 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { pebbleConfig.Opts.TableCache = tableCache pebbleConfig.Opts.MaxOpenFiles = int(openFileLimitPerStore) pebbleConfig.Opts.Experimental.MaxWriterConcurrency = 2 - // TODO(jackson): Implement the new pebble.CPUWorkPermissionGranter - // interface. - //pebbleConfig.Opts.Experimental.CPUWorkPermissionGranter = &cpuWorkPermissionGranter{ - //cfg.SoftSlotGranter, - //} if storeKnobs.SmallEngineBlocks { for i := range pebbleConfig.Opts.Levels { pebbleConfig.Opts.Levels[i].BlockSize = 1 diff --git a/pkg/server/server.go b/pkg/server/server.go index 0196687970b5..1111b063e55c 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -246,11 +246,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { admissionOptions.Override(opts) } gcoords := admission.NewGrantCoordinators(cfg.AmbientCtx, st, admissionOptions, registry) - ssg, err := admission.MakeSoftSlotGranter(gcoords.Regular) - if err != nil { - return nil, errors.Wrap(err, "failed to soft slot granter") - } - cfg.SoftSlotGranter = ssg engines, err := cfg.CreateEngines(ctx) if err != nil { diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index e6fd8c5a2d19..8d0dc2d08719 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -3748,8 +3748,6 @@ var charts = []sectionDescription{ Metrics: []string{ "admission.granter.total_slots.kv", "admission.granter.used_slots.kv", - "admission.granter.total_moderate_slots.kv", - "admission.granter.used_soft_slots.kv", "admission.granter.used_slots.sql-leaf-start", "admission.granter.used_slots.sql-root-start", }, diff --git a/pkg/util/admission/grant_coordinator.go b/pkg/util/admission/grant_coordinator.go index 1a826be62123..49b179806346 100644 --- a/pkg/util/admission/grant_coordinator.go +++ b/pkg/util/admission/grant_coordinator.go @@ -272,12 +272,8 @@ var _ CPULoadListener = &GrantCoordinator{} // Options for constructing GrantCoordinators. type Options struct { - MinCPUSlots int - MaxCPUSlots int - // RunnableAlphaOverride is used to override the alpha value used to - // compute the ewma of the runnable goroutine counts. It is only used - // during testing. A 0 value indicates that there is no override. - RunnableAlphaOverride float64 + MinCPUSlots int + MaxCPUSlots int SQLKVResponseBurstTokens int64 SQLSQLResponseBurstTokens int64 SQLStatementLeafStartWorkSlots int @@ -434,10 +430,7 @@ func makeRegularGrantCoordinator( settings: st, minCPUSlots: opts.MinCPUSlots, maxCPUSlots: opts.MaxCPUSlots, - moderateSlotsClamp: opts.MaxCPUSlots, - runnableAlphaOverride: opts.RunnableAlphaOverride, totalSlotsMetric: metrics.KVTotalSlots, - totalModerateSlotsMetric: metrics.KVTotalModerateSlots, cpuLoadShortPeriodDurationMetric: metrics.KVCPULoadShortPeriodDuration, cpuLoadLongPeriodDurationMetric: metrics.KVCPULoadLongPeriodDuration, slotAdjusterIncrementsMetric: metrics.KVSlotAdjusterIncrements, @@ -457,10 +450,8 @@ func makeRegularGrantCoordinator( kvg := &slotGranter{ coord: coord, workKind: KVWork, - totalHighLoadSlots: opts.MinCPUSlots, - totalModerateLoadSlots: opts.MinCPUSlots, + totalSlots: opts.MinCPUSlots, usedSlotsMetric: metrics.KVUsedSlots, - usedSoftSlotsMetric: metrics.KVUsedSoftSlots, slotsExhaustedDurationMetric: metrics.KVSlotsExhaustedDuration, } @@ -500,11 +491,11 @@ func makeRegularGrantCoordinator( coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + coord: coord, + workKind: SQLStatementLeafStartWork, + totalSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry, admissionpb.NormalPri, admissionpb.LockingPri) req = makeRequester(ambientCtx, @@ -514,11 +505,11 @@ func makeRegularGrantCoordinator( coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: kvSlotAdjuster, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, + coord: coord, + workKind: SQLStatementRootStartWork, + totalSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: kvSlotAdjuster, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, } wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry, admissionpb.NormalPri, admissionpb.LockingPri) req = makeRequester(ambientCtx, @@ -585,11 +576,11 @@ func NewGrantCoordinatorSQL( coord.granters[SQLSQLResponseWork] = tg sg := &slotGranter{ - coord: coord, - workKind: SQLStatementLeafStartWork, - totalHighLoadSlots: opts.SQLStatementLeafStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLLeafStartUsedSlots, + coord: coord, + workKind: SQLStatementLeafStartWork, + totalSlots: opts.SQLStatementLeafStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLLeafStartUsedSlots, } wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry) req = makeRequester(ambientCtx, @@ -599,11 +590,11 @@ func NewGrantCoordinatorSQL( coord.granters[SQLStatementLeafStartWork] = sg sg = &slotGranter{ - coord: coord, - workKind: SQLStatementRootStartWork, - totalHighLoadSlots: opts.SQLStatementRootStartWorkSlots, - cpuOverload: sqlNodeCPU, - usedSlotsMetric: metrics.SQLRootStartUsedSlots, + coord: coord, + workKind: SQLStatementRootStartWork, + totalSlots: opts.SQLStatementRootStartWorkSlots, + cpuOverload: sqlNodeCPU, + usedSlotsMetric: metrics.SQLRootStartUsedSlots, } wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry) req = makeRequester(ambientCtx, @@ -919,13 +910,7 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { case KVWork: switch g := coord.granters[i].(type) { case *slotGranter: - kvsa := coord.cpuLoadListener.(*kvSlotAdjuster) - s.Printf( - "%s%s: used: %d, high(moderate)-total: %d(%d) moderate-clamp: %d", curSep, workKindString(kind), - g.usedSlots, g.totalHighLoadSlots, g.totalModerateLoadSlots, kvsa.moderateSlotsClamp) - if g.usedSoftSlots > 0 { - s.Printf(" used-soft: %d", g.usedSoftSlots) - } + s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots) case *kvStoreTokenGranter: s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens, g.elasticDiskBWTokensAvailable) @@ -933,7 +918,7 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { case SQLStatementLeafStartWork, SQLStatementRootStartWork: if coord.granters[i] != nil { g := coord.granters[i].(*slotGranter) - s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalHighLoadSlots) + s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots) } case SQLKVResponseWork, SQLSQLResponseWork: if coord.granters[i] != nil { @@ -953,8 +938,6 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) { type GrantCoordinatorMetrics struct { KVTotalSlots *metric.Gauge KVUsedSlots *metric.Gauge - KVTotalModerateSlots *metric.Gauge - KVUsedSoftSlots *metric.Gauge KVSlotsExhaustedDuration *metric.Counter KVCPULoadShortPeriodDuration *metric.Counter KVCPULoadLongPeriodDuration *metric.Counter @@ -970,12 +953,8 @@ func (GrantCoordinatorMetrics) MetricStruct() {} func makeGrantCoordinatorMetrics() GrantCoordinatorMetrics { m := GrantCoordinatorMetrics{ - KVTotalSlots: metric.NewGauge(totalSlots), - KVUsedSlots: metric.NewGauge(addName(workKindString(KVWork), usedSlots)), - // TODO(sumeer): remove moderate load slots and soft slots code and - // metrics #88032. - KVTotalModerateSlots: metric.NewGauge(totalModerateSlots), - KVUsedSoftSlots: metric.NewGauge(usedSoftSlots), + KVTotalSlots: metric.NewGauge(totalSlots), + KVUsedSlots: metric.NewGauge(addName(workKindString(KVWork), usedSlots)), KVSlotsExhaustedDuration: metric.NewCounter(kvSlotsExhaustedDuration), KVCPULoadShortPeriodDuration: metric.NewCounter(kvCPULoadShortPeriodDuration), KVCPULoadLongPeriodDuration: metric.NewCounter(kvCPULoadLongPeriodDuration), diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go index 13e3fe844bad..acae136a3f97 100644 --- a/pkg/util/admission/granter.go +++ b/pkg/util/admission/granter.go @@ -14,7 +14,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -22,14 +21,6 @@ import ( "github.com/cockroachdb/pebble" ) -// EnabledSoftSlotGranting can be set to false to disable soft slot granting. -var EnabledSoftSlotGranting = settings.RegisterBoolSetting( - settings.TenantWritable, - "admission.soft_slot_granting.enabled", - "soft slot granting is disabled if this setting is set to false", - true, -) - // noGrantChain is a sentinel value representing that the grant is not // responsible for continuing a grant chain. It is only used internally in // this file -- requester implementations do not need to concern themselves @@ -40,68 +31,21 @@ type requesterClose interface { close() } -// For the cpu-bound slot case we have background activities (like Pebble -// compactions) that would like to utilize additional slots if available (e.g. -// to do concurrent compression of ssblocks). These activities do not want to -// wait for a slot, since they can proceed without the slot at their usual -// slower pace (e.g. without doing concurrent compression). They also are -// sensitive to small overheads in their tight loops, and cannot afford the -// overhead of interacting with admission control at a fine granularity (like -// asking for a slot when compressing each ssblock). A coarse granularity -// interaction causes a delay in returning slots to admission control, and we -// don't want that delay to cause admission delay for normal work. Hence, we -// model slots granted to background activities as "soft-slots". Think of -// regular used slots as "hard-slots", in that we assume that the holder of -// the slot is still "using" it, while a soft-slot is "squishy" and in some -// cases we can pretend that it is not being used. Say we are allowed -// to allocate up to M slots. In this scheme, when allocating a soft-slot -// one must conform to usedSoftSlots+usedSlots <= M, and when allocating -// a regular (hard) slot one must conform to usedSlots <= M. -// -// That is, soft-slots allow for over-commitment until the soft-slots are -// returned, which may mean some additional queueing in the goroutine -// scheduler. -// -// We have another wrinkle in that we do not want to maintain a single M. For -// these optional background activities we desire to do them only when the -// load is low enough. This is because at high load, all work suffers from -// additional queueing in the goroutine scheduler. So we want to make sure -// regular work does not suffer such goroutine scheduler queueing because we -// granted too many soft-slots and caused CPU utilization to be high. So we -// maintain two kinds of M, totalHighLoadSlots and totalModerateLoadSlots. -// totalHighLoadSlots are estimated so as to allow CPU utilization to be high, -// while totalModerateLoadSlots are trying to keep queuing in the goroutine -// scheduler to a lower level. So the revised equations for allocation are: -// - Allocating a soft-slot: usedSoftSlots+usedSlots <= totalModerateLoadSlots -// - Allocating a regular slot: usedSlots <= totalHighLoadSlots -// -// NB: we may in the future add other kinds of background activities that do -// not have a lag in interacting with admission control, but want to schedule -// them only under moderate load. Those activities will be counted in -// usedSlots but when granting a slot to such an activity, the equation will -// be usedSoftSlots+usedSlots <= totalModerateLoadSlots. -// -// That is, let us not confuse that moderate load slot allocation is only for -// soft-slots. Soft-slots are introduced only for squishiness. -// // slotGranter implements granterWithLockedCalls. type slotGranter struct { - coord *GrantCoordinator - workKind WorkKind - requester requester - usedSlots int - usedSoftSlots int - totalHighLoadSlots int - totalModerateLoadSlots int - skipSlotEnforcement bool + coord *GrantCoordinator + workKind WorkKind + requester requester + usedSlots int + totalSlots int + skipSlotEnforcement bool // Optional. Nil for a slotGranter used for KVWork since the slots for that // slotGranter are directly adjusted by the kvSlotAdjuster (using the // kvSlotAdjuster here would provide a redundant identical signal). cpuOverload cpuOverloadIndicator - usedSlotsMetric *metric.Gauge - usedSoftSlotsMetric *metric.Gauge + usedSlotsMetric *metric.Gauge // Non-nil for KV slots. slotsExhaustedDurationMetric *metric.Counter exhaustedStart time.Time @@ -128,9 +72,9 @@ func (sg *slotGranter) tryGetLocked(count int64, _ int8) grantResult { if sg.cpuOverload != nil && sg.cpuOverload.isOverloaded() { return grantFailDueToSharedResource } - if sg.usedSlots < sg.totalHighLoadSlots || sg.skipSlotEnforcement { + if sg.usedSlots < sg.totalSlots || sg.skipSlotEnforcement { sg.usedSlots++ - if sg.usedSlots == sg.totalHighLoadSlots && sg.slotsExhaustedDurationMetric != nil { + if sg.usedSlots == sg.totalSlots && sg.slotsExhaustedDurationMetric != nil { sg.exhaustedStart = timeutil.Now() } sg.usedSlotsMetric.Update(int64(sg.usedSlots)) @@ -147,38 +91,12 @@ func (sg *slotGranter) returnGrant(count int64) { sg.coord.returnGrant(sg.workKind, count, 0 /*arbitrary*/) } -func (sg *slotGranter) tryGetSoftSlots(count int) int { - sg.coord.mu.Lock() - defer sg.coord.mu.Unlock() - spareModerateLoadSlots := sg.totalModerateLoadSlots - sg.usedSoftSlots - sg.usedSlots - if spareModerateLoadSlots <= 0 { - return 0 - } - allocatedSlots := count - if allocatedSlots > spareModerateLoadSlots { - allocatedSlots = spareModerateLoadSlots - } - sg.usedSoftSlots += allocatedSlots - sg.usedSoftSlotsMetric.Update(int64(sg.usedSoftSlots)) - return allocatedSlots -} - -func (sg *slotGranter) returnSoftSlots(count int) { - sg.coord.mu.Lock() - defer sg.coord.mu.Unlock() - sg.usedSoftSlots -= count - sg.usedSoftSlotsMetric.Update(int64(sg.usedSoftSlots)) - if sg.usedSoftSlots < 0 { - panic("used soft slots is negative") - } -} - // returnGrantLocked implements granterWithLockedCalls. func (sg *slotGranter) returnGrantLocked(count int64, _ int8) { if count != 1 { panic(errors.AssertionFailedf("unexpected count: %d", count)) } - if sg.usedSlots == sg.totalHighLoadSlots && sg.slotsExhaustedDurationMetric != nil { + if sg.usedSlots == sg.totalSlots && sg.slotsExhaustedDurationMetric != nil { now := timeutil.Now() exhaustedMicros := now.Sub(sg.exhaustedStart).Microseconds() sg.slotsExhaustedDurationMetric.Inc(exhaustedMicros) @@ -201,7 +119,7 @@ func (sg *slotGranter) tookWithoutPermissionLocked(count int64, _ int8) { panic(errors.AssertionFailedf("unexpected count: %d", count)) } sg.usedSlots++ - if sg.usedSlots == sg.totalHighLoadSlots && sg.slotsExhaustedDurationMetric != nil { + if sg.usedSlots == sg.totalSlots && sg.slotsExhaustedDurationMetric != nil { sg.exhaustedStart = timeutil.Now() } sg.usedSlotsMetric.Update(int64(sg.usedSlots)) @@ -234,29 +152,29 @@ func (sg *slotGranter) tryGrantLocked(grantChainID grantChainID) grantResult { } //gcassert:inline -func (sg *slotGranter) setTotalHighLoadSlotsLocked(totalHighLoadSlots int) { +func (sg *slotGranter) setTotalSlotsLocked(totalSlots int) { // Mid-stack inlining. - if totalHighLoadSlots == sg.totalHighLoadSlots { + if totalSlots == sg.totalSlots { return } - sg.setTotalHighLoadSlotsLockedInternal(totalHighLoadSlots) + sg.setTotalSlotsLockedInternal(totalSlots) } -func (sg *slotGranter) setTotalHighLoadSlotsLockedInternal(totalHighLoadSlots int) { +func (sg *slotGranter) setTotalSlotsLockedInternal(totalSlots int) { if sg.slotsExhaustedDurationMetric != nil { - if totalHighLoadSlots > sg.totalHighLoadSlots { - if sg.totalHighLoadSlots <= sg.usedSlots && totalHighLoadSlots > sg.usedSlots { + if totalSlots > sg.totalSlots { + if sg.totalSlots <= sg.usedSlots && totalSlots > sg.usedSlots { now := timeutil.Now() exhaustedMicros := now.Sub(sg.exhaustedStart).Microseconds() sg.slotsExhaustedDurationMetric.Inc(exhaustedMicros) } - } else if totalHighLoadSlots < sg.totalHighLoadSlots { - if sg.totalHighLoadSlots > sg.usedSlots && totalHighLoadSlots <= sg.usedSlots { + } else if totalSlots < sg.totalSlots { + if sg.totalSlots > sg.usedSlots && totalSlots <= sg.usedSlots { sg.exhaustedStart = timeutil.Now() } } } - sg.totalHighLoadSlots = totalHighLoadSlots + sg.totalSlots = totalSlots } // tokenGranter implements granterWithLockedCalls. @@ -712,12 +630,6 @@ var ( Measurement: "Slots", Unit: metric.Unit_COUNT, } - totalModerateSlots = metric.Metadata{ - Name: "admission.granter.total_moderate_slots.kv", - Help: "Total moderate load slots for low priority work", - Measurement: "Slots", - Unit: metric.Unit_COUNT, - } usedSlots = metric.Metadata{ // Note: we append a WorkKind string to this name. Name: "admission.granter.used_slots.", @@ -725,12 +637,6 @@ var ( Measurement: "Slots", Unit: metric.Unit_COUNT, } - usedSoftSlots = metric.Metadata{ - Name: "admission.granter.used_soft_slots.kv", - Help: "Used soft slots", - Measurement: "Slots", - Unit: metric.Unit_COUNT, - } // NB: this metric is independent of whether slots enforcement is happening // or not. kvSlotsExhaustedDuration = metric.Metadata{ @@ -793,35 +699,3 @@ var ( // uses the term "slot" for these is that we have a completion indicator, and // when we do have such an indicator it can be beneficial to be able to keep // track of how many ongoing work items we have. - -// SoftSlotGranter grants soft slots without queueing. See the comment with -// kvGranter. -type SoftSlotGranter struct { - kvGranter *slotGranter -} - -// MakeSoftSlotGranter constructs a SoftSlotGranter given a GrantCoordinator -// that is responsible for KV and lower layers. -func MakeSoftSlotGranter(gc *GrantCoordinator) (*SoftSlotGranter, error) { - kvGranter, ok := gc.granters[KVWork].(*slotGranter) - if !ok { - return nil, errors.Errorf("GrantCoordinator does not support soft slots") - } - return &SoftSlotGranter{ - kvGranter: kvGranter, - }, nil -} - -// TryGetSlots attempts to acquire count slots and returns what was acquired -// (possibly 0). -func (ssg *SoftSlotGranter) TryGetSlots(count int) int { - if !EnabledSoftSlotGranting.Get(&ssg.kvGranter.coord.settings.SV) { - return 0 - } - 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 cb9b8e0cf200..33a1f66d0456 100644 --- a/pkg/util/admission/granter_test.go +++ b/pkg/util/admission/granter_test.go @@ -35,18 +35,15 @@ import ( // // init-grant-coordinator min-cpu= max-cpu= sql-kv-tokens= // sql-sql-tokens= sql-leaf= sql-root= -// enabled-soft-slot-granting= // set-has-waiting-requests work= v= // set-return-value-from-granted work= v= // try-get work= [v=] // return-grant work= [v=] // took-without-permission work= [v=] // continue-grant-chain work= -// cpu-load runnable= procs= [infrequent=] [clamp=] +// cpu-load runnable= procs= [infrequent=] // init-store-grant-coordinator // set-io-tokens tokens= -// try-get-soft-slots slots= -// return-soft-slots slots= func TestGranterBasic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -56,7 +53,6 @@ func TestGranterBasic(t *testing.T) { // store grant coordinator. var requesters [numWorkKinds + 1]*testRequester var coord *GrantCoordinator - var ssg *SoftSlotGranter clearRequesterAndCoord := func() { coord = nil for i := range requesters { @@ -101,21 +97,9 @@ func TestGranterBasic(t *testing.T) { return req } delayForGrantChainTermination = 0 - opts.RunnableAlphaOverride = 1 // This gives weight to only the most recent sample. coords := NewGrantCoordinators(ambientCtx, settings, opts, registry) defer coords.Close() coord = coords.Regular - var err error - ssg, err = MakeSoftSlotGranter(coord) - require.NoError(t, err) - if d.HasArg("enabled-soft-slot-granting") { - var enabledSoftSlotGranting bool - d.ScanArgs(t, "enabled-soft-slot-granting", &enabledSoftSlotGranting) - if !enabledSoftSlotGranting { - EnabledSoftSlotGranting.Override(context.Background(), &settings.SV, false) - } - } - return flushAndReset() case "init-store-grant-coordinator": @@ -216,12 +200,6 @@ func TestGranterBasic(t *testing.T) { if d.HasArg("infrequent") { d.ScanArgs(t, "infrequent", &infrequent) } - if d.HasArg("clamp") { - var clamp int - d.ScanArgs(t, "clamp", &clamp) - kvsa := coord.cpuLoadListener.(*kvSlotAdjuster) - kvsa.setModerateSlotsClamp(clamp) - } samplePeriod := time.Millisecond if infrequent { @@ -272,19 +250,6 @@ 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) } @@ -452,12 +417,6 @@ func (str *storeTestRequester) setStoreRequestEstimates(estimates storeRequestEs // Only used by ioLoadListener, so don't bother. } -// setModerateSlotsClamp is used in testing to force a value for kvsa.moderateSlotsClamp. -func (kvsa *kvSlotAdjuster) setModerateSlotsClamp(val int) { - kvsa.moderateSlotsClampOverride = val - kvsa.moderateSlotsClamp = val -} - func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 { var kindStr string d.ScanArgs(t, "work", &kindStr) diff --git a/pkg/util/admission/kv_slot_adjuster.go b/pkg/util/admission/kv_slot_adjuster.go index 20e5961caf37..8410af15d5c1 100644 --- a/pkg/util/admission/kv_slot_adjuster.go +++ b/pkg/util/admission/kv_slot_adjuster.go @@ -42,24 +42,8 @@ type kvSlotAdjuster struct { granter *slotGranter minCPUSlots int maxCPUSlots int - // moderateSlotsClamp is the most recent value which may have been used to - // clamp down on slotGranter.totalModerateLoadSlots. Justification for - // clamping down on totalModerateLoadSlots is given where the moderateSlotsClamp - // value is written to. - moderateSlotsClamp int - // moderateSlotsClampOverride is used during testing to override the value of the - // moderateSlotsClamp. Its purpose is to make it easier to write tests. A default - // value of 0 implies no override. - moderateSlotsClampOverride int - // runnableEWMA is a weighted average of the most recent runnable goroutine counts. - // runnableEWMA is used to tune the slotGranter.totalModerateLoadSlots. - runnableEWMA float64 - // runnableAlphaOverride is used to override the value of runnable alpha during testing. - // A 0 value indicates that there is no override. - runnableAlphaOverride float64 totalSlotsMetric *metric.Gauge - totalModerateSlotsMetric *metric.Gauge cpuLoadShortPeriodDurationMetric *metric.Counter cpuLoadLongPeriodDurationMetric *metric.Counter slotAdjusterIncrementsMetric *metric.Counter @@ -78,21 +62,10 @@ func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, samplePeriod time.D } else { kvsa.cpuLoadShortPeriodDurationMetric.Inc(periodDurationMicros) } - // 0.009 gives weight to at least a few hundred samples at a 1ms sampling rate. - alpha := 0.009 * float64(samplePeriod/time.Millisecond) - if alpha > 0.5 { - alpha = 0.5 - } else if alpha < 0.001 { - alpha = 0.001 - } - if kvsa.runnableAlphaOverride > 0 { - alpha = kvsa.runnableAlphaOverride - } - kvsa.runnableEWMA = kvsa.runnableEWMA*(1-alpha) + float64(runnable)*alpha // Simple heuristic, which worked ok in experiments. More sophisticated ones // could be devised. - usedSlots := kvsa.granter.usedSlots + kvsa.granter.usedSoftSlots + usedSlots := kvsa.granter.usedSlots tryDecreaseSlots := func(total int, adjustMetric bool) int { // Overload. // If using some slots, and the used slots is less than the total slots, @@ -135,95 +108,18 @@ func (kvsa *kvSlotAdjuster) CPULoad(runnable int, procs int, samplePeriod time.D } if runnable >= threshold*procs { - // Very overloaded. - kvsa.granter.setTotalHighLoadSlotsLocked( - tryDecreaseSlots(kvsa.granter.totalHighLoadSlots, true)) - kvsa.granter.totalModerateLoadSlots = tryDecreaseSlots( - kvsa.granter.totalModerateLoadSlots, false) - } else if float64(runnable) <= float64((threshold*procs)/4) { - // Very underloaded. - kvsa.granter.setTotalHighLoadSlotsLocked( - tryIncreaseSlots(kvsa.granter.totalHighLoadSlots, true)) - kvsa.granter.totalModerateLoadSlots = tryIncreaseSlots( - kvsa.granter.totalModerateLoadSlots, false) + // Overloaded. + kvsa.granter.setTotalSlotsLocked( + tryDecreaseSlots(kvsa.granter.totalSlots, true)) } else if float64(runnable) <= float64((threshold*procs)/2) { - // Moderately underloaded -- can afford to increase regular slots. - kvsa.granter.setTotalHighLoadSlotsLocked( - tryIncreaseSlots(kvsa.granter.totalHighLoadSlots, true)) - } 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, false) - } - // Consider the following cases, when we started this method with - // totalHighLoadSlots==totalModerateLoadSlots. - // - underload such that we are able to increase totalModerateLoadSlots: in - // this case we will also be able to increase totalHighLoadSlots (since - // the used and total comparisons gating the increase in tryIncreaseSlots - // will also be true for totalHighLoadSlots). - // - overload such that we are able to decrease totalHighLoadSlots: in this - // case the logic in tryDecreaseSlots will also be able to decrease - // totalModerateLoadSlots. - // So the natural behavior of the slot adjustment itself guarantees - // totalHighLoadSlots >= totalModerateLoadSlots. But as a defensive measure - // we clamp totalModerateLoadSlots to not exceed totalHighLoadSlots. - if kvsa.granter.totalHighLoadSlots < kvsa.granter.totalModerateLoadSlots { - kvsa.granter.totalModerateLoadSlots = kvsa.granter.totalHighLoadSlots - } - - // During a kv50 workload, we noticed soft slots grants succeeding despite - // high cpu utilization, and high runnable goroutine counts. - // - // Consider the following log lines from the kv50 experiment: - // [runnable count 372 threshold*procs 256] - // [totalHighLoadSlots 254 totalModerateLoadSlots 164 usedSlots 0 usedSoftSlots 1] - // - // Note that even though the runnable count is high, of the (254, 164), - // (totalHighLoad, totalModerateLoad) slots respectively, only 1 slot is - // being used. The slot mechanism behaves in a bi-modal manner in nodes that - // do both KV and SQL processing. While there is backlogged KV work, the slot - // usage is high, and blocks all SQL work, but eventually all callers have done - // their KV processing and are queued up for SQL work. The latter causes bursts - // of grants (because it uses tokens), gated only by the grant-chain mechanism, - // during which runnable count is high but used (KV) slots are low. This is exactly - // the case where we have low slot usage, but high CPU utilization. - // - // We can afford to be more conservative in calculating totalModerateLoadSlots - // since we don't care about saturating CPU for the less important work that is - // controlled by these slots. So we could use a slow reacting and conservative - // signal to decide on the value of totalModerateLoadSlots. - // - // To account for the increased CPU utilization and runnable counts when the used - // slots are low, we clamp down on the totalModerateSlots value by keeping track - // of a historical runnable goroutine average. - kvsa.moderateSlotsClamp = int(float64(threshold*procs)/2 - kvsa.runnableEWMA) - if kvsa.moderateSlotsClampOverride != 0 { - kvsa.moderateSlotsClamp = kvsa.moderateSlotsClampOverride - } - if kvsa.granter.totalModerateLoadSlots > kvsa.moderateSlotsClamp { - kvsa.granter.totalModerateLoadSlots = kvsa.moderateSlotsClamp - } - if kvsa.granter.totalModerateLoadSlots < 0 { - kvsa.granter.totalModerateLoadSlots = 0 + // Underloaded -- can afford to increase regular slots. + kvsa.granter.setTotalSlotsLocked( + tryIncreaseSlots(kvsa.granter.totalSlots, true)) } - kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalHighLoadSlots)) - kvsa.totalModerateSlotsMetric.Update(int64(kvsa.granter.totalModerateLoadSlots)) + kvsa.totalSlotsMetric.Update(int64(kvsa.granter.totalSlots)) } func (kvsa *kvSlotAdjuster) isOverloaded() bool { - return kvsa.granter.usedSlots >= kvsa.granter.totalHighLoadSlots && !kvsa.granter.skipSlotEnforcement + return kvsa.granter.usedSlots >= kvsa.granter.totalSlots && !kvsa.granter.skipSlotEnforcement } diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter index 0c87e89a46eb..b8cc04d7cc9c 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 1 active: true index: 0) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 +(chain: id: 2 active: false index: 1) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 +(chain: id: 2 active: true index: 1) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 1) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 3) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 3) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 2 active: true index: 4) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 3 active: false index: 5) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 0, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 3 active: true index: 3) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 1, total: 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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 0 +(chain: id: 4 active: false index: 3) kv: used: 2, total: 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, high(moderate)-total: 2(0) moderate-clamp: 0 sql-kv-response: avail: 2 +(chain: id: 4 active: false index: 1) kv: used: 2, total: 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 SlotAdjuster metrics: slots: 2, duration (short, long) millis: (1, 0), inc: 1, dec: 0 @@ -228,7 +228,7 @@ 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, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 1 +(chain: id: 4 active: true index: 1) kv: used: 2, total: 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 SlotAdjuster metrics: slots: 3, duration (short, long) millis: (2, 0), inc: 2, dec: 0 @@ -236,7 +236,7 @@ SlotAdjuster metrics: slots: 3, duration (short, long) millis: (2, 0), inc: 2, d cpu-load runnable=2 procs=1 ---- GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, total: 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 SlotAdjuster metrics: slots: 2, duration (short, long) millis: (3, 0), inc: 2, dec: 1 @@ -245,7 +245,7 @@ continue-grant-chain work=sql-kv-response ---- sql-kv-response: continueGrantChain GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, total: 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 @@ -254,7 +254,7 @@ return-grant work=sql-leaf-start ---- sql-leaf-start: returnGrant(1) GrantCoordinator: -(chain: id: 5 active: false index: 1) kv: used: 2, high(moderate)-total: 2(0) moderate-clamp: -1 sql-kv-response: avail: 2 +(chain: id: 5 active: false index: 1) kv: used: 2, total: 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 @@ -267,7 +267,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, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 SlotAdjuster metrics: slots: 3, duration (short, long) millis: (4, 0), inc: 3, dec: 1 @@ -276,7 +276,7 @@ return-grant work=sql-root-start ---- sql-root-start: returnGrant(1) GrantCoordinator: -(chain: id: 5 active: true index: 3) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 0, total: 1 # Continuing with chain-id=0 has no effect. @@ -284,7 +284,7 @@ continue-grant-chain work=sql-kv-response ---- sql-kv-response: continueGrantChain GrantCoordinator: -(chain: id: 5 active: true index: 3) kv: used: 2, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 +(chain: id: 5 active: true index: 3) kv: used: 2, total: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 0, total: 1 # Continuing chain-id=5 causes a grant to sql-root-start and the chain dies @@ -294,7 +294,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, high(moderate)-total: 3(0) moderate-clamp: 0 sql-kv-response: avail: 0 +(chain: id: 6 active: false index: 5) kv: used: 2, total: 3 sql-kv-response: avail: 0 sql-sql-response: avail: 1 sql-leaf-start: used: 2, total: 2 sql-root-start: used: 1, total: 1 ##################################################################### @@ -302,7 +302,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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 0, total: 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. @@ -310,7 +310,7 @@ try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 1, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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. @@ -318,7 +318,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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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. @@ -326,7 +326,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, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 0) kv: used: 1, total: 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 @@ -334,7 +334,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, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: 1 +(chain: id: 1 active: false index: 5) kv: used: 1, total: 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 SlotAdjuster metrics: slots: 1, duration (short, long) millis: (0, 250), inc: 0, dec: 0 @@ -343,7 +343,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, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: 0 +(chain: id: 1 active: false index: 5) kv: used: 1, total: 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. @@ -351,7 +351,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, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, total: 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. @@ -359,7 +359,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, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, total: 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. @@ -367,7 +367,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, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 1, total: 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. @@ -375,7 +375,7 @@ try-get work=kv ---- kv: tryGet(1) returned true GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 1(0) moderate-clamp: -19 sql-kv-response: avail: -1 +(chain: id: 1 active: false index: 5) kv: used: 2, total: 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 ##################################################################### @@ -589,353 +589,3 @@ store-write-done work=kv-elastic orig-tokens=400 write-bytes=40 ---- GrantCoordinator: (chain: id: 0 active: false index: 5) io-avail: 650, elastic-disk-bw-tokens-avail: -10 - - -##################################################################### -# Test soft slots -init-grant-coordinator min-cpu=1 max-cpu=6 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 0 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -return-soft-slots slots=1 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=0 procs=4 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 2, duration (short, long) millis: (1, 0), inc: 1, dec: 0 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -# Over-commitment. -try-get work=kv ----- -kv: tryGet(1) returned true -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -# Over-commitment. -try-get work=kv ----- -kv: tryGet(1) returned true -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get work=kv ----- -kv: tryGet(1) returned false -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -# No longer over-committed. -return-soft-slots slots=2 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 0 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=4 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(2) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (2, 0), inc: 2, dec: 0 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 0 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(2) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=1 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (3, 0), inc: 2, dec: 0 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=3 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 4, duration (short, long) millis: (4, 0), inc: 3, dec: 0 - -return-grant work=kv ----- -kv: returnGrant(1) -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(3) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=2 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 4, duration (short, long) millis: (5, 0), inc: 3, dec: 0 - -cpu-load runnable=2 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 4, duration (short, long) millis: (6, 0), inc: 3, dec: 0 - -try-get-soft-slots slots=2 ----- -requested: 2, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 4(4) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=2 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(5) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 5, duration (short, long) millis: (7, 0), inc: 4, dec: 0 - -cpu-load runnable=6 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(4) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 5, duration (short, long) millis: (8, 0), inc: 4, dec: 0 - -cpu-load runnable=6 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(3) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 5, duration (short, long) millis: (9, 0), inc: 4, dec: 0 - -cpu-load runnable=6 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 5(3) moderate-clamp: 100 used-soft: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 5, duration (short, long) millis: (10, 0), inc: 4, dec: 0 - -##################################################################### -# Test soft slots is not higher than regular slots. -init-grant-coordinator min-cpu=1 max-cpu=6 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 6 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=2 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 2, duration (short, long) millis: (1, 0), inc: 1, dec: 0 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=2 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (2, 0), inc: 2, dec: 0 - -return-soft-slots slots=2 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=10 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (3, 0), inc: 2, dec: 0 - -try-get work=kv ----- -kv: tryGet(1) returned true -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 1, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get work=kv ----- -kv: tryGet(1) returned true -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 2, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get work=kv ----- -kv: tryGet(1) returned true -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 3, high(moderate)-total: 3(3) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=10 procs=8 clamp=100 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 3, high(moderate)-total: 2(2) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 2, duration (short, long) millis: (4, 0), inc: 2, dec: 1 - -##################################################################### -# Test clamping down on total moderate slots. -init-grant-coordinator min-cpu=1 max-cpu=100 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 100 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 100 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=0 procs=8 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 4 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 2, duration (short, long) millis: (1, 0), inc: 1, dec: 0 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 4 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -cpu-load runnable=0 procs=2 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(1) moderate-clamp: 1 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (2, 0), inc: 2, dec: 0 - -# The moderate slots clamp is set to -1, because 10/2-6=-1. -cpu-load runnable=6 procs=10 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(0) moderate-clamp: -1 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (3, 0), inc: 2, dec: 0 - -# Making sure total moderate slots is set to 0, and not a negative value. -cpu-load runnable=6 procs=10 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 3(0) moderate-clamp: -1 used-soft: 2 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 3, duration (short, long) millis: (4, 0), inc: 2, dec: 0 - -##################################################################### -# Make sure the moderate slots clamp has no effect when it is higher than the total moderate slots. -init-grant-coordinator min-cpu=1 max-cpu=3 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 1 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -# Should increase total moderate slots to 2, since the moderate clamp was 3 which is higher. -cpu-load runnable=0 procs=2 clamp=3 ----- -GrantCoordinator: -(chain: id: 1 active: false index: 5) kv: used: 0, high(moderate)-total: 2(2) moderate-clamp: 3 used-soft: 1 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 -SlotAdjuster metrics: slots: 2, duration (short, long) millis: (1, 0), inc: 1, dec: 0 - -init-grant-coordinator min-cpu=1 max-cpu=3 sql-kv-tokens=2 sql-sql-tokens=1 sql-leaf=2 sql-root=1 enabled-soft-slot-granting=false ----- -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1 - -try-get-soft-slots slots=1 ----- -requested: 1, granted: 0 -GrantCoordinator: -(chain: id: 1 active: false index: 0) kv: used: 0, high(moderate)-total: 1(1) moderate-clamp: 3 sql-kv-response: avail: 2 -sql-sql-response: avail: 1 sql-leaf-start: used: 0, total: 2 sql-root-start: used: 0, total: 1