+admission.disk_bandwidth_tokens.elastic.enabled | boolean | true | when true, and provisioned bandwidth for the disk corresponding to a store is configured, tokens for elastic work will be limited if disk bandwidth becomes a bottleneck |
admission.epoch_lifo.enabled | boolean | false | when true, epoch-LIFO behavior is enabled when there is significant delay in admission |
admission.epoch_lifo.epoch_closing_delta_duration | duration | 5ms | the delta duration before closing an epoch, for epoch-LIFO admission control ordering |
admission.epoch_lifo.epoch_duration | duration | 100ms | the duration of an epoch, for epoch-LIFO admission control ordering |
diff --git a/pkg/util/admission/BUILD.bazel b/pkg/util/admission/BUILD.bazel
index 904c3fa97b65..11e528f3b374 100644
--- a/pkg/util/admission/BUILD.bazel
+++ b/pkg/util/admission/BUILD.bazel
@@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "admission",
srcs = [
+ "disk_bandwidth.go",
"doc.go",
"granter.go",
"store_token_estimation.go",
@@ -32,6 +33,7 @@ go_library(
go_test(
name = "admission_test",
srcs = [
+ "disk_bandwidth_test.go",
"granter_test.go",
"store_token_estimation_test.go",
"work_queue_test.go",
diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go
new file mode 100644
index 000000000000..022c08293de8
--- /dev/null
+++ b/pkg/util/admission/disk_bandwidth.go
@@ -0,0 +1,336 @@
+// Copyright 2022 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+package admission
+
+import (
+ "context"
+ "math"
+
+ "github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/redact"
+)
+
+// The functionality in this file is geared towards preventing chronic overload
+// of disk bandwidth which typically results in severely high latency for all work.
+//
+// For now, we assume that:
+// - There is a provisioned limit on the sum of read and write bandwidth. This
+// limit is allowed to change. This is true for block devices of major cloud
+// providers.
+// - Admission control can only shape the rate of admission of writes. Writes
+// also cause reads, since compactions do reads and writes.
+//
+// There are multiple challenges:
+// - We are unable to precisely track the causes of disk read bandwidth, since
+// we do not have observability into what reads missed the OS page cache.
+// That is we don't know how much of the reads were due to incoming reads
+// (that we don't shape) and how much due to compaction read bandwidth.
+//
+// - We don't shape incoming reads.
+//
+// - There can be a large lag (1+min) between the shaping of incoming writes,
+// and when it affects actual writes in the system, since compaction backlog
+// can build up in various levels of the LSM store.
+//
+// - Signals of overload are coarse, since we cannot view all the internal
+// queues that can build up due to resource overload. For instance,
+// different examples of bandwidth saturation exhibit very different
+// latency effects, presumably because the queue buildup is different. So it
+// is non-trivial to approach full utilization without risking high latency.
+//
+// Due to these challenges, we adopt a goal of simplicity of design, and
+// strong abstraction boundaries.
+//
+// - The disk load is abstracted using an enum, diskLoadLevel. The
+// diskLoadWatcher, that maps load signals to this enum, can be evolved
+// independently.
+//
+// - The approach uses easy to understand additive increase and multiplicative
+// decrease, (unlike what we do for flush and compaction tokens, where we
+// try to more precisely calculate the sustainable rates).
+//
+// Since we are using a simple approach that is somewhat coarse in its behavior,
+// we start by limiting its application to two kinds of writes (the second one
+// is future work, and not yet implemented):
+//
+// - Incoming writes that are deemed "elastic": This can be done by
+// introducing a work-class (in addition to admissionpb.WorkPriority), or by
+// implying a work-class from the priority (e.g. priorities < NormalPri are
+// deemed elastic).
+//
+// - Optional compactions: We assume that the LSM store is configured with a
+// ceiling on number of regular concurrent compactions, and if it needs more
+// it can request resources for additional (optional) compactions. These
+// latter compactions can be limited by this approach. See
+// https://github.com/cockroachdb/pebble/issues/1329 for motivation.
+// TODO(sumeer): this compaction control is not yet done, though how to do
+// it is included in the prototype in
+// https://github.com/cockroachdb/cockroach/pull/82813
+//
+// Extending this to all incoming writes is future work.
+
+// The load level of a disk.
+type diskLoadLevel int8
+
+const (
+ // diskLoadLow implies no need to shape anything.
+ diskLoadLow diskLoadLevel = iota
+ // diskLoadModerate implies shaping and additive increase.
+ diskLoadModerate
+ // diskLoadHigh implies shaping and hold steady.
+ diskLoadHigh
+ // diskLoadOverload implies shaping and multiplicative decrease.
+ diskLoadOverload
+)
+
+// diskLoadWatcher computes the diskLoadLevel based on provided stats.
+type diskLoadWatcher struct {
+ lastInterval intervalDiskLoadInfo
+ lastUtil float64
+ loadLevel diskLoadLevel
+}
+
+// intervalDiskLoadInfo provides disk stats over an adjustmentInterval.
+type intervalDiskLoadInfo struct {
+ // readBandwidth is the measure disk read bandwidth in bytes/s.
+ readBandwidth int64
+ // writeBandwidth is the measured disk write bandwidth in bytes/s.
+ writeBandwidth int64
+ // provisionedBandwidth is the aggregate (read+write) provisioned bandwidth
+ // in bytes/s.
+ provisionedBandwidth int64
+}
+
+// setIntervalInfo is called at the same time as ioLoadListener.pebbleMetricsTick.
+func (d *diskLoadWatcher) setIntervalInfo(ctx context.Context, load intervalDiskLoadInfo) {
+ lastInterval := load
+ util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth)
+ // The constants and other heuristics in the following logic can seem
+ // extremely arbitrary: they were subject to some tuning and evolution based
+ // on the experiments in https://github.com/cockroachdb/cockroach/pull/82813
+ // that used (a) an artificial provisioned bandwidth limit lower than the
+ // actual, to see how well the system stayed within that limit, (b) an
+ // actual provisioned bandwidth limit. The difficulty in general is that
+ // small changes can have outsize influence if a higher number of
+ // compactions start happening.
+ var loadLevel diskLoadLevel
+ const lowUtilThreshold = 0.3
+ const moderateUtilThreshold = 0.7
+ const highUtilThreshold = 0.95
+ const highlyOverUtilizedThreshold = 2.0
+ const smallDelta = 0.05
+ if util < lowUtilThreshold {
+ // Were at moderate or lower and have not increased significantly and the
+ // lastUtil was also low, then we can afford to stop limiting tokens. We
+ // are trying to carefully narrow this case since not limiting tokens can
+ // blow up bandwidth.
+ if d.loadLevel <= diskLoadModerate && util < d.lastUtil+smallDelta &&
+ d.lastUtil < lowUtilThreshold {
+ loadLevel = diskLoadLow
+ } else {
+ // util is increasing, or we just dropped from something higher than
+ // moderate. Give it more time at moderate, where we will gradually
+ // increase tokens.
+ loadLevel = diskLoadModerate
+ }
+ } else if util < moderateUtilThreshold {
+ // Wide band from [0.3,0.7) where we gradually increase tokens. Also, 0.7
+ // is deliberately a lowish fraction since the effect on compactions can
+ // lag and kick in later. We are ok with accepting a lower utilization for
+ // elastic work to make progress.
+ loadLevel = diskLoadModerate
+ } else if util < highUtilThreshold ||
+ (util < highlyOverUtilizedThreshold && util < d.lastUtil-smallDelta) {
+ // Wide band from [0.7,0.95) where we will hold the number of tokens
+ // steady. We don't want to overreact and decrease too early since
+ // compaction bandwidth usage can be lumpy. For this same reason, if we
+ // are trending downward, we want to hold. Note that util < 2 will always
+ // be true in typical configurations where one cannot actually exceed
+ // provisioned bandwidth -- but we also run experiments where we
+ // artificially constrain the provisioned bandwidth, where this is useful.
+ // And it is possible that some production settings may set a slightly
+ // lower value of provisioned bandwidth, if they want to further reduce
+ // the probability of hitting the real provisioned bandwidth due to
+ // elastic work.
+ loadLevel = diskLoadHigh
+ } else {
+ // Overloaded. We will reduce tokens.
+ loadLevel = diskLoadOverload
+ }
+ *d = diskLoadWatcher{
+ lastInterval: lastInterval,
+ lastUtil: util,
+ loadLevel: loadLevel,
+ }
+ // TODO(sumeer): Use the history of fsync latency and the value in the
+ // current interval, and if high, increase the load level computed earlier.
+ // We shouldn't rely fully on syncLatencyMicros since (a) sync latency could
+ // arise due to an external unrelated outage, (b) some customers may set
+ // fsync to be a noop. As an alternative to sync latency, we could also
+ // consider looking at fluctuations of peak-rate that the WAL writer can
+ // sustain.
+}
+
+func (d *diskLoadWatcher) getLoadLevel() (level diskLoadLevel, unusedBandwidth int64) {
+ return d.loadLevel,
+ d.lastInterval.provisionedBandwidth - d.lastInterval.readBandwidth - d.lastInterval.writeBandwidth
+}
+
+func (d diskLoadWatcher) SafeFormat(p redact.SafePrinter, _ rune) {
+ p.Printf("disk bandwidth: read: %s, write: %s, provisioned: %s, util: %.2f",
+ humanizeutil.IBytes(d.lastInterval.readBandwidth),
+ humanizeutil.IBytes(d.lastInterval.writeBandwidth),
+ humanizeutil.IBytes(d.lastInterval.provisionedBandwidth), d.lastUtil)
+}
+
+// intervalLSMInfo provides stats about the LSM over an adjustmentInterval.
+type intervalLSMInfo struct {
+ // Flushed bytes + Ingested bytes seen by the LSM. Ingested bytes also incur
+ // the cost of writing a sstable, even though that is done outside Pebble.
+ // Ingested bytes don't cause WAL writes, but we ignore that difference for
+ // simplicity.
+ incomingBytes int64
+ // regularTokensUsed and elasticTokensUsed are the byte tokens used for
+ // regular and elastic work respectively. Each of these includes both
+ // writes that will get flushed and ingested bytes. The
+ // regularTokensUsed+elasticTokensUsed do not need to sum up to
+ // incomingBytes, since these stats are produced by different sources.
+ regularTokensUsed int64
+ elasticTokensUsed int64
+}
+
+// diskBandwidthLimiter produces tokens for elastic work.
+type diskBandwidthLimiter struct {
+ diskLoadWatcher diskLoadWatcher
+
+ smoothedIncomingBytes float64
+ smoothedElasticFraction float64
+ elasticTokens int64
+}
+
+func makeDiskBandwidthLimiter() diskBandwidthLimiter {
+ return diskBandwidthLimiter{
+ elasticTokens: math.MaxInt64,
+ }
+}
+
+// computeElasticTokens is called every adjustmentInterval.
+func (d *diskBandwidthLimiter) computeElasticTokens(
+ ctx context.Context, id intervalDiskLoadInfo, il intervalLSMInfo,
+) (elasticTokens int64) {
+ d.diskLoadWatcher.setIntervalInfo(ctx, id)
+ const alpha = 0.5
+ d.smoothedIncomingBytes = alpha*float64(il.incomingBytes) + (1-alpha)*d.smoothedIncomingBytes
+ var intElasticFraction float64
+ if il.regularTokensUsed+il.elasticTokensUsed > 0 {
+ intElasticFraction =
+ float64(il.elasticTokensUsed) / float64(il.regularTokensUsed+il.elasticTokensUsed)
+ d.smoothedElasticFraction = alpha*intElasticFraction + (1-alpha)*d.smoothedElasticFraction
+ }
+ intElasticBytes := int64(float64(il.incomingBytes) * intElasticFraction)
+
+ // NB: unusedBW can be negative, and that is ok -- we use it in logging
+ // below.
+ ll, unusedBW := d.diskLoadWatcher.getLoadLevel()
+
+ // The constants and other heuristics in the following logic can seem
+ // arbitrary: they were subject to some tuning and evolution based on the
+ // experiments in https://github.com/cockroachdb/cockroach/pull/82813 that
+ // used (a) an artificial provisioned bandwidth limit lower than the actual,
+ // to see how well the system stayed within that limit, (b) an actual
+ // provisioned bandwidth limit. The difficulty in general is that small
+ // changes can have outsize influence if a higher number of compactions
+ // start happening, or the compaction backlog is cleared.
+ //
+ // TODO(sumeer): experiment with a PID controller.
+ switch ll {
+ case diskLoadLow:
+ lastElasticTokens := d.elasticTokens
+ d.elasticTokens = math.MaxInt64
+ if d.elasticTokens != lastElasticTokens {
+ log.Infof(ctx, "diskBandwidthLimiter low load (%s)", d.diskLoadWatcher)
+ }
+ // else we stay in the common case of low bandwidth usage.
+ case diskLoadModerate:
+ tokensFullyUtilized := func() bool {
+ // elasticTokens == MaxInt64 is also considered fully utilized since we
+ // can never fully utilize unlimited tokens.
+ return d.elasticTokens == 0 || d.elasticTokens == math.MaxInt64 ||
+ (d.elasticTokens > 0 && float64(il.elasticTokensUsed)/float64(d.elasticTokens) >= 0.8)
+ }
+ if tokensFullyUtilized() {
+ // Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to
+ // elastic work.
+ elasticBytes := (d.smoothedElasticFraction + 0.1) * d.smoothedIncomingBytes
+ // Sometimes we see the tokens not increasing even though we are staying
+ // for multiple intervals at moderate. This is because the smoothed
+ // fraction and incoming bytes can be decreasing. We do want to increase
+ // tokens since we know there is spare capacity, so we try many ways
+ // (that don't look at smoothed numbers only). Also, we sometimes come
+ // here due to an overload=>moderate transition because compaction
+ // bandwidth usage can be lumpy (high when there is a backlog and then
+ // dropping severely) -- in that case we want to start increasing
+ // immediately, since we have likely decreased too much.
+ intBasedElasticTokens := (d.smoothedElasticFraction + 0.1) * float64(il.incomingBytes)
+ if elasticBytes < intBasedElasticTokens {
+ elasticBytes = intBasedElasticTokens
+ }
+ if elasticBytes < 1.1*float64(il.elasticTokensUsed) {
+ elasticBytes = 1.1 * float64(il.elasticTokensUsed)
+ }
+ d.elasticTokens = int64(elasticBytes)
+ if d.elasticTokens == 0 {
+ // Don't get stuck in a situation where smoothedIncomingBytes are 0.
+ d.elasticTokens = math.MaxInt64
+ }
+ log.Infof(ctx, "diskBandwidthLimiter moderate (%s): elastic-fr: %.2f, incoming-bytes: %s, "+
+ "unused-bw: %s/s, increasing elastic-tokens: %s", d.diskLoadWatcher,
+ d.smoothedElasticFraction, humanizeutil.IBytes(int64(d.smoothedIncomingBytes)),
+ humanizeutil.IBytes(unusedBW), humanizeutil.IBytes(int64(elasticBytes)))
+ } else {
+ log.Infof(ctx,
+ "diskBandwidthLimiter moderate (%s): elastic-fr: %.2f, incoming-bytes: %s, "+
+ "unused-bw: %s/s, under-utilized elastic-tokens: limit %s used %s",
+ d.diskLoadWatcher, d.smoothedElasticFraction,
+ humanizeutil.IBytes(int64(d.smoothedIncomingBytes)), humanizeutil.IBytes(unusedBW),
+ humanizeutil.IBytes(d.elasticTokens), humanizeutil.IBytes(il.elasticTokensUsed))
+ }
+ case diskLoadHigh:
+ log.Infof(ctx, "diskBandwidthLimiter high (%s): elastic-fr: %.2f, incoming-bytes: %s, "+
+ "unused-bw: %s/s, elastic-tokens: limit %s, used %s", d.diskLoadWatcher,
+ d.smoothedElasticFraction, humanizeutil.IBytes(int64(d.smoothedIncomingBytes)),
+ humanizeutil.IBytes(unusedBW),
+ humanizeutil.IBytes(d.elasticTokens), humanizeutil.IBytes(il.elasticTokensUsed))
+ case diskLoadOverload:
+ // Sometimes we come here after a low => overload transition. The
+ // intElasticBytes will be very high because tokens were unlimited. We
+ // don't want to use that as the starting point of the decrease if the
+ // smoothed value is lower. Hence, the min logic below, to try to dampen
+ // the increase quickly.
+ d.elasticTokens = intElasticBytes / 2
+ smoothedElasticBytes := int64(d.smoothedElasticFraction * d.smoothedIncomingBytes)
+ if smoothedElasticBytes/2 < d.elasticTokens {
+ d.elasticTokens = smoothedElasticBytes / 2
+ }
+ log.Infof(ctx, "diskBandwidthLimiter overload (%s): elastic-tokens: limit %s, used: %s",
+ d.diskLoadWatcher, humanizeutil.IBytes(d.elasticTokens),
+ humanizeutil.IBytes(il.elasticTokensUsed))
+ }
+ // We can end up with 0 elastic tokens here -- e.g. if intElasticBytes was 0
+ // but we were still overloaded because of compactions. The trouble with 0
+ // elastic tokens is that if we don't admit anything, we cannot correct an
+ // occasional poor estimate of the per-request bytes. So we decide to give
+ // out at least 1 token. A single elastic request should not be too big for
+ // this to matter.
+ d.elasticTokens = max(1, d.elasticTokens)
+ return d.elasticTokens
+}
diff --git a/pkg/util/admission/disk_bandwidth_test.go b/pkg/util/admission/disk_bandwidth_test.go
new file mode 100644
index 000000000000..1e9db8b56057
--- /dev/null
+++ b/pkg/util/admission/disk_bandwidth_test.go
@@ -0,0 +1,11 @@
+// Copyright 2022 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+package admission
diff --git a/pkg/util/admission/doc.go b/pkg/util/admission/doc.go
index 57c525408fc9..62cdfdd55fdf 100644
--- a/pkg/util/admission/doc.go
+++ b/pkg/util/admission/doc.go
@@ -48,6 +48,8 @@
// either in a comment here or a separate RFC.
//
+// TODO(sumeer): update with all the recent changes.
+
// Internal organization:
//
// The package is mostly structured as a set of interfaces that are meant to
diff --git a/pkg/util/admission/granter.go b/pkg/util/admission/granter.go
index da7188805b7e..cb5ffe3bf466 100644
--- a/pkg/util/admission/granter.go
+++ b/pkg/util/admission/granter.go
@@ -92,6 +92,19 @@ var MinFlushUtilizationFraction = settings.RegisterFloatSetting(
"consider lowering it slightly (after consultation with domain experts)", 0.5,
settings.PositiveFloat)
+// DiskBandwidthTokensForElasticEnabled controls whether the disk bandwidth
+// resource is considered as a possible bottleneck resource. When it becomes a
+// bottleneck, tokens for elastic work are limited based on available disk
+// bandwidth. The default is true since actually considering disk bandwidth as
+// a bottleneck resource requires additional configuration (outside the
+// admission package) to calculate the provisioned bandwidth.
+var DiskBandwidthTokensForElasticEnabled = settings.RegisterBoolSetting(
+ settings.SystemOnly,
+ "admission.disk_bandwidth_tokens.elastic.enabled",
+ "when true, and provisioned bandwidth for the disk corresponding to a store is configured, "+
+ "tokens for elastic work will be limited if disk bandwidth becomes a bottleneck",
+ true).WithPublic()
+
// grantChainID is the ID for a grant chain. See continueGrantChain for
// details.
type grantChainID uint64
@@ -121,6 +134,10 @@ type requester interface {
close()
}
+type requesterClose interface {
+ close()
+}
+
// grantKind represents the two kind of ways we grant admission: using a slot
// or a token. The slot terminology is akin to a scheduler, where a scheduling
// slot must be free for a thread to run. But unlike a scheduler, we don't
@@ -174,6 +191,9 @@ type granter interface {
// when the goroutine doing the work noticed that it had been granted, there
// is a possibility that that raced with cancellation.
//
+ // Do not use this for doing store IO-related token adjustments when work is
+ // done -- that should be done via granterWithStoreWriteDone.storeWriteDone.
+ //
// REQUIRES: count > 0. count == 1 for slots.
returnGrant(count int64)
// tookWithoutPermission informs the granter that a slot or tokens were
@@ -188,6 +208,9 @@ type granter interface {
// - Tokens: this is useful when the initial estimated tokens for a unit of
// work turned out to be an underestimate.
//
+ // Do not use this for doing store IO-related token adjustments when work is
+ // done -- that should be done via granterWithStoreWriteDone.storeWriteDone.
+ //
// REQUIRES: count > 0. count == 1 for slots.
tookWithoutPermission(count int64)
// continueGrantChain is called by the requester at some point after grant
@@ -346,31 +369,42 @@ const (
// grant because a shared resource (CPU or memory) is overloaded. For grant
// chains, this is a signal to terminate.
grantFailDueToSharedResource
- // grantFailLocal is returned when the granter is unable to grant due to a
- // local constraint -- insufficient tokens or slots.
+ // grantFailLocal is returned when the granter is unable to grant due to (a)
+ // a local constraint -- insufficient tokens or slots, or (b) no work is
+ // waiting.
grantFailLocal
)
-// granterWithLockedCalls is an extension of the granter and requester
-// interfaces that is used as an internal implementation detail of the
-// GrantCoordinator. Note that an implementer of granterWithLockedCalls is
-// mainly passing things through to the GrantCoordinator where the main logic
-// lives. The *Locked() methods are where the differences in slots and tokens
-// are handled.
+// granterWithLockedCalls is an encapsulation of typically one
+// granter-requester pair, and for kvStoreTokenGranter of two
+// granter-requester pairs. It is used as an internal implementation detail of
+// the GrantCoordinator. An implementer of granterWithLockedCalls responds to
+// calls from its granter(s) by calling into the GrantCoordinator, which then
+// calls the various *Locked() methods. The demuxHandle is meant to be opaque
+// to the GrantCoordinator, and is used when this interface encapsulates
+// multiple granter-requester pairs. The *Locked() methods are where the
+// differences in slots and various kinds of tokens are handled.
type granterWithLockedCalls interface {
- granter
- // tryGetLocked is the real implementation of tryGet in the granter interface.
- // Additionally, it is also used when continuing a grant chain.
- tryGetLocked(count int64) grantResult
- // returnGrantLocked is the real implementation of returnGrant.
- returnGrantLocked(count int64)
+ // tryGetLocked is the real implementation of tryGet from the granter
+ // interface. demuxHandle is an opaque handle that was passed into the
+ // GrantCoordinator.
+ tryGetLocked(count int64, demuxHandle int8) grantResult
+ // returnGrantLocked is the real implementation of returnGrant from the
+ // granter interface. demuxHandle is an opaque handle that was passed into
+ // the GrantCoordinator.
+ returnGrantLocked(count int64, demuxHandle int8)
// tookWithoutPermissionLocked is the real implementation of
- // tookWithoutPermission.
- tookWithoutPermissionLocked(count int64)
+ // tookWithoutPermission from the granter interface. demuxHandle is an
+ // opaque handle that was passed into the GrantCoordinator.
+ tookWithoutPermissionLocked(count int64, demuxHandle int8)
- // getPairedRequester returns the requester implementation that this granter
- // interacts with.
- getPairedRequester() requester
+ // The following methods are for direct use by GrantCoordinator.
+
+ // requesterHasWaitingRequests returns whether some requester associated
+ // with the granter has waiting requests.
+ requesterHasWaitingRequests() bool
+ // tryGrantLocked is used to attempt to grant to waiting requests.
+ tryGrantLocked(grantChainID grantChainID) grantResult
}
// For the cpu-bound slot case we have background activities (like Pebble
@@ -438,11 +472,7 @@ type slotGranter struct {
}
var _ granterWithLockedCalls = &slotGranter{}
-
-// getPairedRequester implements granterWithLockedCalls.
-func (sg *slotGranter) getPairedRequester() requester {
- return sg.requester
-}
+var _ granter = &slotGranter{}
// grantKind implements granter.
func (sg *slotGranter) grantKind() grantKind {
@@ -451,11 +481,11 @@ func (sg *slotGranter) grantKind() grantKind {
// tryGet implements granter.
func (sg *slotGranter) tryGet(count int64) bool {
- return sg.coord.tryGet(sg.workKind, count)
+ return sg.coord.tryGet(sg.workKind, count, 0 /*arbitrary*/)
}
// tryGetLocked implements granterWithLockedCalls.
-func (sg *slotGranter) tryGetLocked(count int64) grantResult {
+func (sg *slotGranter) tryGetLocked(count int64, _ int8) grantResult {
if count != 1 {
panic(errors.AssertionFailedf("unexpected count: %d", count))
}
@@ -475,7 +505,7 @@ func (sg *slotGranter) tryGetLocked(count int64) grantResult {
// returnGrant implements granter.
func (sg *slotGranter) returnGrant(count int64) {
- sg.coord.returnGrant(sg.workKind, count)
+ sg.coord.returnGrant(sg.workKind, count, 0 /*arbitrary*/)
}
func (sg *slotGranter) tryGetSoftSlots(count int) int {
@@ -505,7 +535,7 @@ func (sg *slotGranter) returnSoftSlots(count int) {
}
// returnGrantLocked implements granterWithLockedCalls.
-func (sg *slotGranter) returnGrantLocked(count int64) {
+func (sg *slotGranter) returnGrantLocked(count int64, _ int8) {
if count != 1 {
panic(errors.AssertionFailedf("unexpected count: %d", count))
}
@@ -518,11 +548,11 @@ func (sg *slotGranter) returnGrantLocked(count int64) {
// tookWithoutPermission implements granter.
func (sg *slotGranter) tookWithoutPermission(count int64) {
- sg.coord.tookWithoutPermission(sg.workKind, count)
+ sg.coord.tookWithoutPermission(sg.workKind, count, 0 /*arbitrary*/)
}
// tookWithoutPermissionLocked implements granterWithLockedCalls.
-func (sg *slotGranter) tookWithoutPermissionLocked(count int64) {
+func (sg *slotGranter) tookWithoutPermissionLocked(count int64, _ int8) {
if count != 1 {
panic(errors.AssertionFailedf("unexpected count: %d", count))
}
@@ -535,6 +565,27 @@ func (sg *slotGranter) continueGrantChain(grantChainID grantChainID) {
sg.coord.continueGrantChain(sg.workKind, grantChainID)
}
+// requesterHasWaitingRequests implements granterWithLockedCalls.
+func (sg *slotGranter) requesterHasWaitingRequests() bool {
+ return sg.requester.hasWaitingRequests()
+}
+
+// tryGrantLocked implements granterWithLockedCalls.
+func (sg *slotGranter) tryGrantLocked(grantChainID grantChainID) grantResult {
+ res := sg.tryGetLocked(1, 0 /*arbitrary*/)
+ if res == grantSuccess {
+ slots := sg.requester.granted(grantChainID)
+ if slots == 0 {
+ // Did not accept grant
+ sg.returnGrantLocked(1, 0 /*arbitrary*/)
+ return grantFailLocal
+ } else if slots != 1 {
+ panic(errors.AssertionFailedf("unexpected count %d", slots))
+ }
+ }
+ return res
+}
+
// tokenGranter implements granterWithLockedCalls.
type tokenGranter struct {
coord *GrantCoordinator
@@ -552,11 +603,7 @@ type tokenGranter struct {
}
var _ granterWithLockedCalls = &tokenGranter{}
-
-// getPairedRequester implements granterWithLockedCalls.
-func (tg *tokenGranter) getPairedRequester() requester {
- return tg.requester
-}
+var _ granter = &tokenGranter{}
func (tg *tokenGranter) refillBurstTokens(skipTokenEnforcement bool) {
tg.availableBurstTokens = tg.maxBurstTokens
@@ -570,11 +617,11 @@ func (tg *tokenGranter) grantKind() grantKind {
// tryGet implements granter.
func (tg *tokenGranter) tryGet(count int64) bool {
- return tg.coord.tryGet(tg.workKind, count)
+ return tg.coord.tryGet(tg.workKind, count, 0 /*arbitrary*/)
}
// tryGetLocked implements granterWithLockedCalls.
-func (tg *tokenGranter) tryGetLocked(count int64) grantResult {
+func (tg *tokenGranter) tryGetLocked(count int64, _ int8) grantResult {
if tg.cpuOverload != nil && tg.cpuOverload.isOverloaded() {
return grantFailDueToSharedResource
}
@@ -587,11 +634,11 @@ func (tg *tokenGranter) tryGetLocked(count int64) grantResult {
// returnGrant implements granter.
func (tg *tokenGranter) returnGrant(count int64) {
- tg.coord.returnGrant(tg.workKind, count)
+ tg.coord.returnGrant(tg.workKind, count, 0 /*arbitrary*/)
}
// returnGrantLocked implements granterWithLockedCalls.
-func (tg *tokenGranter) returnGrantLocked(count int64) {
+func (tg *tokenGranter) returnGrantLocked(count int64, _ int8) {
tg.availableBurstTokens += count
if tg.availableBurstTokens > tg.maxBurstTokens {
tg.availableBurstTokens = tg.maxBurstTokens
@@ -600,11 +647,11 @@ func (tg *tokenGranter) returnGrantLocked(count int64) {
// tookWithoutPermission implements granter.
func (tg *tokenGranter) tookWithoutPermission(count int64) {
- tg.coord.tookWithoutPermission(tg.workKind, count)
+ tg.coord.tookWithoutPermission(tg.workKind, count, 0 /*arbitrary*/)
}
// tookWithoutPermissionLocked implements granterWithLockedCalls.
-func (tg *tokenGranter) tookWithoutPermissionLocked(count int64) {
+func (tg *tokenGranter) tookWithoutPermissionLocked(count int64, _ int8) {
tg.availableBurstTokens -= count
}
@@ -613,11 +660,49 @@ func (tg *tokenGranter) continueGrantChain(grantChainID grantChainID) {
tg.coord.continueGrantChain(tg.workKind, grantChainID)
}
+// requesterHasWaitingRequests implements granterWithLockedCalls.
+func (tg *tokenGranter) requesterHasWaitingRequests() bool {
+ return tg.requester.hasWaitingRequests()
+}
+
+// tryGrantLocked implements granterWithLockedCalls.
+func (tg *tokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult {
+ res := tg.tryGetLocked(1, 0 /*arbitrary*/)
+ if res == grantSuccess {
+ tokens := tg.requester.granted(grantChainID)
+ if tokens == 0 {
+ // Did not accept grant
+ tg.returnGrantLocked(1, 0 /*arbitrary*/)
+ return grantFailLocal
+ } else if tokens > 1 {
+ tg.tookWithoutPermissionLocked(tokens-1, 0 /*arbitrary*/)
+ }
+ }
+ return res
+}
+
+type workClass int8
+
+const (
+ // regularWorkClass is for work corresponding to workloads that are
+ // throughput and latency sensitive.
+ regularWorkClass workClass = iota
+ // elasticWorkClass is for work corresponding to workloads that can handle
+ // reduced throughput, possibly by taking longer to finish a workload. It is
+ // not latency sensitive.
+ elasticWorkClass
+ numWorkClasses
+)
+
// kvStoreTokenGranter implements granterWithLockedCalls. It is used for
-// grants to KVWork to a store, that is limited by IO tokens.
+// grants to KVWork to a store, that is limited by IO tokens. It encapsulates
+// two granter-requester pairs, for the two workClasses. The granter in these
+// pairs is implemented by kvStoreTokenChildGranter, and the requester by
+// WorkQueue.
type kvStoreTokenGranter struct {
- coord *GrantCoordinator
- requester requester
+ coord *GrantCoordinator
+ regularRequester requester
+ elasticRequester requester
// There is no rate limiting in granting these tokens. That is, they are all
// burst tokens.
availableIOTokens int64
@@ -628,55 +713,117 @@ type kvStoreTokenGranter struct {
ioTokensExhaustedDurationMetric *metric.Counter
exhaustedStart time.Time
- writeLM, ingestedLM tokensLinearModel
+ // Disk bandwidth tokens.
+ elasticDiskBWTokensAvailable int64
+ diskBWTokensUsed [numWorkClasses]int64
+
+ // Estimation models.
+ writeLM, ingestedLM, diskBWLM tokensLinearModel
}
var _ granterWithLockedCalls = &kvStoreTokenGranter{}
var _ granterWithIOTokens = &kvStoreTokenGranter{}
-var _ granterWithStoreWriteDone = &kvStoreTokenGranter{}
-// getPairedRequester implements granterWithLockedCalls.
-func (sg *kvStoreTokenGranter) getPairedRequester() requester {
- return sg.requester
+// kvStoreTokenChildGranter handles a particular workClass. Its methods
+// pass-through to the parent after adding the workClass as a parameter.
+type kvStoreTokenChildGranter struct {
+ workClass workClass
+ parent *kvStoreTokenGranter
}
+var _ granterWithStoreWriteDone = &kvStoreTokenChildGranter{}
+var _ granter = &kvStoreTokenChildGranter{}
+
// grantKind implements granter.
-func (sg *kvStoreTokenGranter) grantKind() grantKind {
+func (cg *kvStoreTokenChildGranter) grantKind() grantKind {
return token
}
// tryGet implements granter.
-func (sg *kvStoreTokenGranter) tryGet(count int64) bool {
- return sg.coord.tryGet(KVWork, count)
+func (cg *kvStoreTokenChildGranter) tryGet(count int64) bool {
+ return cg.parent.tryGet(cg.workClass, count)
+}
+
+// returnGrant implements granter.
+func (cg *kvStoreTokenChildGranter) returnGrant(count int64) {
+ cg.parent.returnGrant(cg.workClass, count)
+}
+
+// tookWithoutPermission implements granter.
+func (cg *kvStoreTokenChildGranter) tookWithoutPermission(count int64) {
+ cg.parent.tookWithoutPermission(cg.workClass, count)
+}
+
+// continueGrantChain implements granter.
+func (cg *kvStoreTokenChildGranter) continueGrantChain(grantChainID grantChainID) {
+ // Ignore since grant chains are not used for store tokens.
+}
+
+// storeWriteDone implements granterWithStoreWriteDone.
+func (cg *kvStoreTokenChildGranter) storeWriteDone(
+ originalTokens int64, doneInfo StoreWorkDoneInfo,
+) (additionalTokens int64) {
+ return cg.parent.storeWriteDone(cg.workClass, originalTokens, doneInfo)
+}
+
+func (sg *kvStoreTokenGranter) tryGet(workClass workClass, count int64) bool {
+ return sg.coord.tryGet(KVWork, count, int8(workClass))
}
// tryGetLocked implements granterWithLockedCalls.
-func (sg *kvStoreTokenGranter) tryGetLocked(count int64) grantResult {
- if sg.availableIOTokens > 0 {
- sg.subtractTokens(count, false)
- return grantSuccess
+func (sg *kvStoreTokenGranter) tryGetLocked(count int64, demuxHandle int8) grantResult {
+ wc := workClass(demuxHandle)
+ // NB: ideally if regularRequester.hasWaitingRequests() returns true and
+ // wc==elasticWorkClass we should reject this request, since it means that
+ // more important regular work is waiting. However, we rely on the
+ // assumption that elasticWorkClass, once throttled, will have a non-empty
+ // queue, and since the only case where tryGetLocked is called for
+ // elasticWorkClass is when the queue is empty, this case should be rare
+ // (and not cause a performance isolation failure).
+ switch wc {
+ case regularWorkClass:
+ if sg.availableIOTokens > 0 {
+ sg.subtractTokens(count, false)
+ sg.diskBWTokensUsed[wc] += count
+ return grantSuccess
+ }
+ case elasticWorkClass:
+ if sg.elasticDiskBWTokensAvailable > 0 && sg.availableIOTokens > 0 {
+ sg.elasticDiskBWTokensAvailable -= count
+ sg.subtractTokens(count, false)
+ sg.diskBWTokensUsed[wc] += count
+ return grantSuccess
+ }
}
return grantFailLocal
}
-// returnGrant implements granter.
-func (sg *kvStoreTokenGranter) returnGrant(count int64) {
- sg.coord.returnGrant(KVWork, count)
+func (sg *kvStoreTokenGranter) returnGrant(workClass workClass, count int64) {
+ sg.coord.returnGrant(KVWork, count, int8(workClass))
}
// returnGrantLocked implements granterWithLockedCalls.
-func (sg *kvStoreTokenGranter) returnGrantLocked(count int64) {
+func (sg *kvStoreTokenGranter) returnGrantLocked(count int64, demuxHandle int8) {
+ wc := workClass(demuxHandle)
sg.subtractTokens(-count, false)
+ if wc == elasticWorkClass {
+ sg.elasticDiskBWTokensAvailable += count
+ }
+ sg.diskBWTokensUsed[wc] -= count
}
-// tookWithoutPermission implements granter.
-func (sg *kvStoreTokenGranter) tookWithoutPermission(count int64) {
- sg.coord.tookWithoutPermission(KVWork, count)
+func (sg *kvStoreTokenGranter) tookWithoutPermission(workClass workClass, count int64) {
+ sg.coord.tookWithoutPermission(KVWork, count, int8(workClass))
}
// tookWithoutPermissionLocked implements granterWithLockedCalls.
-func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64) {
+func (sg *kvStoreTokenGranter) tookWithoutPermissionLocked(count int64, demuxHandle int8) {
+ wc := workClass(demuxHandle)
sg.subtractTokens(count, false)
+ if wc == elasticWorkClass {
+ sg.elasticDiskBWTokensAvailable -= count
+ }
+ sg.diskBWTokensUsed[wc] += count
}
// subtractTokens is a helper function that subtracts count tokens (count can
@@ -701,9 +848,45 @@ func (sg *kvStoreTokenGranter) subtractTokens(count int64, forceTickMetric bool)
}
}
-// continueGrantChain implements granter.
-func (sg *kvStoreTokenGranter) continueGrantChain(grantChainID grantChainID) {
- sg.coord.continueGrantChain(KVWork, grantChainID)
+// requesterHasWaitingRequests implements granterWithLockedCalls.
+func (sg *kvStoreTokenGranter) requesterHasWaitingRequests() bool {
+ return sg.regularRequester.hasWaitingRequests() || sg.elasticRequester.hasWaitingRequests()
+}
+
+// tryGrantLocked implements granterWithLockedCalls.
+func (sg *kvStoreTokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult {
+ // First try granting to regular requester.
+ for wc := range sg.diskBWTokensUsed {
+ req := sg.regularRequester
+ if workClass(wc) == elasticWorkClass {
+ req = sg.elasticRequester
+ }
+ if req.hasWaitingRequests() {
+ res := sg.tryGetLocked(1, int8(wc))
+ if res == grantSuccess {
+ tookTokenCount := req.granted(grantChainID)
+ if tookTokenCount == 0 {
+ // Did not accept grant.
+ sg.returnGrantLocked(1, int8(wc))
+ // Continue with the loop since this requester does not have waiting
+ // requests. If the loop terminates we will correctly return
+ // grantFailLocal.
+ } else {
+ // May have taken more.
+ if tookTokenCount > 1 {
+ sg.tookWithoutPermissionLocked(tookTokenCount-1, int8(wc))
+ }
+ return grantSuccess
+ }
+ } else {
+ // Was not able to get token. Do not continue with looping to grant to
+ // less important work (though it would be harmless since won't be
+ // able to get a token for that either).
+ return res
+ }
+ }
+ }
+ return grantFailLocal
}
// setAvailableIOTokensLocked implements granterWithIOTokens.
@@ -722,17 +905,36 @@ func (sg *kvStoreTokenGranter) setAvailableIOTokensLocked(tokens int64) (tokensU
return tokensUsed
}
+// setAvailableElasticDiskBandwidthTokensLocked implements
+// granterWithIOTokens.
+func (sg *kvStoreTokenGranter) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) {
+ sg.elasticDiskBWTokensAvailable += tokens
+ if sg.elasticDiskBWTokensAvailable > tokens {
+ sg.elasticDiskBWTokensAvailable = tokens
+ }
+}
+
+// getDiskTokensUsedAndResetLocked implements granterWithIOTokens.
+func (sg *kvStoreTokenGranter) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 {
+ result := sg.diskBWTokensUsed
+ for i := range sg.diskBWTokensUsed {
+ sg.diskBWTokensUsed[i] = 0
+ }
+ return result
+}
+
// setAdmittedModelsLocked implements granterWithIOTokens.
func (sg *kvStoreTokenGranter) setAdmittedDoneModelsLocked(
- writeLM tokensLinearModel, ingestedLM tokensLinearModel,
+ writeLM tokensLinearModel, ingestedLM tokensLinearModel, diskBWLM tokensLinearModel,
) {
sg.writeLM = writeLM
sg.ingestedLM = ingestedLM
+ sg.diskBWLM = diskBWLM
}
// storeWriteDone implements granterWithStoreWriteDone.
func (sg *kvStoreTokenGranter) storeWriteDone(
- originalTokens int64, doneInfo StoreWorkDoneInfo,
+ wc workClass, originalTokens int64, doneInfo StoreWorkDoneInfo,
) (additionalTokens int64) {
// Normally, we follow the structure of a foo() method calling into a foo()
// method on the GrantCoordinator, which then calls fooLocked() on the
@@ -751,7 +953,8 @@ func (sg *kvStoreTokenGranter) storeWriteDone(
// Reminder: coord.mu protects the state in the kvStoreTokenGranter.
sg.coord.mu.Lock()
exhaustedFunc := func() bool {
- return sg.availableIOTokens <= 0
+ return sg.availableIOTokens <= 0 ||
+ (wc == elasticWorkClass && sg.elasticDiskBWTokensAvailable <= 0)
}
wasExhausted := exhaustedFunc()
actualTokens :=
@@ -759,13 +962,25 @@ func (sg *kvStoreTokenGranter) storeWriteDone(
int64(float64(doneInfo.IngestedBytes)*sg.ingestedLM.multiplier) + sg.ingestedLM.constant
additionalTokensNeeded := actualTokens - originalTokens
sg.subtractTokens(additionalTokensNeeded, false)
- if additionalTokensNeeded < 0 {
+ actualDiskBWTokens :=
+ int64(float64(doneInfo.WriteBytes+doneInfo.IngestedBytes)*sg.diskBWLM.multiplier) +
+ sg.diskBWLM.constant
+ additionalDiskBWTokensNeeded := actualDiskBWTokens - originalTokens
+ if wc == elasticWorkClass {
+ sg.elasticDiskBWTokensAvailable -= additionalDiskBWTokensNeeded
+ }
+ sg.diskBWTokensUsed[wc] += additionalDiskBWTokensNeeded
+ if additionalTokensNeeded < 0 || additionalDiskBWTokensNeeded < 0 {
isExhausted := exhaustedFunc()
if wasExhausted && !isExhausted {
sg.coord.tryGrant()
}
}
sg.coord.mu.Unlock()
+ // For multi-tenant fairness accounting, we choose to ignore disk bandwidth
+ // tokens. Ideally, we'd have multiple resource dimensions for the fairness
+ // decisions, but we don't necessarily need something more sophisticated
+ // like "Dominant Resource Fairness".
return additionalTokensNeeded
}
@@ -788,8 +1003,9 @@ type GrantCoordinator struct {
// NB: Some granters can be nil.
granters [numWorkKinds]granterWithLockedCalls
// The WorkQueues behaving as requesters in each granterWithLockedCalls.
- // This is kept separately only to service GetWorkQueue calls.
- queues [numWorkKinds]requester
+ // This is kept separately only to service GetWorkQueue calls and to call
+ // close().
+ queues [numWorkKinds]requesterClose
// The cpu fields can be nil, and the IO field can be nil, since a
// GrantCoordinator typically handles one of these two resources.
cpuOverloadIndicator cpuOverloadIndicator
@@ -893,8 +1109,8 @@ type makeRequesterFunc func(
opts workQueueOptions) requester
type makeStoreRequesterFunc func(
- _ log.AmbientContext, granter granterWithStoreWriteDone, settings *cluster.Settings,
- opts workQueueOptions) storeRequester
+ _ log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone,
+ settings *cluster.Settings, opts workQueueOptions) storeRequester
// NewGrantCoordinators constructs GrantCoordinators and WorkQueues for a
// regular cluster node. Caller is responsible for hooking up
@@ -950,8 +1166,9 @@ func NewGrantCoordinators(
}
kvSlotAdjuster.granter = kvg
- coord.queues[KVWork] = makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork))
- kvg.requester = coord.queues[KVWork]
+ req := makeRequester(ambientCtx, KVWork, kvg, st, makeWorkQueueOptions(KVWork))
+ coord.queues[KVWork] = req
+ kvg.requester = req
coord.granters[KVWork] = kvg
tg := &tokenGranter{
@@ -961,9 +1178,10 @@ func NewGrantCoordinators(
maxBurstTokens: opts.SQLKVResponseBurstTokens,
cpuOverload: kvSlotAdjuster,
}
- coord.queues[SQLKVResponseWork] = makeRequester(
+ req = makeRequester(
ambientCtx, SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork))
- tg.requester = coord.queues[SQLKVResponseWork]
+ coord.queues[SQLKVResponseWork] = req
+ tg.requester = req
coord.granters[SQLKVResponseWork] = tg
tg = &tokenGranter{
@@ -973,9 +1191,10 @@ func NewGrantCoordinators(
maxBurstTokens: opts.SQLSQLResponseBurstTokens,
cpuOverload: kvSlotAdjuster,
}
- coord.queues[SQLSQLResponseWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork))
- tg.requester = coord.queues[SQLSQLResponseWork]
+ coord.queues[SQLSQLResponseWork] = req
+ tg.requester = req
coord.granters[SQLSQLResponseWork] = tg
sg := &slotGranter{
@@ -985,9 +1204,10 @@ func NewGrantCoordinators(
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
- coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork))
- sg.requester = coord.queues[SQLStatementLeafStartWork]
+ coord.queues[SQLStatementLeafStartWork] = req
+ sg.requester = req
coord.granters[SQLStatementLeafStartWork] = sg
sg = &slotGranter{
@@ -997,13 +1217,18 @@ func NewGrantCoordinators(
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
- coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork))
- sg.requester = coord.queues[SQLStatementRootStartWork]
+ coord.queues[SQLStatementRootStartWork] = req
+ sg.requester = req
coord.granters[SQLStatementRootStartWork] = sg
metricStructs = appendMetricStructsForQueues(metricStructs, coord)
+ // TODO(sumeerbhola): these metrics are shared across all stores and all
+ // priorities across stores (even the coarser workClasses, which are a
+ // mapping from priority, share the same metrics). Fix this by adding
+ // labeled Prometheus metrics.
storeWorkQueueMetrics := makeWorkQueueMetrics(string(workKindString(KVWork)) + "-stores")
metricStructs = append(metricStructs, storeWorkQueueMetrics)
makeStoreRequester := makeStoreWorkQueue
@@ -1052,9 +1277,10 @@ func NewGrantCoordinatorSQL(
maxBurstTokens: opts.SQLKVResponseBurstTokens,
cpuOverload: sqlNodeCPU,
}
- coord.queues[SQLKVResponseWork] = makeRequester(ambientCtx,
+ req := makeRequester(ambientCtx,
SQLKVResponseWork, tg, st, makeWorkQueueOptions(SQLKVResponseWork))
- tg.requester = coord.queues[SQLKVResponseWork]
+ coord.queues[SQLKVResponseWork] = req
+ tg.requester = req
coord.granters[SQLKVResponseWork] = tg
tg = &tokenGranter{
@@ -1064,9 +1290,10 @@ func NewGrantCoordinatorSQL(
maxBurstTokens: opts.SQLSQLResponseBurstTokens,
cpuOverload: sqlNodeCPU,
}
- coord.queues[SQLSQLResponseWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLSQLResponseWork, tg, st, makeWorkQueueOptions(SQLSQLResponseWork))
- tg.requester = coord.queues[SQLSQLResponseWork]
+ coord.queues[SQLSQLResponseWork] = req
+ tg.requester = req
coord.granters[SQLSQLResponseWork] = tg
sg := &slotGranter{
@@ -1076,9 +1303,10 @@ func NewGrantCoordinatorSQL(
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
- coord.queues[SQLStatementLeafStartWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLStatementLeafStartWork, sg, st, makeWorkQueueOptions(SQLStatementLeafStartWork))
- sg.requester = coord.queues[SQLStatementLeafStartWork]
+ coord.queues[SQLStatementLeafStartWork] = req
+ sg.requester = req
coord.granters[SQLStatementLeafStartWork] = sg
sg = &slotGranter{
@@ -1088,9 +1316,10 @@ func NewGrantCoordinatorSQL(
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
- coord.queues[SQLStatementRootStartWork] = makeRequester(ambientCtx,
+ req = makeRequester(ambientCtx,
SQLStatementRootStartWork, sg, st, makeWorkQueueOptions(SQLStatementRootStartWork))
- sg.requester = coord.queues[SQLStatementRootStartWork]
+ coord.queues[SQLStatementRootStartWork] = req
+ sg.requester = req
coord.granters[SQLStatementRootStartWork] = sg
return coord, appendMetricStructsForQueues(metricStructs, coord)
@@ -1192,14 +1421,14 @@ func (coord *GrantCoordinator) CPULoad(runnable int, procs int, samplePeriod tim
}
// tryGet is called by granter.tryGet with the WorkKind.
-func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64) bool {
+func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64, demuxHandle int8) bool {
coord.mu.Lock()
defer coord.mu.Unlock()
// It is possible that a grant chain is active, and has not yet made its way
// to this workKind. So it may be more reasonable to queue. But we have some
// concerns about incurring the delay of multiple goroutine context switches
// so we ignore this case.
- res := coord.granters[workKind].tryGetLocked(count)
+ res := coord.granters[workKind].tryGetLocked(count, demuxHandle)
switch res {
case grantSuccess:
// Grant chain may be active, but it did not get in the way of this grant,
@@ -1222,13 +1451,13 @@ func (coord *GrantCoordinator) tryGet(workKind WorkKind, count int64) bool {
}
// returnGrant is called by granter.returnGrant with the WorkKind.
-func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64) {
+func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64, demuxHandle int8) {
coord.mu.Lock()
defer coord.mu.Unlock()
- coord.granters[workKind].returnGrantLocked(count)
+ coord.granters[workKind].returnGrantLocked(count, demuxHandle)
if coord.grantChainActive {
if coord.grantChainIndex > workKind &&
- coord.granters[workKind].getPairedRequester().hasWaitingRequests() {
+ coord.granters[workKind].requesterHasWaitingRequests() {
// There are waiting requests that will not be served by the grant chain.
// Better to terminate it and start afresh.
if !coord.tryTerminateGrantChain() {
@@ -1244,10 +1473,12 @@ func (coord *GrantCoordinator) returnGrant(workKind WorkKind, count int64) {
// tookWithoutPermission is called by granter.tookWithoutPermission with the
// WorkKind.
-func (coord *GrantCoordinator) tookWithoutPermission(workKind WorkKind, count int64) {
+func (coord *GrantCoordinator) tookWithoutPermission(
+ workKind WorkKind, count int64, demuxHandle int8,
+) {
coord.mu.Lock()
defer coord.mu.Unlock()
- coord.granters[workKind].tookWithoutPermissionLocked(count)
+ coord.granters[workKind].tookWithoutPermissionLocked(count, demuxHandle)
}
// continueGrantChain is called by granter.continueGrantChain with the
@@ -1341,33 +1572,21 @@ OuterLoop:
// remaining will be nil.
continue
}
- req := granter.getPairedRequester()
- for req.hasWaitingRequests() && !localDone {
- // Get 1 token or slot.
- res := granter.tryGetLocked(1)
+ for granter.requesterHasWaitingRequests() && !localDone {
+ chainID := noGrantChain
+ if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains {
+ chainID = coord.grantChainID
+ }
+ res := granter.tryGrantLocked(chainID)
switch res {
case grantSuccess:
- chainID := noGrantChain
- if grantBurstCount+1 == grantBurstLimit && coord.useGrantChains {
- chainID = coord.grantChainID
- }
- tookCount := req.granted(chainID)
- if tookCount == 0 {
- // Did not accept grant.
- granter.returnGrantLocked(1)
- } else {
- // May have taken more.
- if tookCount > 1 {
- granter.tookWithoutPermissionLocked(tookCount - 1)
- }
- grantBurstCount++
- if grantBurstCount == grantBurstLimit && coord.useGrantChains {
- coord.grantChainActive = true
- if startingChain {
- coord.grantChainStartTime = timeutil.Now()
- }
- return
+ grantBurstCount++
+ if grantBurstCount == grantBurstLimit && coord.useGrantChains {
+ coord.grantChainActive = true
+ if startingChain {
+ coord.grantChainStartTime = timeutil.Now()
}
+ return
}
case grantFailDueToSharedResource:
break OuterLoop
@@ -1426,7 +1645,8 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, verb rune) {
s.Printf(" used-soft: %d", g.usedSoftSlots)
}
case *kvStoreTokenGranter:
- s.Printf(" io-avail: %d", g.availableIOTokens)
+ s.Printf(" io-avail: %d, elastic-disk-bw-tokens-avail: %d", g.availableIOTokens,
+ g.elasticDiskBWTokensAvailable)
}
case SQLStatementLeafStartWork, SQLStatementRootStartWork:
if coord.granters[i] != nil {
@@ -1566,8 +1786,14 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo
}
kvg := &kvStoreTokenGranter{
- coord: coord,
+ coord: coord,
+ // Setting tokens to unlimited is defensive. We expect that
+ // pebbleMetricsTick and allocateIOTokensTick will get called during
+ // initialization, which will also set these to unlimited.
+ availableIOTokens: unlimitedTokens / ticksInAdjustmentInterval,
+ startingIOTokens: unlimitedTokens / ticksInAdjustmentInterval,
ioTokensExhaustedDurationMetric: sgc.kvIOTokensExhaustedDuration,
+ elasticDiskBWTokensAvailable: unlimitedTokens / ticksInAdjustmentInterval,
}
opts := makeWorkQueueOptions(KVWork)
// This is IO work, so override the usesTokens value.
@@ -1575,15 +1801,29 @@ func (sgc *StoreGrantCoordinators) initGrantCoordinator(storeID int32) *GrantCoo
// Share the WorkQueue metrics across all stores.
// TODO(sumeer): add per-store WorkQueue state for debug.zip and db console.
opts.metrics = &sgc.workQueueMetrics
- storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, kvg, sgc.settings, opts)
+ granters := [numWorkClasses]granterWithStoreWriteDone{
+ &kvStoreTokenChildGranter{
+ workClass: regularWorkClass,
+ parent: kvg,
+ },
+ &kvStoreTokenChildGranter{
+ workClass: elasticWorkClass,
+ parent: kvg,
+ },
+ }
+
+ storeReq := sgc.makeStoreRequesterFunc(sgc.ambientCtx, granters, sgc.settings, opts)
coord.queues[KVWork] = storeReq
- kvg.requester = storeReq
+ requesters := storeReq.getRequesters()
+ kvg.regularRequester = requesters[regularWorkClass]
+ kvg.elasticRequester = requesters[elasticWorkClass]
coord.granters[KVWork] = kvg
coord.ioLoadListener = &ioLoadListener{
storeID: storeID,
settings: sgc.settings,
kvRequester: storeReq,
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
+ diskBandwidthLimiter: makeDiskBandwidthLimiter(),
}
coord.ioLoadListener.mu.Mutex = &coord.mu
coord.ioLoadListener.mu.kvGranter = kvg
@@ -1860,6 +2100,26 @@ type StoreMetrics struct {
*pebble.Metrics
WriteStallCount int64
*pebble.InternalIntervalMetrics
+ // Optional.
+ DiskStats DiskStats
+}
+
+// DiskStats provide low-level stats about the disk resources used for a
+// store. We assume that the disk is not shared across multiple stores.
+// However, transient and moderate usage that is not due to the store is
+// tolerable, since the diskBandwidthLimiter is only using this to compute
+// elastic tokens and is designed to deal with significant attribution
+// uncertainty.
+//
+// DiskStats are not always populated. A ProvisionedBandwidth of 0 represents
+// that the stats should be ignored.
+type DiskStats struct {
+ // BytesRead is the cumulative bytes read.
+ BytesRead uint64
+ // BytesWritten is the cumulative bytes written.
+ BytesWritten uint64
+ // ProvisionedBandwidth is the total provisioned bandwidth in bytes/s.
+ ProvisionedBandwidth int64
}
// granterWithIOTokens is used to abstract kvStoreTokenGranter for testing.
@@ -1876,6 +2136,13 @@ type granterWithIOTokens interface {
// that tokensUsed can be negative, though that will be rare, since it is
// possible for tokens to be returned.
setAvailableIOTokensLocked(tokens int64) (tokensUsed int64)
+ // setAvailableElasticDiskBandwidthTokensLocked bounds the available tokens
+ // that can be granted to elastic work. These tokens are based on disk
+ // bandwidth being a bottleneck resource.
+ setAvailableElasticDiskBandwidthTokensLocked(tokens int64)
+ // getDiskTokensUsedAndResetLocked returns the disk bandwidth tokens used
+ // since the last such call.
+ getDiskTokensUsedAndResetLocked() [numWorkClasses]int64
// setAdmittedDoneModelsLocked supplies the models to use when
// storeWriteDone is called, to adjust token consumption. Note that these
// models are not used for token adjustment at admission time -- that is
@@ -1883,7 +2150,8 @@ type granterWithIOTokens interface {
// asymmetry is due to the need to use all the functionality of WorkQueue at
// admission time. See the long explanatory comment at the beginning of
// store_token_estimation.go, regarding token estimation.
- setAdmittedDoneModelsLocked(writeLM tokensLinearModel, ingestedLM tokensLinearModel)
+ setAdmittedDoneModelsLocked(writeLM tokensLinearModel, ingestedLM tokensLinearModel,
+ diskBWLM tokensLinearModel)
}
// granterWithStoreWriteDone is used to abstract kvStoreTokenGranter for
@@ -1941,7 +2209,8 @@ type storeRequestEstimates struct {
// storeRequester is used to abstract *StoreWorkQueue for testing.
type storeRequester interface {
- requester
+ requesterClose
+ getRequesters() [numWorkClasses]requester
getStoreAdmissionStats() storeAdmissionStats
setStoreRequestEstimates(estimates storeRequestEstimates)
}
@@ -1953,6 +2222,12 @@ type ioLoadListenerState struct {
curL0Bytes int64
// Cumulative.
cumWriteStallCount int64
+ diskBW struct {
+ // Cumulative
+ bytesRead uint64
+ bytesWritten uint64
+ incomingLSMBytes uint64
+ }
// Exponentially smoothed per interval values.
@@ -1969,13 +2244,19 @@ type ioLoadListenerState struct {
flushUtilTargetFraction float64
// totalNumByteTokens represents the tokens to give out until the next call to
- // adjustTokens. They are parceled out in small intervals. tokensAllocated
+ // adjustTokens. They are parceled out in small intervals. byteTokensAllocated
// represents what has been given out.
- totalNumByteTokens int64
- tokensAllocated int64
+ totalNumByteTokens int64
+ byteTokensAllocated int64
// Used tokens can be negative if some tokens taken in one interval were
// returned in another, but that will be extremely rare.
- tokensUsed int64
+ byteTokensUsed int64
+
+ // elasticDiskBWTokens represents the tokens to give out until the next call
+ // to adjustTokens. They are parceled out in small intervals.
+ // elasticDiskTokensAllocated represents what has been given out.
+ elasticDiskBWTokens int64
+ elasticDiskBWTokensAllocated int64
}
// ioLoadListener adjusts tokens in kvStoreTokenGranter for IO, specifically due to
@@ -1983,6 +2264,52 @@ type ioLoadListenerState struct {
// completion is not an indicator that the "resource usage" has ceased -- it
// just means that the write has been applied to the WAL. Most of the work is
// in flushing to sstables and the following compactions, which happens later.
+//
+//
+// Token units are in bytes and used to protect a number of virtual or
+// physical resource bottlenecks:
+// - Compactions out of L0: compactions out of L0 can fall behind and cause
+// too many sub-levels or files in L0.
+// - Flushes into L0: flushes of memtables to L0 can fall behind and cause
+// write stalls due to too many memtables.
+// - Disk bandwidth: there is typically an aggregate read+write provisioned
+// bandwidth, and if it is fully utilized, IO operations can start queueing
+// and encounter high latency.
+//
+// For simplicity, after ioLoadListener computes the tokens due to compaction
+// or flush bottlenecks, it computes the minimum and passes that value to
+// granterWithIOTokens.setAvailableIOTokensLocked. That is, instead of working
+// with multiple token dimensions, these two token dimensions get collapsed
+// into one for enforcement purposes. This also helps simplify the integration
+// with WorkQueue which is dealing with a single dimension. The consumption of
+// these tokens is based on how many bytes an admitted work adds to L0.
+//
+// The disk bandwidth constraint is used to compute a token count for elastic
+// work (see disk_bandwidth.go for the reasons why this is limited to elastic
+// work). Additionally, these tokens are meant be consumed for all incoming
+// bytes into the LSM, and not just those written to L0 e.g. ingested bytes
+// into L3 should also consume such tokens. Note that we call these disk
+// bandwidth tokens, but that is a misnomer -- these are tokens for incoming
+// bytes into the LSM, motivated by disk bandwidth as a bottleneck resource,
+// and not consumed for every write to the disk (e.g. by compactions). Since
+// these tokens are consumed by all incoming bytes into the LSM, and not just
+// those into L0, it suggests explicitly modeling this as a separate
+// dimension. However, since modeling as a separate dimension everywhere would
+// lead to code complexity, we adopt the following compromise:
+//
+// - Like the other token dimensions, ioLoadListener computes a different
+// elastic token count (using diskBandwidthLimiter), and a different model
+// for consumption (via
+// storePerWorkTokenEstimator.atDoneDiskBWTokensLinearModel).
+//
+// - granterWithIOTokens, implemented by kvStoreTokenGranter, which enforces
+// the token count, also treats this as a separate dimension.
+//
+// - WorkQueue works with a single dimension, so the tokens consumed at
+// admission time are based on L0-bytes estimate. However, when
+// StoreWorkQueue informs kvStoreTokenGranter of work completion (by calling
+// storeWriteDone), the tokens are adjusted differently for the
+// flush/compaction L0 tokens and for the "disk bandwidth" tokens.
type ioLoadListener struct {
storeID int32
settings *cluster.Settings
@@ -1998,6 +2325,7 @@ type ioLoadListener struct {
statsInitialized bool
adjustTokensResult
perWorkTokenEstimator storePerWorkTokenEstimator
+ diskBandwidthLimiter diskBandwidthLimiter
}
const unlimitedTokens = math.MaxInt64
@@ -2056,21 +2384,32 @@ const adjustmentInterval = 15
const ticksInAdjustmentInterval = 60
const ioTokenTickDuration = 250 * time.Millisecond
+func cumLSMWriteAndIngestedBytes(m *pebble.Metrics) uint64 {
+ var b uint64
+ for i := range m.Levels {
+ b += m.Levels[i].BytesIngested + m.Levels[i].BytesFlushed
+ }
+ return b
+}
+
// pebbleMetricsTicks is called every adjustmentInterval seconds, and decides
// the token allocations until the next call.
func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMetrics) {
+ ctx = logtags.AddTag(ctx, "s", io.storeID)
m := metrics.Metrics
if !io.statsInitialized {
io.statsInitialized = true
sas := io.kvRequester.getStoreAdmissionStats()
- io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], sas)
+ cumLSMIncomingBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics)
+ io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIncomingBytes, sas)
io.adjustTokensResult = adjustTokensResult{
ioLoadListenerState: ioLoadListenerState{
cumL0AddedBytes: m.Levels[0].BytesFlushed + m.Levels[0].BytesIngested,
curL0Bytes: m.Levels[0].Size,
cumWriteStallCount: metrics.WriteStallCount,
// No initial limit, i.e, the first interval is unlimited.
- totalNumByteTokens: unlimitedTokens,
+ totalNumByteTokens: unlimitedTokens,
+ elasticDiskBWTokens: unlimitedTokens,
},
aux: adjustTokensAuxComputations{},
ioThreshold: &admissionpb.IOThreshold{
@@ -2080,6 +2419,9 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe
L0NumFilesThreshold: math.MaxInt64,
},
}
+ io.diskBW.bytesRead = metrics.DiskStats.BytesRead
+ io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten
+ io.diskBW.incomingLSMBytes = cumLSMIncomingBytes
io.copyAuxEtcFromPerWorkEstimator()
return
}
@@ -2087,32 +2429,57 @@ func (io *ioLoadListener) pebbleMetricsTick(ctx context.Context, metrics StoreMe
}
// allocateTokensTick gives out 1/ticksInAdjustmentInterval of the
-// totalNumByteTokens every 250ms.
+// various okens every 250ms.
func (io *ioLoadListener) allocateTokensTick() {
- var toAllocate int64
- // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up
- // calculation.
- if io.totalNumByteTokens >= unlimitedTokens-(ticksInAdjustmentInterval-1) {
- toAllocate = io.totalNumByteTokens / ticksInAdjustmentInterval
- } else {
- // Round up so that we don't accumulate tokens to give in a burst on the
- // last tick.
- toAllocate = (io.totalNumByteTokens + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval
- if toAllocate < 0 {
- panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate))
- }
- if toAllocate+io.tokensAllocated > io.totalNumByteTokens {
- toAllocate = io.totalNumByteTokens - io.tokensAllocated
+ allocateFunc := func(total int64, allocated int64) (toAllocate int64) {
+ // unlimitedTokens==MaxInt64, so avoid overflow in the rounding up
+ // calculation.
+ if total >= unlimitedTokens-(ticksInAdjustmentInterval-1) {
+ toAllocate = total / ticksInAdjustmentInterval
+ } else {
+ // Round up so that we don't accumulate tokens to give in a burst on the
+ // last tick.
+ toAllocate = (total + ticksInAdjustmentInterval - 1) / ticksInAdjustmentInterval
+ if toAllocate < 0 {
+ panic(errors.AssertionFailedf("toAllocate is negative %d", toAllocate))
+ }
+ if toAllocate+allocated > total {
+ toAllocate = total - allocated
+ }
}
+ return toAllocate
+ }
+ // INVARIANT: toAllocate* >= 0.
+ toAllocateByteTokens := allocateFunc(io.totalNumByteTokens, io.byteTokensAllocated)
+ if toAllocateByteTokens < 0 {
+ panic(errors.AssertionFailedf("toAllocateByteTokens is negative %d", toAllocateByteTokens))
+ }
+ toAllocateElasticDiskBWTokens :=
+ allocateFunc(io.elasticDiskBWTokens, io.elasticDiskBWTokensAllocated)
+ if toAllocateElasticDiskBWTokens < 0 {
+ panic(errors.AssertionFailedf("toAllocateElasticDiskBWTokens is negative %d",
+ toAllocateElasticDiskBWTokens))
}
// INVARIANT: toAllocate >= 0.
io.mu.Lock()
defer io.mu.Unlock()
- io.tokensAllocated += toAllocate
- if io.tokensAllocated < 0 {
- panic(errors.AssertionFailedf("tokens allocated is negative %d", io.tokensAllocated))
+ io.byteTokensAllocated += toAllocateByteTokens
+ if io.byteTokensAllocated < 0 {
+ panic(errors.AssertionFailedf("tokens allocated is negative %d", io.byteTokensAllocated))
+ }
+ io.byteTokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocateByteTokens)
+ io.elasticDiskBWTokensAllocated += toAllocateElasticDiskBWTokens
+ io.mu.kvGranter.setAvailableElasticDiskBandwidthTokensLocked(toAllocateElasticDiskBWTokens)
+}
+
+func computeIntervalDiskLoadInfo(
+ prevCumBytesRead uint64, prevCumBytesWritten uint64, diskStats DiskStats,
+) intervalDiskLoadInfo {
+ return intervalDiskLoadInfo{
+ readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval),
+ writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval),
+ provisionedBandwidth: diskStats.ProvisionedBandwidth,
}
- io.tokensUsed += io.mu.kvGranter.setAvailableIOTokensLocked(toAllocate)
}
// adjustTokens computes a new value of totalNumByteTokens (and resets
@@ -2133,17 +2500,43 @@ func (io *ioLoadListener) adjustTokens(ctx context.Context, metrics StoreMetrics
MinFlushUtilizationFraction.Get(&io.settings.SV),
)
io.adjustTokensResult = res
- io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], sas)
+ cumLSMIncomingBytes := cumLSMWriteAndIngestedBytes(metrics.Metrics)
+ {
+ // Disk Bandwidth tokens.
+ io.aux.diskBW.intervalDiskLoadInfo = computeIntervalDiskLoadInfo(
+ io.diskBW.bytesRead, io.diskBW.bytesWritten, metrics.DiskStats)
+ io.mu.Lock()
+ diskTokensUsed := io.mu.kvGranter.getDiskTokensUsedAndResetLocked()
+ io.mu.Unlock()
+ io.aux.diskBW.intervalLSMInfo = intervalLSMInfo{
+ incomingBytes: int64(cumLSMIncomingBytes) - int64(io.diskBW.incomingLSMBytes),
+ regularTokensUsed: diskTokensUsed[regularWorkClass],
+ elasticTokensUsed: diskTokensUsed[elasticWorkClass],
+ }
+ if metrics.DiskStats.ProvisionedBandwidth > 0 {
+ io.elasticDiskBWTokens = io.diskBandwidthLimiter.computeElasticTokens(ctx,
+ io.aux.diskBW.intervalDiskLoadInfo, io.aux.diskBW.intervalLSMInfo)
+ io.elasticDiskBWTokensAllocated = 0
+ }
+ if metrics.DiskStats.ProvisionedBandwidth == 0 ||
+ !DiskBandwidthTokensForElasticEnabled.Get(&io.settings.SV) {
+ io.elasticDiskBWTokens = unlimitedTokens
+ }
+ io.diskBW.bytesRead = metrics.DiskStats.BytesRead
+ io.diskBW.bytesWritten = metrics.DiskStats.BytesWritten
+ io.diskBW.incomingLSMBytes = cumLSMIncomingBytes
+ }
+ io.perWorkTokenEstimator.updateEstimates(metrics.Levels[0], cumLSMIncomingBytes, sas)
io.copyAuxEtcFromPerWorkEstimator()
requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission()
io.kvRequester.setStoreRequestEstimates(requestEstimates)
- writeLM, ingestedLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone()
+ writeLM, ingestedLM, diskBWLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone()
io.mu.Lock()
- io.mu.kvGranter.setAdmittedDoneModelsLocked(writeLM, ingestedLM)
+ io.mu.kvGranter.setAdmittedDoneModelsLocked(writeLM, ingestedLM, diskBWLM)
io.mu.Unlock()
- if _, overloaded := res.ioThreshold.Score(); overloaded || res.aux.doLogFlush {
- log.Infof(logtags.AddTag(ctx, "s", io.storeID), "IO overload: %s",
- io.adjustTokensResult)
+ if _, overloaded := io.ioThreshold.Score(); overloaded || io.aux.doLogFlush ||
+ io.elasticDiskBWTokens != unlimitedTokens {
+ log.Infof(ctx, "IO overload: %s", io.adjustTokensResult)
}
}
@@ -2156,9 +2549,10 @@ func (io *ioLoadListener) copyAuxEtcFromPerWorkEstimator() {
io.adjustTokensResult.aux.perWorkTokensAux = io.perWorkTokenEstimator.aux
requestEstimates := io.perWorkTokenEstimator.getStoreRequestEstimatesAtAdmission()
io.adjustTokensResult.requestEstimates = requestEstimates
- writeLM, ingestedLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone()
+ writeLM, ingestedLM, diskBWLM := io.perWorkTokenEstimator.getModelsAtAdmittedDone()
io.adjustTokensResult.writeLM = writeLM
io.adjustTokensResult.ingestedLM = ingestedLM
+ io.adjustTokensResult.diskBWLM = diskBWLM
}
type tokenKind int8
@@ -2183,8 +2577,15 @@ type adjustTokensAuxComputations struct {
perWorkTokensAux perWorkTokensAux
doLogFlush bool
+
+ diskBW struct {
+ intervalDiskLoadInfo intervalDiskLoadInfo
+ intervalLSMInfo intervalLSMInfo
+ }
}
+// adjustTokensInner is used for computing tokens based on compaction and
+// flush bottlenecks.
func (*ioLoadListener) adjustTokensInner(
ctx context.Context,
prev ioLoadListenerState,
@@ -2360,7 +2761,7 @@ func (*ioLoadListener) adjustTokensInner(
// Have we used, over the last (15s) cycle, more than 90% of the tokens we
// would give out for the next cycle? If yes, highTokenUsage is true.
highTokenUsage :=
- float64(prev.tokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction
+ float64(prev.byteTokensUsed) >= 0.9*smoothedNumFlushTokens*flushUtilTargetFraction
if intWriteStalls > 0 {
// Try decrease since there were write-stalls.
numDecreaseSteps := 1
@@ -2456,8 +2857,8 @@ func (*ioLoadListener) adjustTokensInner(
smoothedNumFlushTokens: smoothedNumFlushTokens,
flushUtilTargetFraction: flushUtilTargetFraction,
totalNumByteTokens: totalNumByteTokens,
- tokensAllocated: 0,
- tokensUsed: 0,
+ byteTokensAllocated: 0,
+ byteTokensUsed: 0,
},
aux: adjustTokensAuxComputations{
intL0AddedBytes: intL0AddedBytes,
@@ -2465,7 +2866,7 @@ func (*ioLoadListener) adjustTokensInner(
intFlushTokens: intFlushTokens,
intFlushUtilization: intFlushUtilization,
intWriteStalls: intWriteStalls,
- prevTokensUsed: prev.tokensUsed,
+ prevTokensUsed: prev.byteTokensUsed,
tokenKind: tokenKind,
doLogFlush: doLogFlush,
},
@@ -2479,6 +2880,7 @@ type adjustTokensResult struct {
requestEstimates storeRequestEstimates
writeLM tokensLinearModel
ingestedLM tokensLinearModel
+ diskBWLM tokensLinearModel
aux adjustTokensAuxComputations
ioThreshold *admissionpb.IOThreshold // never nil
}
@@ -2508,8 +2910,8 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) {
ib(res.aux.perWorkTokensAux.intL0WriteBypassedAccountedBytes))
// Ingestion bytes that we expected because requests told admission control.
p.Printf("%s acc-ingest (%s bypassed) + ",
- ib(res.aux.perWorkTokensAux.intL0IngestedAccountedBytes),
- ib(res.aux.perWorkTokensAux.intL0IngestedBypassedAccountedBytes))
+ ib(res.aux.perWorkTokensAux.intIngestedAccountedBytes),
+ ib(res.aux.perWorkTokensAux.intIngestedBypassedAccountedBytes))
// The models we are fitting to compute tokens based on the reported size of
// the write and ingest.
p.Printf("write-model %.2fx+%s (smoothed %.2fx+%s) + ",
@@ -2542,6 +2944,16 @@ func (res adjustTokensResult) SafeFormat(p redact.SafePrinter, _ rune) {
} else {
p.SafeString("all")
}
+ if res.elasticDiskBWTokens != unlimitedTokens {
+ p.Printf("; elastic tokens %s (used %s, regular used %s): work model %.2fx+%s, ",
+ ib(res.elasticDiskBWTokens), ib(res.aux.diskBW.intervalLSMInfo.elasticTokensUsed),
+ ib(res.aux.diskBW.intervalLSMInfo.regularTokensUsed),
+ res.diskBWLM.multiplier, ib(res.diskBWLM.constant))
+ p.Printf("disk bw read %s write %s provisioned %s",
+ ib(res.aux.diskBW.intervalDiskLoadInfo.readBandwidth),
+ ib(res.aux.diskBW.intervalDiskLoadInfo.writeBandwidth),
+ ib(res.aux.diskBW.intervalDiskLoadInfo.provisionedBandwidth))
+ }
}
func (res adjustTokensResult) String() string {
@@ -2594,6 +3006,15 @@ var (
}
)
+// TODO(irfansharif): we are lacking metrics for IO tokens and load, including
+// metrics from helper classes used by ioLoadListener, like the code in
+// disk_bandwidth.go and store_token_estimation.go. Additionally, what we have
+// below is per node, while we want such metrics per store. We should add
+// these metrics via StoreGrantCoordinators.SetPebbleMetricsProvider, which is
+// used to construct the per-store GrantCoordinator. These metrics should be
+// embedded in kvserver.StoreMetrics. We should also separate the metrics
+// related to cpu slots from the IO metrics.
+
// GranterMetrics are metrics associated with a GrantCoordinator.
type GranterMetrics struct {
KVTotalSlots *metric.Gauge
diff --git a/pkg/util/admission/granter_test.go b/pkg/util/admission/granter_test.go
index 146beef9ea61..88a6b075c2d8 100644
--- a/pkg/util/admission/granter_test.go
+++ b/pkg/util/admission/granter_test.go
@@ -37,10 +37,11 @@ import (
)
type testRequester struct {
- workKind WorkKind
- granter granter
- usesTokens bool
- buf *strings.Builder
+ workKind WorkKind
+ additionalID string
+ granter granter
+ usesTokens bool
+ buf *strings.Builder
waitingRequests bool
returnValueFromGranted int64
@@ -48,14 +49,14 @@ type testRequester struct {
}
var _ requester = &testRequester{}
-var _ storeRequester = &testRequester{}
func (tr *testRequester) hasWaitingRequests() bool {
return tr.waitingRequests
}
func (tr *testRequester) granted(grantChainID grantChainID) int64 {
- fmt.Fprintf(tr.buf, "%s: granted in chain %d, and returning %d\n", workKindString(tr.workKind),
+ fmt.Fprintf(tr.buf, "%s%s: granted in chain %d, and returning %d\n",
+ workKindString(tr.workKind), tr.additionalID,
grantChainID, tr.returnValueFromGranted)
tr.grantChainID = grantChainID
return tr.returnValueFromGranted
@@ -65,30 +66,50 @@ func (tr *testRequester) close() {}
func (tr *testRequester) tryGet(count int64) {
rv := tr.granter.tryGet(count)
- fmt.Fprintf(tr.buf, "%s: tryGet(%d) returned %t\n", workKindString(tr.workKind), count, rv)
+ fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", workKindString(tr.workKind),
+ tr.additionalID, count, rv)
}
func (tr *testRequester) returnGrant(count int64) {
- fmt.Fprintf(tr.buf, "%s: returnGrant(%d)\n", workKindString(tr.workKind), count)
+ fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", workKindString(tr.workKind), tr.additionalID,
+ count)
tr.granter.returnGrant(count)
}
func (tr *testRequester) tookWithoutPermission(count int64) {
- fmt.Fprintf(tr.buf, "%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind), count)
+ fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind),
+ tr.additionalID, count)
tr.granter.tookWithoutPermission(count)
}
func (tr *testRequester) continueGrantChain() {
- fmt.Fprintf(tr.buf, "%s: continueGrantChain\n", workKindString(tr.workKind))
+ fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", workKindString(tr.workKind),
+ tr.additionalID)
tr.granter.continueGrantChain(tr.grantChainID)
}
-func (tr *testRequester) getStoreAdmissionStats() storeAdmissionStats {
+type storeTestRequester struct {
+ requesters [numWorkClasses]*testRequester
+}
+
+var _ storeRequester = &storeTestRequester{}
+
+func (str *storeTestRequester) getRequesters() [numWorkClasses]requester {
+ var rv [numWorkClasses]requester
+ for i := range str.requesters {
+ rv[i] = str.requesters[i]
+ }
+ return rv
+}
+
+func (str *storeTestRequester) close() {}
+
+func (str *storeTestRequester) getStoreAdmissionStats() storeAdmissionStats {
// Only used by ioLoadListener, so don't bother.
return storeAdmissionStats{}
}
-func (tr *testRequester) setStoreRequestEstimates(estimates storeRequestEstimates) {
+func (str *storeTestRequester) setStoreRequestEstimates(estimates storeRequestEstimates) {
// Only used by ioLoadListener, so don't bother.
}
@@ -119,7 +140,9 @@ func TestGranterBasic(t *testing.T) {
defer log.Scope(t).Close(t)
var ambientCtx log.AmbientContext
- var requesters [numWorkKinds]*testRequester
+ // requesters[numWorkKinds] is used for kv elastic work, when working with a
+ // store grant coordinator.
+ var requesters [numWorkKinds + 1]*testRequester
var coord *GrantCoordinator
var ssg *SoftSlotGranter
clearRequesterAndCoord := func() {
@@ -188,16 +211,29 @@ func TestGranterBasic(t *testing.T) {
storeCoordinators := &StoreGrantCoordinators{
settings: settings,
makeStoreRequesterFunc: func(
- ambientCtx log.AmbientContext, granter granterWithStoreWriteDone,
+ ambientCtx log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings, opts workQueueOptions) storeRequester {
- req := &testRequester{
- workKind: KVWork,
- granter: granter,
- usesTokens: true,
- buf: &buf,
- returnValueFromGranted: 0,
+ makeTestRequester := func(wc workClass) *testRequester {
+ req := &testRequester{
+ workKind: KVWork,
+ granter: granters[wc],
+ usesTokens: true,
+ buf: &buf,
+ returnValueFromGranted: 0,
+ }
+ switch wc {
+ case regularWorkClass:
+ req.additionalID = "-regular"
+ case elasticWorkClass:
+ req.additionalID = "-elastic"
+ }
+ return req
}
- requesters[KVWork] = req
+ req := &storeTestRequester{}
+ req.requesters[regularWorkClass] = makeTestRequester(regularWorkClass)
+ req.requesters[elasticWorkClass] = makeTestRequester(elasticWorkClass)
+ requesters[KVWork] = req.requesters[regularWorkClass]
+ requesters[numWorkKinds] = req.requesters[elasticWorkClass]
return req
},
kvIOTokensExhaustedDuration: metrics.KVIOTokensExhaustedDuration,
@@ -210,6 +246,12 @@ func TestGranterBasic(t *testing.T) {
unsafeGranter, ok := storeCoordinators.gcMap.Load(int64(1))
require.True(t, ok)
coord = (*GrantCoordinator)(unsafeGranter)
+ kvStoreGranter := coord.granters[KVWork].(*kvStoreTokenGranter)
+ // Use the same model for all 3 kinds of models.
+ tlm := tokensLinearModel{multiplier: 0.5, constant: 50}
+ coord.mu.Lock()
+ kvStoreGranter.setAdmittedDoneModelsLocked(tlm, tlm, tlm)
+ coord.mu.Unlock()
return flushAndReset()
case "set-has-waiting-requests":
@@ -285,6 +327,27 @@ func TestGranterBasic(t *testing.T) {
coord.testingTryGrant()
return flushAndReset()
+ case "set-elastic-disk-bw-tokens":
+ var tokens int
+ d.ScanArgs(t, "tokens", &tokens)
+ // We are not using a real ioLoadListener, and simply setting the
+ // tokens (the ioLoadListener has its own test).
+ coord.mu.Lock()
+ coord.granters[KVWork].(*kvStoreTokenGranter).setAvailableElasticDiskBandwidthTokensLocked(
+ int64(tokens))
+ coord.mu.Unlock()
+ coord.testingTryGrant()
+ return flushAndReset()
+
+ case "store-write-done":
+ var origTokens, writeBytes int
+ d.ScanArgs(t, "orig-tokens", &origTokens)
+ d.ScanArgs(t, "write-bytes", &writeBytes)
+ requesters[scanWorkKind(t, d)].granter.(granterWithStoreWriteDone).storeWriteDone(
+ int64(origTokens), StoreWorkDoneInfo{WriteBytes: int64(writeBytes)})
+ coord.testingTryGrant()
+ return flushAndReset()
+
case "try-get-soft-slots":
var slots int
d.ScanArgs(t, "slots", &slots)
@@ -304,20 +367,22 @@ func TestGranterBasic(t *testing.T) {
})
}
-func scanWorkKind(t *testing.T, d *datadriven.TestData) WorkKind {
+func scanWorkKind(t *testing.T, d *datadriven.TestData) int8 {
var kindStr string
d.ScanArgs(t, "work", &kindStr)
switch kindStr {
case "kv":
- return KVWork
+ return int8(KVWork)
case "sql-kv-response":
- return SQLKVResponseWork
+ return int8(SQLKVResponseWork)
case "sql-sql-response":
- return SQLSQLResponseWork
+ return int8(SQLSQLResponseWork)
case "sql-leaf-start":
- return SQLStatementLeafStartWork
+ return int8(SQLStatementLeafStartWork)
case "sql-root-start":
- return SQLStatementRootStartWork
+ return int8(SQLStatementRootStartWork)
+ case "kv-elastic":
+ return int8(numWorkKinds)
}
panic("unknown WorkKind")
}
@@ -375,10 +440,16 @@ func TestStoreCoordinators(t *testing.T) {
Settings: settings,
makeRequesterFunc: makeRequesterFunc,
makeStoreRequesterFunc: func(
- ctx log.AmbientContext, granter granterWithStoreWriteDone, settings *cluster.Settings,
- opts workQueueOptions) storeRequester {
- req := makeRequesterFunc(ctx, KVWork, granter, settings, opts)
- return req.(*testRequester)
+ ctx log.AmbientContext, granters [numWorkClasses]granterWithStoreWriteDone,
+ settings *cluster.Settings, opts workQueueOptions) storeRequester {
+ reqReg := makeRequesterFunc(ctx, KVWork, granters[regularWorkClass], settings, opts)
+ reqElastic := makeRequesterFunc(ctx, KVWork, granters[elasticWorkClass], settings, opts)
+ str := &storeTestRequester{}
+ str.requesters[regularWorkClass] = reqReg.(*testRequester)
+ str.requesters[regularWorkClass].additionalID = "-regular"
+ str.requesters[elasticWorkClass] = reqElastic.(*testRequester)
+ str.requesters[elasticWorkClass].additionalID = "-elastic"
+ return str
},
}
coords, _ := NewGrantCoordinators(ambientCtx, opts)
@@ -392,8 +463,8 @@ func TestStoreCoordinators(t *testing.T) {
// Setting the metrics provider will cause the initialization of two
// GrantCoordinators for the two stores.
storeCoords.SetPebbleMetricsProvider(context.Background(), &mp, &mp)
- // Now we have 1+2 = 3 KVWork requesters.
- require.Equal(t, 3, len(requesters))
+ // Now we have 1+2*2 = 5 KVWork requesters.
+ require.Equal(t, 5, len(requesters))
// Confirm that the store IDs are as expected.
var actualStores []int32
@@ -415,7 +486,9 @@ func TestStoreCoordinators(t *testing.T) {
requesters[i].tryGet(1)
}
require.Equal(t,
- "kv: tryGet(1) returned false\nkv: tryGet(1) returned true\nkv: tryGet(1) returned true\n",
+ "kv: tryGet(1) returned false\n"+
+ "kv-regular: tryGet(1) returned true\nkv-elastic: tryGet(1) returned true\n"+
+ "kv-regular: tryGet(1) returned true\nkv-elastic: tryGet(1) returned true\n",
buf.String())
coords.Close()
}
@@ -427,16 +500,12 @@ type testRequesterForIOLL struct {
var _ storeRequester = &testRequesterForIOLL{}
-func (r *testRequesterForIOLL) hasWaitingRequests() bool {
- panic("unimplemented")
-}
+func (r *testRequesterForIOLL) close() {}
-func (r *testRequesterForIOLL) granted(grantChainID grantChainID) int64 {
+func (r *testRequesterForIOLL) getRequesters() [numWorkClasses]requester {
panic("unimplemented")
}
-func (r *testRequesterForIOLL) close() {}
-
func (r *testRequesterForIOLL) getStoreAdmissionStats() storeAdmissionStats {
return r.stats
}
@@ -446,8 +515,9 @@ func (r *testRequesterForIOLL) setStoreRequestEstimates(estimates storeRequestEs
}
type testGranterWithIOTokens struct {
- buf strings.Builder
- allTokensUsed bool
+ buf strings.Builder
+ allTokensUsed bool
+ diskBandwidthTokensUsed [numWorkClasses]int64
}
var _ granterWithIOTokens = &testGranterWithIOTokens{}
@@ -460,13 +530,24 @@ func (g *testGranterWithIOTokens) setAvailableIOTokensLocked(tokens int64) (toke
return 0
}
+func (g *testGranterWithIOTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) {
+ fmt.Fprintf(&g.buf, " setAvailableElasticDiskTokens: %s",
+ tokensForTokenTickDurationToString(tokens))
+}
+
+func (g *testGranterWithIOTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 {
+ return g.diskBandwidthTokensUsed
+}
+
func (g *testGranterWithIOTokens) setAdmittedDoneModelsLocked(
- writeLM tokensLinearModel, ingestedLM tokensLinearModel,
+ writeLM tokensLinearModel, ingestedLM tokensLinearModel, diskBWLM tokensLinearModel,
) {
fmt.Fprintf(&g.buf, "setAdmittedDoneModelsLocked: write-lm: ")
printLinearModel(&g.buf, writeLM)
fmt.Fprintf(&g.buf, " ingested-lm: ")
printLinearModel(&g.buf, ingestedLM)
+ fmt.Fprintf(&g.buf, " disk-bw-lm: ")
+ printLinearModel(&g.buf, diskBWLM)
fmt.Fprintf(&g.buf, "\n")
}
@@ -497,6 +578,7 @@ func TestIOLoadListener(t *testing.T) {
settings: st,
kvRequester: req,
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
+ diskBandwidthLimiter: makeDiskBandwidthLimiter(),
}
// The mutex is needed by ioLoadListener but is not useful in this
// test -- the channels provide synchronization and prevent this
@@ -568,6 +650,25 @@ func TestIOLoadListener(t *testing.T) {
d.ScanArgs(t, "all-tokens-used", &allTokensUsed)
}
kvGranter.allTokensUsed = allTokensUsed
+ var provisionedBandwidth, bytesRead, bytesWritten int
+ if d.HasArg("provisioned-bandwidth") {
+ d.ScanArgs(t, "provisioned-bandwidth", &provisionedBandwidth)
+ }
+ if d.HasArg("bytes-read") {
+ d.ScanArgs(t, "bytes-read", &bytesRead)
+ }
+ if d.HasArg("bytes-written") {
+ d.ScanArgs(t, "bytes-written", &bytesWritten)
+ }
+ if d.HasArg("disk-bw-tokens-used") {
+ var regularTokensUsed, elasticTokensUsed int
+ d.ScanArgs(t, "disk-bw-tokens-used", ®ularTokensUsed, &elasticTokensUsed)
+ kvGranter.diskBandwidthTokensUsed[regularWorkClass] = int64(regularTokensUsed)
+ kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = int64(elasticTokensUsed)
+ } else {
+ kvGranter.diskBandwidthTokensUsed[regularWorkClass] = 0
+ kvGranter.diskBandwidthTokensUsed[elasticWorkClass] = 0
+ }
var printOnlyFirstTick bool
if d.HasArg("print-only-first-tick") {
d.ScanArgs(t, "print-only-first-tick", &printOnlyFirstTick)
@@ -576,6 +677,11 @@ func TestIOLoadListener(t *testing.T) {
Metrics: &metrics,
WriteStallCount: int64(writeStallCount),
InternalIntervalMetrics: im,
+ DiskStats: DiskStats{
+ BytesRead: uint64(bytesRead),
+ BytesWritten: uint64(bytesWritten),
+ ProvisionedBandwidth: int64(provisionedBandwidth),
+ },
})
var buf strings.Builder
// Do the ticks until just before next adjustment.
@@ -617,7 +723,7 @@ func TestIOLoadListenerOverflow(t *testing.T) {
for i := int64(0); i < adjustmentInterval; i++ {
// Override the totalNumByteTokens manually to trigger the overflow bug.
ioll.totalNumByteTokens = math.MaxInt64 - i
- ioll.tokensAllocated = 0
+ ioll.byteTokensAllocated = 0
for j := 0; j < ticksInAdjustmentInterval; j++ {
ioll.allocateTokensTick()
}
@@ -639,18 +745,30 @@ type testGranterNonNegativeTokens struct {
t *testing.T
}
+var _ granterWithIOTokens = &testGranterNonNegativeTokens{}
+
func (g *testGranterNonNegativeTokens) setAvailableIOTokensLocked(tokens int64) (tokensUsed int64) {
require.LessOrEqual(g.t, int64(0), tokens)
return 0
}
+func (g *testGranterNonNegativeTokens) setAvailableElasticDiskBandwidthTokensLocked(tokens int64) {
+ require.LessOrEqual(g.t, int64(0), tokens)
+}
+
+func (g *testGranterNonNegativeTokens) getDiskTokensUsedAndResetLocked() [numWorkClasses]int64 {
+ return [numWorkClasses]int64{}
+}
+
func (g *testGranterNonNegativeTokens) setAdmittedDoneModelsLocked(
- writeLM tokensLinearModel, ingestedLM tokensLinearModel,
+ writeLM tokensLinearModel, ingestedLM tokensLinearModel, diskBWLM tokensLinearModel,
) {
require.LessOrEqual(g.t, 0.5, writeLM.multiplier)
require.LessOrEqual(g.t, int64(0), writeLM.constant)
require.Less(g.t, 0.0, ingestedLM.multiplier)
require.LessOrEqual(g.t, int64(0), ingestedLM.constant)
+ require.LessOrEqual(g.t, 0.5, diskBWLM.multiplier)
+ require.LessOrEqual(g.t, int64(0), diskBWLM.constant)
}
// TODO(sumeer): we now do more work outside adjustTokensInner, so the parts
@@ -706,6 +824,7 @@ func TestAdjustTokensInnerAndLogging(t *testing.T) {
// stats and negative values) don't cause panics or tokens to be negative.
func TestBadIOLoadListenerStats(t *testing.T) {
var m pebble.Metrics
+ var d DiskStats
req := &testRequesterForIOLL{}
ctx := context.Background()
@@ -715,10 +834,17 @@ func TestBadIOLoadListenerStats(t *testing.T) {
m.Levels[0].NumFiles = int64(rand.Uint64())
m.Levels[0].Size = int64(rand.Uint64())
m.Levels[0].BytesFlushed = rand.Uint64()
- m.Levels[0].BytesIngested = rand.Uint64()
+ for i := range m.Levels {
+ m.Levels[i].BytesIngested = rand.Uint64()
+ }
+ d.BytesRead = rand.Uint64()
+ d.BytesWritten = rand.Uint64()
+ d.ProvisionedBandwidth = 1 << 20
req.stats.admittedCount = rand.Uint64()
req.stats.writeAccountedBytes = rand.Uint64()
req.stats.ingestedAccountedBytes = rand.Uint64()
+ req.stats.statsToIgnore.Bytes = rand.Uint64()
+ req.stats.statsToIgnore.ApproxIngestedIntoL0Bytes = rand.Uint64()
}
kvGranter := &testGranterNonNegativeTokens{t: t}
st := cluster.MakeTestingClusterSettings()
@@ -726,6 +852,7 @@ func TestBadIOLoadListenerStats(t *testing.T) {
settings: st,
kvRequester: req,
perWorkTokenEstimator: makeStorePerWorkTokenEstimator(),
+ diskBandwidthLimiter: makeDiskBandwidthLimiter(),
}
ioll.mu.Mutex = &syncutil.Mutex{}
ioll.mu.kvGranter = kvGranter
@@ -734,6 +861,7 @@ func TestBadIOLoadListenerStats(t *testing.T) {
ioll.pebbleMetricsTick(ctx, StoreMetrics{
Metrics: &m,
InternalIntervalMetrics: &pebble.InternalIntervalMetrics{},
+ DiskStats: d,
})
for j := 0; j < ticksInAdjustmentInterval; j++ {
ioll.allocateTokensTick()
@@ -742,7 +870,9 @@ func TestBadIOLoadListenerStats(t *testing.T) {
require.LessOrEqual(t, float64(0), ioll.smoothedNumFlushTokens)
require.LessOrEqual(t, float64(0), ioll.flushUtilTargetFraction)
require.LessOrEqual(t, int64(0), ioll.totalNumByteTokens)
- require.LessOrEqual(t, int64(0), ioll.tokensAllocated)
+ require.LessOrEqual(t, int64(0), ioll.byteTokensAllocated)
+ require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokens)
+ require.LessOrEqual(t, int64(0), ioll.elasticDiskBWTokensAllocated)
}
}
}
diff --git a/pkg/util/admission/store_token_estimation.go b/pkg/util/admission/store_token_estimation.go
index 41d92dce3f64..115ca48d9e63 100644
--- a/pkg/util/admission/store_token_estimation.go
+++ b/pkg/util/admission/store_token_estimation.go
@@ -252,10 +252,17 @@ type storePerWorkTokenEstimator struct {
atAdmissionWorkTokens int64
atDoneWriteTokensLinearModel tokensLinearModelFitter
atDoneIngestTokensLinearModel tokensLinearModelFitter
+ // Unlike the models above that model bytes into L0, for disk bandwidth
+ // tokens we need to look at all the incoming bytes into the LSM. There is
+ // of course a multiplier over these incoming bytes, in the actual disk
+ // bandwidth consumed -- that is the concern of the code in
+ // disk_bandwidth.go and not the code in this file.
+ atDoneDiskBWTokensLinearModel tokensLinearModelFitter
- cumStoreAdmissionStats storeAdmissionStats
- cumL0WriteBytes uint64
- cumL0IngestedBytes uint64
+ cumStoreAdmissionStats storeAdmissionStats
+ cumL0WriteBytes uint64
+ cumL0IngestedBytes uint64
+ cumLSMWriteAndIngestedBytes uint64
// Tracked for logging and copied out of here.
aux perWorkTokensAux
@@ -267,19 +274,22 @@ type perWorkTokensAux struct {
intWorkCount int64
intL0WriteBytes int64
intL0IngestedBytes int64
+ intLSMWriteAndIngestedBytes int64
intL0WriteAccountedBytes int64
- intL0IngestedAccountedBytes int64
+ intIngestedAccountedBytes int64
intWriteLinearModel tokensLinearModel
intIngestedLinearModel tokensLinearModel
+ intDiskBWLinearModel tokensLinearModel
// The bypassed count and bytes are also included in the overall interval
// stats.
- intBypassedWorkCount int64
- intL0WriteBypassedAccountedBytes int64
- intL0IngestedBypassedAccountedBytes int64
+ intBypassedWorkCount int64
+ intL0WriteBypassedAccountedBytes int64
+ intIngestedBypassedAccountedBytes int64
- // The ignored bytes are included in intL0IngestedBytes, and may even be
- // higher than that value because these are from a different source.
+ // The ignored bytes are included in intL0IngestedBytes, and in
+ // intLSMWriteAndIngestedBytes, and may even be higher than that value
+ // because these are from a different source.
intL0IgnoredIngestedBytes int64
}
@@ -288,17 +298,21 @@ func makeStorePerWorkTokenEstimator() storePerWorkTokenEstimator {
atAdmissionWorkTokens: 1,
atDoneWriteTokensLinearModel: makeTokensLinearModelFitter(0.5, 3, false),
atDoneIngestTokensLinearModel: makeTokensLinearModelFitter(0.001, 1.5, true),
+ atDoneDiskBWTokensLinearModel: makeTokensLinearModelFitter(0.5, 3, false),
}
}
// NB: first call to updateEstimates only initializes the cumulative values.
func (e *storePerWorkTokenEstimator) updateEstimates(
- l0Metrics pebble.LevelMetrics, admissionStats storeAdmissionStats,
+ l0Metrics pebble.LevelMetrics,
+ cumLSMWriteAndIngestedBytes uint64,
+ admissionStats storeAdmissionStats,
) {
if e.cumL0WriteBytes == 0 {
e.cumStoreAdmissionStats = admissionStats
e.cumL0WriteBytes = l0Metrics.BytesFlushed
e.cumL0IngestedBytes = l0Metrics.BytesIngested
+ e.cumLSMWriteAndIngestedBytes = cumLSMWriteAndIngestedBytes
return
}
intL0WriteBytes := int64(l0Metrics.BytesFlushed) - int64(e.cumL0WriteBytes)
@@ -313,14 +327,25 @@ func (e *storePerWorkTokenEstimator) updateEstimates(
int64(e.cumStoreAdmissionStats.admittedCount)
intL0WriteAccountedBytes :=
int64(admissionStats.writeAccountedBytes) - int64(e.cumStoreAdmissionStats.writeAccountedBytes)
- // Note that these are not really L0 ingested bytes, since we don't know how
+ // Note that these are not L0 ingested bytes, since we don't know how
// many did go to L0.
- intL0IngestedAccountedBytes := int64(admissionStats.ingestedAccountedBytes) -
+ intIngestedAccountedBytes := int64(admissionStats.ingestedAccountedBytes) -
int64(e.cumStoreAdmissionStats.ingestedAccountedBytes)
e.atDoneWriteTokensLinearModel.updateModelUsingIntervalStats(
intL0WriteAccountedBytes, intL0WriteBytes, intWorkCount)
e.atDoneIngestTokensLinearModel.updateModelUsingIntervalStats(
- intL0IngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount)
+ intIngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount)
+ // Disk bandwidth model
+ intLSMWriteAndIngestedBytes := int64(cumLSMWriteAndIngestedBytes) -
+ int64(e.cumLSMWriteAndIngestedBytes)
+ adjustedIntLSMWriteAndIngestedBytes := intLSMWriteAndIngestedBytes -
+ (int64(admissionStats.statsToIgnore.Bytes) - int64(e.cumStoreAdmissionStats.statsToIgnore.Bytes))
+ if adjustedIntLSMWriteAndIngestedBytes < 0 {
+ adjustedIntLSMWriteAndIngestedBytes = 0
+ }
+ e.atDoneDiskBWTokensLinearModel.updateModelUsingIntervalStats(
+ intL0WriteAccountedBytes+intIngestedAccountedBytes, adjustedIntLSMWriteAndIngestedBytes,
+ intWorkCount)
intL0TotalBytes := intL0WriteBytes + adjustedIntL0IngestedBytes
if intWorkCount > 1 && intL0TotalBytes > 0 {
@@ -335,15 +360,17 @@ func (e *storePerWorkTokenEstimator) updateEstimates(
intWorkCount: intWorkCount,
intL0WriteBytes: intL0WriteBytes,
intL0IngestedBytes: intL0IngestedBytes,
+ intLSMWriteAndIngestedBytes: intLSMWriteAndIngestedBytes,
intL0WriteAccountedBytes: intL0WriteAccountedBytes,
- intL0IngestedAccountedBytes: intL0IngestedAccountedBytes,
+ intIngestedAccountedBytes: intIngestedAccountedBytes,
intWriteLinearModel: e.atDoneWriteTokensLinearModel.intLinearModel,
intIngestedLinearModel: e.atDoneIngestTokensLinearModel.intLinearModel,
+ intDiskBWLinearModel: e.atDoneDiskBWTokensLinearModel.intLinearModel,
intBypassedWorkCount: int64(admissionStats.aux.bypassedCount) -
int64(e.cumStoreAdmissionStats.aux.bypassedCount),
intL0WriteBypassedAccountedBytes: int64(admissionStats.aux.writeBypassedAccountedBytes) -
int64(e.cumStoreAdmissionStats.aux.writeBypassedAccountedBytes),
- intL0IngestedBypassedAccountedBytes: int64(admissionStats.aux.ingestedBypassedAccountedBytes) -
+ intIngestedBypassedAccountedBytes: int64(admissionStats.aux.ingestedBypassedAccountedBytes) -
int64(e.cumStoreAdmissionStats.aux.ingestedBypassedAccountedBytes),
intL0IgnoredIngestedBytes: intL0IgnoredIngestedBytes,
}
@@ -351,6 +378,7 @@ func (e *storePerWorkTokenEstimator) updateEstimates(
e.cumStoreAdmissionStats = admissionStats
e.cumL0WriteBytes = l0Metrics.BytesFlushed
e.cumL0IngestedBytes = l0Metrics.BytesIngested
+ e.cumLSMWriteAndIngestedBytes = cumLSMWriteAndIngestedBytes
}
func (e *storePerWorkTokenEstimator) getStoreRequestEstimatesAtAdmission() storeRequestEstimates {
@@ -360,7 +388,9 @@ func (e *storePerWorkTokenEstimator) getStoreRequestEstimatesAtAdmission() store
func (e *storePerWorkTokenEstimator) getModelsAtAdmittedDone() (
writeLM tokensLinearModel,
ingestedLM tokensLinearModel,
+ diskBWLM tokensLinearModel,
) {
return e.atDoneWriteTokensLinearModel.smoothedLinearModel,
- e.atDoneIngestTokensLinearModel.smoothedLinearModel
+ e.atDoneIngestTokensLinearModel.smoothedLinearModel,
+ e.atDoneDiskBWTokensLinearModel.smoothedLinearModel
}
diff --git a/pkg/util/admission/store_token_estimation_test.go b/pkg/util/admission/store_token_estimation_test.go
index 0115688a36f8..c8f163589535 100644
--- a/pkg/util/admission/store_token_estimation_test.go
+++ b/pkg/util/admission/store_token_estimation_test.go
@@ -89,6 +89,7 @@ func TestStorePerWorkTokenEstimator(t *testing.T) {
var estimator storePerWorkTokenEstimator
var l0Metrics pebble.LevelMetrics
var admissionStats storeAdmissionStats
+ var cumLSMWriteAndIngestedBytes uint64
datadriven.RunTest(t, testutils.TestDataPath(t, "store_per_work_token_estimator"),
func(t *testing.T, d *datadriven.TestData) string {
@@ -106,6 +107,12 @@ func TestStorePerWorkTokenEstimator(t *testing.T) {
d.ScanArgs(t, "ingested", &intIngested)
l0Metrics.BytesFlushed += intFlushed
l0Metrics.BytesIngested += intIngested
+ cumLSMWriteAndIngestedBytes += intFlushed + intIngested
+ if d.HasArg("other-levels-ingested") {
+ var otherLevelsIngested uint64
+ d.ScanArgs(t, "other-levels-ingested", &otherLevelsIngested)
+ cumLSMWriteAndIngestedBytes += otherLevelsIngested
+ }
var admitted, writeAccounted, ingestedAccounted uint64
d.ScanArgs(t, "admitted", &admitted)
d.ScanArgs(t, "write-accounted", &writeAccounted)
@@ -126,11 +133,13 @@ func TestStorePerWorkTokenEstimator(t *testing.T) {
var ignoreIngestedIntoL0 int
d.ScanArgs(t, "ignore-ingested-into-L0", &ignoreIngestedIntoL0)
admissionStats.statsToIgnore.ApproxIngestedIntoL0Bytes += uint64(ignoreIngestedIntoL0)
+ admissionStats.statsToIgnore.Bytes += uint64(ignoreIngestedIntoL0)
}
- estimator.updateEstimates(l0Metrics, admissionStats)
- wlm, ilm := estimator.getModelsAtAdmittedDone()
+ estimator.updateEstimates(l0Metrics, cumLSMWriteAndIngestedBytes, admissionStats)
+ wlm, ilm, dbwlm := estimator.getModelsAtAdmittedDone()
require.Equal(t, wlm, estimator.atDoneWriteTokensLinearModel.smoothedLinearModel)
require.Equal(t, ilm, estimator.atDoneIngestTokensLinearModel.smoothedLinearModel)
+ require.Equal(t, dbwlm, estimator.atDoneDiskBWTokensLinearModel.smoothedLinearModel)
var b strings.Builder
fmt.Fprintf(&b, "interval state: %+v\n", estimator.aux)
fmt.Fprintf(&b, "at-admission-tokens: %d\n",
@@ -139,6 +148,8 @@ func TestStorePerWorkTokenEstimator(t *testing.T) {
printLinearModelFitter(&b, estimator.atDoneWriteTokensLinearModel)
fmt.Fprintf(&b, "ingest-tokens: ")
printLinearModelFitter(&b, estimator.atDoneIngestTokensLinearModel)
+ fmt.Fprintf(&b, "disk-bw-tokens: ")
+ printLinearModelFitter(&b, estimator.atDoneDiskBWTokensLinearModel)
return b.String()
default:
diff --git a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt
index d3e334a9d2e0..50e2c2145f21 100644
--- a/pkg/util/admission/testdata/format_adjust_tokens_stats.txt
+++ b/pkg/util/admission/testdata/format_adjust_tokens_stats.txt
@@ -1,6 +1,6 @@
echo
----
zero:
-compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all
+compaction score 0.000 (0 ssts, 0 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all; elastic tokens 0 B (used 0 B, regular used 0 B): work model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B
real-numbers:
-compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) due to L0 growth (used 0 B)
+compaction score 2.700[L0-overload] (195 ssts, 27 sub-levels), L0 growth 577 MiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 0.00x+0 B) + ingested-model 0.00x+0 B (smoothed 0.00x+0 B) + at-admission-tokens 0 B, compacted 77 MiB [≈62 MiB], flushed 0 B [≈0 B]; admitting 116 MiB (rate 7.7 MiB/s) due to L0 growth (used 0 B); elastic tokens 0 B (used 0 B, regular used 0 B): work model 0.00x+0 B, disk bw read 0 B write 0 B provisioned 0 B
diff --git a/pkg/util/admission/testdata/granter b/pkg/util/admission/testdata/granter
index c37f76c0619f..193c9f875071 100644
--- a/pkg/util/admission/testdata/granter
+++ b/pkg/util/admission/testdata/granter
@@ -378,89 +378,199 @@ sql-sql-response: avail: -1 sql-leaf-start: used: 0, total: 2 sql-root-start: us
init-store-grant-coordinator
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 153722867280912930
+(chain: id: 0 active: false index: 5) io-avail: 153722867280912930, elastic-disk-bw-tokens-avail: 153722867280912930
# Initial tokens are effectively unlimited.
try-get work=kv v=10000
----
-kv: tryGet(10000) returned true
+kv-regular: tryGet(10000) returned true
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 153722867280902930
+(chain: id: 0 active: false index: 5) io-avail: 153722867280902930, elastic-disk-bw-tokens-avail: 153722867280912930
# Set the io tokens to a smaller value.
set-io-tokens tokens=500
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 500
+(chain: id: 0 active: false index: 5) io-avail: 500, elastic-disk-bw-tokens-avail: 153722867280912930
-# Subtract 100 tokens.
-took-without-permission work=kv v=100
+# Subtract 100 tokens for elastic work. Note that elastic-disk-bw-tokens-avail also decreases by 100.
+took-without-permission work=kv-elastic v=100
----
-kv: tookWithoutPermission(100)
+kv-elastic: tookWithoutPermission(100)
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 400
+(chain: id: 0 active: false index: 5) io-avail: 400, elastic-disk-bw-tokens-avail: 153722867280912830
# Add 200 tokens.
return-grant work=kv v=200
----
-kv: returnGrant(200)
+kv-regular: returnGrant(200)
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 600
+(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 153722867280912830
# Setup waiting requests that want 400 tokens each.
set-has-waiting-requests work=kv v=true
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 600
+(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 153722867280912830
set-return-value-from-granted work=kv v=400
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 600
+(chain: id: 0 active: false index: 5) io-avail: 600, elastic-disk-bw-tokens-avail: 153722867280912830
# Returning tokens triggers granting and 2 requests will be granted until the
# tokens become <= 0.
return-grant work=kv v=100
----
-kv: returnGrant(100)
-kv: granted in chain 0, and returning 400
-kv: granted in chain 0, and returning 400
+kv-regular: returnGrant(100)
+kv-regular: granted in chain 0, and returning 400
+kv-regular: granted in chain 0, and returning 400
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: -100
+(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 153722867280912830
set-return-value-from-granted work=kv v=100
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: -100
+(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 153722867280912830
# No tokens to give.
try-get work=kv
----
-kv: tryGet(1) returned false
+kv-regular: tryGet(1) returned false
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: -100
+(chain: id: 0 active: false index: 5) io-avail: -100, elastic-disk-bw-tokens-avail: 153722867280912830
# Increment by 50 tokens.
set-io-tokens tokens=50
----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: -50
+(chain: id: 0 active: false index: 5) io-avail: -50, elastic-disk-bw-tokens-avail: 153722867280912830
# Return another 50 tokens. Since the number of tokens is 0, there is no
# grant.
return-grant work=kv v=50
----
-kv: returnGrant(50)
+kv-regular: returnGrant(50)
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: 0
+(chain: id: 0 active: false index: 5) io-avail: 0, elastic-disk-bw-tokens-avail: 153722867280912830
# As soon as the tokens > 0, it will grant.
return-grant work=kv v=1
----
-kv: returnGrant(1)
-kv: granted in chain 0, and returning 100
+kv-regular: returnGrant(1)
+kv-regular: granted in chain 0, and returning 100
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 153722867280912830
+
+# Have waiting requests for kv-elastic too.
+set-has-waiting-requests work=kv-elastic v=true
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 153722867280912830
+
+# The kv-elastic waiting requests need 200 tokens each.
+set-return-value-from-granted work=kv-elastic v=200
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 153722867280912830
+
+# Since there are regular requests waiting, those are granted first.
+return-grant work=kv-elastic v=400
+----
+kv-elastic: returnGrant(400)
+kv-regular: granted in chain 0, and returning 100
+kv-regular: granted in chain 0, and returning 100
+kv-regular: granted in chain 0, and returning 100
+kv-regular: granted in chain 0, and returning 100
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 153722867280913230
+
+# No more regular requests waiting.
+set-has-waiting-requests work=kv v=false
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -99, elastic-disk-bw-tokens-avail: 153722867280913230
+
+# kv-elastic is granted.
+set-io-tokens tokens=100
+----
+kv-elastic: granted in chain 0, and returning 200
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: 153722867280913030
+
+set-elastic-disk-bw-tokens tokens=50
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: 50
+
+# Both kinds of tokens are decremented and become negative.
+set-io-tokens tokens=200
+----
+kv-elastic: granted in chain 0, and returning 200
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: -199, elastic-disk-bw-tokens-avail: -150
+
+# IO tokens become positive. But no grant to elastic work since
+# elastic-disk-bw tokens are negative.
+set-io-tokens tokens=300
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 101, elastic-disk-bw-tokens-avail: -150
+
+# Regular kv work can get tokens.
+try-get work=kv v=10
+----
+kv-regular: tryGet(10) returned true
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -150
+
+# Elastic kv work cannot get tokens.
+try-get work=kv-elastic v=10
+----
+kv-elastic: tryGet(10) returned false
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -150
+
+# Still negative.
+set-elastic-disk-bw-tokens tokens=50
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 91, elastic-disk-bw-tokens-avail: -100
+
+# Even more IO tokens.
+set-io-tokens tokens=400
+----
GrantCoordinator:
-(chain: id: 0 active: false index: 5) io-avail: -99
+(chain: id: 0 active: false index: 5) io-avail: 400, elastic-disk-bw-tokens-avail: -100
+
+# Finally both tokens are positive and we grant until the elastic-disk-bw
+# tokens become negative.
+set-elastic-disk-bw-tokens tokens=120
+----
+kv-elastic: granted in chain 0, and returning 200
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 200, elastic-disk-bw-tokens-avail: -180
+
+# Models are 0.5x+50, so 0.5*40+50=70. So 70-10=60 additional tokens are
+# needed for disk-bw. For io tokens we also apply the ingest model so
+# 0.5*0+50=50, which means 110 additional tokens are needed.
+store-write-done work=kv-elastic orig-tokens=10 write-bytes=40
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 90, elastic-disk-bw-tokens-avail: -240
+
+store-write-done work=kv orig-tokens=400 write-bytes=40
+----
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 370, elastic-disk-bw-tokens-avail: -240
+
+# Both tokens become positive, so one work is granted.
+store-write-done work=kv-elastic orig-tokens=400 write-bytes=40
+----
+kv-elastic: granted in chain 0, and returning 200
+GrantCoordinator:
+(chain: id: 0 active: false index: 5) io-avail: 450, elastic-disk-bw-tokens-avail: -110
+
#####################################################################
# Test soft slots
diff --git a/pkg/util/admission/testdata/io_load_listener b/pkg/util/admission/testdata/io_load_listener
index da00f07bc167..6be4a13131e1 100644
--- a/pkg/util/admission/testdata/io_load_listener
+++ b/pkg/util/admission/testdata/io_load_listener
@@ -12,67 +12,67 @@ prep-admission-stats admitted=0
set-state l0-bytes=10000 l0-added-write=1000 l0-files=21 l0-sublevels=21
----
compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
-tick: 0, setAvailableIOTokens: unlimited
-tick: 1, setAvailableIOTokens: unlimited
-tick: 2, setAvailableIOTokens: unlimited
-tick: 3, setAvailableIOTokens: unlimited
-tick: 4, setAvailableIOTokens: unlimited
-tick: 5, setAvailableIOTokens: unlimited
-tick: 6, setAvailableIOTokens: unlimited
-tick: 7, setAvailableIOTokens: unlimited
-tick: 8, setAvailableIOTokens: unlimited
-tick: 9, setAvailableIOTokens: unlimited
-tick: 10, setAvailableIOTokens: unlimited
-tick: 11, setAvailableIOTokens: unlimited
-tick: 12, setAvailableIOTokens: unlimited
-tick: 13, setAvailableIOTokens: unlimited
-tick: 14, setAvailableIOTokens: unlimited
-tick: 15, setAvailableIOTokens: unlimited
-tick: 16, setAvailableIOTokens: unlimited
-tick: 17, setAvailableIOTokens: unlimited
-tick: 18, setAvailableIOTokens: unlimited
-tick: 19, setAvailableIOTokens: unlimited
-tick: 20, setAvailableIOTokens: unlimited
-tick: 21, setAvailableIOTokens: unlimited
-tick: 22, setAvailableIOTokens: unlimited
-tick: 23, setAvailableIOTokens: unlimited
-tick: 24, setAvailableIOTokens: unlimited
-tick: 25, setAvailableIOTokens: unlimited
-tick: 26, setAvailableIOTokens: unlimited
-tick: 27, setAvailableIOTokens: unlimited
-tick: 28, setAvailableIOTokens: unlimited
-tick: 29, setAvailableIOTokens: unlimited
-tick: 30, setAvailableIOTokens: unlimited
-tick: 31, setAvailableIOTokens: unlimited
-tick: 32, setAvailableIOTokens: unlimited
-tick: 33, setAvailableIOTokens: unlimited
-tick: 34, setAvailableIOTokens: unlimited
-tick: 35, setAvailableIOTokens: unlimited
-tick: 36, setAvailableIOTokens: unlimited
-tick: 37, setAvailableIOTokens: unlimited
-tick: 38, setAvailableIOTokens: unlimited
-tick: 39, setAvailableIOTokens: unlimited
-tick: 40, setAvailableIOTokens: unlimited
-tick: 41, setAvailableIOTokens: unlimited
-tick: 42, setAvailableIOTokens: unlimited
-tick: 43, setAvailableIOTokens: unlimited
-tick: 44, setAvailableIOTokens: unlimited
-tick: 45, setAvailableIOTokens: unlimited
-tick: 46, setAvailableIOTokens: unlimited
-tick: 47, setAvailableIOTokens: unlimited
-tick: 48, setAvailableIOTokens: unlimited
-tick: 49, setAvailableIOTokens: unlimited
-tick: 50, setAvailableIOTokens: unlimited
-tick: 51, setAvailableIOTokens: unlimited
-tick: 52, setAvailableIOTokens: unlimited
-tick: 53, setAvailableIOTokens: unlimited
-tick: 54, setAvailableIOTokens: unlimited
-tick: 55, setAvailableIOTokens: unlimited
-tick: 56, setAvailableIOTokens: unlimited
-tick: 57, setAvailableIOTokens: unlimited
-tick: 58, setAvailableIOTokens: unlimited
-tick: 59, setAvailableIOTokens: unlimited
+{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
+tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 1, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 2, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 3, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 4, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 5, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 6, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 7, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 8, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 9, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 10, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 11, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 12, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 13, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 14, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 15, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 16, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 17, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 18, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 19, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 20, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 21, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 22, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 23, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 24, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 25, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 26, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 27, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 28, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 29, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 30, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 31, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 32, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 33, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 34, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 35, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 36, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 37, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 38, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 39, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 40, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 41, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 42, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 43, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 44, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 45, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 46, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 47, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 48, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 49, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 50, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 51, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 52, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 53, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 54, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 55, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 56, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 57, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 58, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+tick: 59, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
prep-admission-stats admitted=10000 write-bytes=40000
----
@@ -85,69 +85,69 @@ prep-admission-stats admitted=10000 write-bytes=40000
set-state l0-bytes=10000 l0-added-write=101000 l0-files=21 l0-sublevels=21
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.00x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 5 B, compacted 98 KiB [≈49 KiB], flushed 0 B [≈0 B]; admitting 12 KiB (rate 833 B/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:12500 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:5} writeLM:{multiplier:2 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:101000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:101000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:12500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:12500 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:5} writeLM:{multiplier:2 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:2 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:100000 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:2.25 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:101000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 5
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.00x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 209
-tick: 1, setAvailableIOTokens: 209
-tick: 2, setAvailableIOTokens: 209
-tick: 3, setAvailableIOTokens: 209
-tick: 4, setAvailableIOTokens: 209
-tick: 5, setAvailableIOTokens: 209
-tick: 6, setAvailableIOTokens: 209
-tick: 7, setAvailableIOTokens: 209
-tick: 8, setAvailableIOTokens: 209
-tick: 9, setAvailableIOTokens: 209
-tick: 10, setAvailableIOTokens: 209
-tick: 11, setAvailableIOTokens: 209
-tick: 12, setAvailableIOTokens: 209
-tick: 13, setAvailableIOTokens: 209
-tick: 14, setAvailableIOTokens: 209
-tick: 15, setAvailableIOTokens: 209
-tick: 16, setAvailableIOTokens: 209
-tick: 17, setAvailableIOTokens: 209
-tick: 18, setAvailableIOTokens: 209
-tick: 19, setAvailableIOTokens: 209
-tick: 20, setAvailableIOTokens: 209
-tick: 21, setAvailableIOTokens: 209
-tick: 22, setAvailableIOTokens: 209
-tick: 23, setAvailableIOTokens: 209
-tick: 24, setAvailableIOTokens: 209
-tick: 25, setAvailableIOTokens: 209
-tick: 26, setAvailableIOTokens: 209
-tick: 27, setAvailableIOTokens: 209
-tick: 28, setAvailableIOTokens: 209
-tick: 29, setAvailableIOTokens: 209
-tick: 30, setAvailableIOTokens: 209
-tick: 31, setAvailableIOTokens: 209
-tick: 32, setAvailableIOTokens: 209
-tick: 33, setAvailableIOTokens: 209
-tick: 34, setAvailableIOTokens: 209
-tick: 35, setAvailableIOTokens: 209
-tick: 36, setAvailableIOTokens: 209
-tick: 37, setAvailableIOTokens: 209
-tick: 38, setAvailableIOTokens: 209
-tick: 39, setAvailableIOTokens: 209
-tick: 40, setAvailableIOTokens: 209
-tick: 41, setAvailableIOTokens: 209
-tick: 42, setAvailableIOTokens: 209
-tick: 43, setAvailableIOTokens: 209
-tick: 44, setAvailableIOTokens: 209
-tick: 45, setAvailableIOTokens: 209
-tick: 46, setAvailableIOTokens: 209
-tick: 47, setAvailableIOTokens: 209
-tick: 48, setAvailableIOTokens: 209
-tick: 49, setAvailableIOTokens: 209
-tick: 50, setAvailableIOTokens: 209
-tick: 51, setAvailableIOTokens: 209
-tick: 52, setAvailableIOTokens: 209
-tick: 53, setAvailableIOTokens: 209
-tick: 54, setAvailableIOTokens: 209
-tick: 55, setAvailableIOTokens: 209
-tick: 56, setAvailableIOTokens: 209
-tick: 57, setAvailableIOTokens: 209
-tick: 58, setAvailableIOTokens: 209
-tick: 59, setAvailableIOTokens: 169
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.00x+1 ingested-lm: 0.75x+1 disk-bw-lm: 2.00x+1
+setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 1, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 2, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 3, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 4, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 5, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 6, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 7, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 8, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 9, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 10, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 11, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 12, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 13, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 14, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 15, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 16, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 17, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 18, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 19, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 20, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 21, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 22, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 23, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 24, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 25, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 26, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 27, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 28, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 29, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 30, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 31, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 32, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 33, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 34, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 35, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 36, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 37, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 38, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 39, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 40, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 41, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 42, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 43, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 44, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 45, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 46, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 47, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 48, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 49, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 50, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 51, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 52, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 53, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 54, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 55, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 56, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 57, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 58, setAvailableIOTokens: 209 setAvailableElasticDiskTokens: unlimited
+tick: 59, setAvailableIOTokens: 169 setAvailableElasticDiskTokens: unlimited
prep-admission-stats admitted=20000 write-bytes=80000
----
@@ -157,78 +157,78 @@ prep-admission-stats admitted=20000 write-bytes=80000
set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 2.25x+1 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 98 KiB [≈73 KiB], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:2.125 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:100000 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:2.25 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:2.25 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 7
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 417
-tick: 1, setAvailableIOTokens: 417
-tick: 2, setAvailableIOTokens: 417
-tick: 3, setAvailableIOTokens: 417
-tick: 4, setAvailableIOTokens: 417
-tick: 5, setAvailableIOTokens: 417
-tick: 6, setAvailableIOTokens: 417
-tick: 7, setAvailableIOTokens: 417
-tick: 8, setAvailableIOTokens: 417
-tick: 9, setAvailableIOTokens: 417
-tick: 10, setAvailableIOTokens: 417
-tick: 11, setAvailableIOTokens: 417
-tick: 12, setAvailableIOTokens: 417
-tick: 13, setAvailableIOTokens: 417
-tick: 14, setAvailableIOTokens: 417
-tick: 15, setAvailableIOTokens: 417
-tick: 16, setAvailableIOTokens: 417
-tick: 17, setAvailableIOTokens: 417
-tick: 18, setAvailableIOTokens: 417
-tick: 19, setAvailableIOTokens: 417
-tick: 20, setAvailableIOTokens: 417
-tick: 21, setAvailableIOTokens: 417
-tick: 22, setAvailableIOTokens: 417
-tick: 23, setAvailableIOTokens: 417
-tick: 24, setAvailableIOTokens: 417
-tick: 25, setAvailableIOTokens: 417
-tick: 26, setAvailableIOTokens: 417
-tick: 27, setAvailableIOTokens: 417
-tick: 28, setAvailableIOTokens: 417
-tick: 29, setAvailableIOTokens: 417
-tick: 30, setAvailableIOTokens: 417
-tick: 31, setAvailableIOTokens: 417
-tick: 32, setAvailableIOTokens: 417
-tick: 33, setAvailableIOTokens: 417
-tick: 34, setAvailableIOTokens: 417
-tick: 35, setAvailableIOTokens: 417
-tick: 36, setAvailableIOTokens: 417
-tick: 37, setAvailableIOTokens: 417
-tick: 38, setAvailableIOTokens: 417
-tick: 39, setAvailableIOTokens: 417
-tick: 40, setAvailableIOTokens: 417
-tick: 41, setAvailableIOTokens: 417
-tick: 42, setAvailableIOTokens: 417
-tick: 43, setAvailableIOTokens: 417
-tick: 44, setAvailableIOTokens: 417
-tick: 45, setAvailableIOTokens: 417
-tick: 46, setAvailableIOTokens: 417
-tick: 47, setAvailableIOTokens: 417
-tick: 48, setAvailableIOTokens: 417
-tick: 49, setAvailableIOTokens: 417
-tick: 50, setAvailableIOTokens: 417
-tick: 51, setAvailableIOTokens: 417
-tick: 52, setAvailableIOTokens: 417
-tick: 53, setAvailableIOTokens: 417
-tick: 54, setAvailableIOTokens: 417
-tick: 55, setAvailableIOTokens: 417
-tick: 56, setAvailableIOTokens: 417
-tick: 57, setAvailableIOTokens: 417
-tick: 58, setAvailableIOTokens: 417
-tick: 59, setAvailableIOTokens: 397
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1 disk-bw-lm: 2.12x+1
+setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 1, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 2, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 3, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 4, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 5, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 6, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 7, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 8, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 9, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 10, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 11, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 12, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 13, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 14, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 15, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 16, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 17, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 18, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 19, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 20, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 21, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 22, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 23, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 24, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 25, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 26, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 27, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 28, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 29, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 30, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 31, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 32, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 33, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 34, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 35, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 36, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 37, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 38, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 39, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 40, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 41, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 42, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 43, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 44, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 45, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 46, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 47, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 48, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 49, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 50, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 51, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 52, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 53, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 54, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 55, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 56, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 57, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 58, setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
+tick: 59, setAvailableIOTokens: 397 setAvailableElasticDiskTokens: unlimited
# No delta. This used to trigger an overflow bug.
set-state l0-bytes=10000 l0-added-write=201000 l0-files=21 l0-sublevels=21 print-only-first-tick=true
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 2.12x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 7 B, compacted 0 B [≈37 KiB], flushed 0 B [≈0 B]; admitting 21 KiB (rate 1.4 KiB/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:21875 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:37500 smoothedCompactionByteTokens:21875 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:21875 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:7} writeLM:{multiplier:2.125 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:2.125 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 7
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 365
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.12x+1 ingested-lm: 0.75x+1 disk-bw-lm: 2.12x+1
+setAvailableIOTokens: 365 setAvailableElasticDiskTokens: unlimited
prep-admission-stats admitted=30000 write-bytes=120000
----
@@ -239,10 +239,10 @@ prep-admission-stats admitted=30000 write-bytes=120000
set-state l0-bytes=10000 l0-added-write=501000 l0-files=21 l0-sublevels=20 print-only-first-tick=true
----
compaction score 1.000 (21 ssts, 20 sub-levels), L0 growth 293 KiB (write 293 KiB ingest 0 B ignored 0 B): requests 10000 (0 bypassed) with 39 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 3.00x+18 B (smoothed 2.56x+9 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 18 B, compacted 293 KiB [≈165 KiB], flushed 0 B [≈0 B]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:160937.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:18} writeLM:{multiplier:2.5625 constant:9} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intL0WriteAccountedBytes:40000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:3 constant:18} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:501000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:501000} smoothedIntL0CompactedBytes:168750 smoothedCompactionByteTokens:160937.5 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:18} writeLM:{multiplier:2.5625 constant:9} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:2.5625 constant:9} aux:{intL0AddedBytes:300000 intL0CompactedBytes:300000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10000 intL0WriteBytes:300000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:300000 intL0WriteAccountedBytes:40000 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:3 constant:18} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:3 constant:18} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:501000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 18
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.56x+9 ingested-lm: 0.75x+1
-setAvailableIOTokens: unlimited
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 2.56x+9 ingested-lm: 0.75x+1 disk-bw-lm: 2.56x+9
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
# Test cases with more information in storeAdmissionStats.
init
@@ -255,8 +255,8 @@ prep-admission-stats admitted=0
set-state l0-bytes=1000 l0-added-write=1000 l0-added-ingested=0 l0-files=21 l0-sublevels=21 print-only-first-tick=true
----
compaction score 0.000 (21 ssts, 21 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
-tick: 0, setAvailableIOTokens: unlimited
+{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
+tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
# L0 will see an addition of 200,000 bytes. 150,000 bytes were mentioned by
# the admitted requests.
@@ -267,10 +267,10 @@ prep-admission-stats admitted=10 write-bytes=130000 ingested-bytes=20000
set-state l0-bytes=1000 l0-added-write=171000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 195 KiB (write 166 KiB ingest 29 KiB ignored 0 B): requests 10 (0 bypassed) with 127 KiB acc-write (0 B bypassed) + 20 KiB acc-ingest (0 B bypassed) + write-model 1.31x+1 B (smoothed 1.53x+1 B) + ingested-model 1.50x+1 B (smoothed 1.12x+1 B) + at-admission-tokens 9.8 KiB, compacted 195 KiB [≈98 KiB], flushed 0 B [≈0 B]; admitting 24 KiB (rate 1.6 KiB/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:10000} writeLM:{multiplier:1.5288076923076923 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intL0WriteAccountedBytes:130000 intL0IngestedAccountedBytes:20000 intWriteLinearModel:{multiplier:1.3076153846153846 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:201000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:201000} smoothedIntL0CompactedBytes:100000 smoothedCompactionByteTokens:25000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:25000 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:10000} writeLM:{multiplier:1.5288076923076923 constant:1} ingestedLM:{multiplier:1.125 constant:1} diskBWLM:{multiplier:1.5416333333333334 constant:1} aux:{intL0AddedBytes:200000 intL0CompactedBytes:200000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:170000 intL0IngestedBytes:30000 intLSMWriteAndIngestedBytes:200000 intL0WriteAccountedBytes:130000 intIngestedAccountedBytes:20000 intWriteLinearModel:{multiplier:1.3076153846153846 constant:1} intIngestedLinearModel:{multiplier:1.4995 constant:1} intDiskBWLinearModel:{multiplier:1.3332666666666666 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:201000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 10000
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.53x+1 ingested-lm: 1.12x+1
-setAvailableIOTokens: 417
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.53x+1 ingested-lm: 1.12x+1 disk-bw-lm: 1.54x+1
+setAvailableIOTokens: 417 setAvailableElasticDiskTokens: unlimited
# L0 will see an addition of 20,000 bytes, all of which are accounted for.
# Since the ingested bytes in this interval are 0, the constant for the
@@ -282,10 +282,10 @@ prep-admission-stats admitted=20 write-bytes=150000 ingested-bytes=20000
set-state l0-bytes=1000 l0-added-write=191000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB ingest 0 B ignored 0 B): requests 10 (0 bypassed) with 20 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 1.00x+1 B (smoothed 1.26x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 5.9 KiB, compacted 20 KiB [≈59 KiB], flushed 0 B [≈0 B]; admitting 27 KiB (rate 1.8 KiB/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:27500 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:6000} writeLM:{multiplier:1.2641538461538462 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intL0WriteAccountedBytes:20000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.9995 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:221000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:221000} smoothedIntL0CompactedBytes:60000 smoothedCompactionByteTokens:27500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:27500 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:6000} writeLM:{multiplier:1.2641538461538462 constant:1} ingestedLM:{multiplier:1.125 constant:1} diskBWLM:{multiplier:1.2705666666666668 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:20000 intL0WriteAccountedBytes:20000 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.9995 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0.9995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:221000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 6000
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.26x+1 ingested-lm: 1.12x+1
-setAvailableIOTokens: 459
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.26x+1 ingested-lm: 1.12x+1 disk-bw-lm: 1.27x+1
+setAvailableIOTokens: 459 setAvailableElasticDiskTokens: unlimited
# L0 will see an addition of 20,000 bytes, but we think we have added 100,000
# bytes to L0. We don't let unaccounted bytes become negative.
@@ -298,10 +298,10 @@ prep-admission-stats admitted=30 write-bytes=250000 ingested-bytes=20000 ingeste
set-state l0-bytes=1000 l0-added-write=211000 l0-added-ingested=30000 l0-files=21 l0-sublevels=21 print-only-first-tick=true
----
compaction score 1.050[L0-overload] (21 ssts, 21 sub-levels), L0 growth 20 KiB (write 20 KiB ingest 0 B ignored 0 B): requests 10 (0 bypassed) with 98 KiB acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.50x+1 B (smoothed 0.88x+1 B) + ingested-model 0.00x+0 B (smoothed 1.12x+1 B) + at-admission-tokens 3.9 KiB, compacted 20 KiB [≈39 KiB], flushed 0 B [≈0 B]; admitting 23 KiB (rate 1.5 KiB/s) due to L0 growth (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:23750 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:4000} writeLM:{multiplier:0.8820769230769231 constant:1} ingestedLM:{multiplier:1.125 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intL0WriteAccountedBytes:100000 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.5 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:241000 curL0Bytes:1000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:241000} smoothedIntL0CompactedBytes:40000 smoothedCompactionByteTokens:23750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:23750 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:4000} writeLM:{multiplier:0.8820769230769231 constant:1} ingestedLM:{multiplier:1.125 constant:1} diskBWLM:{multiplier:0.8852833333333334 constant:1} aux:{intL0AddedBytes:20000 intL0CompactedBytes:20000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:10 intL0WriteBytes:20000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:20000 intL0WriteAccountedBytes:100000 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0.5 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0.5 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:241000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 4000
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 0.88x+1 ingested-lm: 1.12x+1
-setAvailableIOTokens: 396
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 0.88x+1 ingested-lm: 1.12x+1 disk-bw-lm: 0.89x+1
+setAvailableIOTokens: 396 setAvailableElasticDiskTokens: unlimited
# Test case with flush tokens.
init
@@ -314,27 +314,27 @@ prep-admission-stats admitted=0
set-state l0-bytes=10000 l0-added-write=1000 l0-files=1 l0-sublevels=1 print-only-first-tick=true
----
compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
-tick: 0, setAvailableIOTokens: unlimited
+{ioLoadListenerState:{cumL0AddedBytes:1000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:1000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
+tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
# Flush loop utilization is too low for the interval flush tokens to
# contribute to the smoothed value, or for tokens to become limited.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=100 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 9.8 KiB (write 9.8 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 9.8 KiB [≈4.9 KiB], flushed 7.3 KiB [≈0 B]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:5000 smoothedCompactionByteTokens:5000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:10000 intL0CompactedBytes:10000 intFlushTokens:7500 intFlushUtilization:0.0196078431372549 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:10000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:10000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: unlimited
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
# Flush loop utilization is high enough, so we compute flush tokens for limiting admission.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=1000 flush-work-sec=2 flush-idle-sec=10 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈2.4 KiB], flushed 7.3 KiB [≈7.3 KiB]; admitting 11 KiB (rate 750 B/s) due to memtable flush (multiplier 1.500) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2500 smoothedCompactionByteTokens:2500 smoothedNumFlushTokens:7500 flushUtilTargetFraction:1.5 totalNumByteTokens:11250 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:7500 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 188
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 188 setAvailableElasticDiskTokens: unlimited
# Write stalls are happening, so decrease the flush utilization target
# fraction from 1.5 to 1.475. But the peak flush rate has also increased since
@@ -342,10 +342,10 @@ setAvailableIOTokens: 188
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=1 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈1.2 KiB], flushed 73 KiB [≈40 KiB]; admitting 59 KiB (rate 4.0 KiB/s) due to memtable flush (multiplier 1.475) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:1 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1250 smoothedCompactionByteTokens:1250 smoothedNumFlushTokens:41250 flushUtilTargetFraction:1.475 totalNumByteTokens:60843 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1015
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1015 setAvailableElasticDiskTokens: unlimited
# Two write stalls happened, so decrease the flush utilization target fraction
# by a bigger step, from 1.475 to 1.425. Since the smoothed peak flush rate is
@@ -353,29 +353,29 @@ setAvailableIOTokens: 1015
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=3 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈625 B], flushed 73 KiB [≈57 KiB]; admitting 81 KiB (rate 5.4 KiB/s) due to memtable flush (multiplier 1.425) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:3 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:625 smoothedCompactionByteTokens:625 smoothedNumFlushTokens:58125 flushUtilTargetFraction:1.4250000000000003 totalNumByteTokens:82828 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:2 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1381
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1381 setAvailableElasticDiskTokens: unlimited
# Five more write stalls, so the the flush utilization target fraction is
# decreased to 1.35. The smoothed peak flush rate continues to increase.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=8 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈312 B], flushed 73 KiB [≈65 KiB]; admitting 88 KiB (rate 5.8 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:8 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:312 smoothedCompactionByteTokens:312.5 smoothedNumFlushTokens:66562.5 flushUtilTargetFraction:1.3500000000000005 totalNumByteTokens:89859 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:5 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1498
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1498 setAvailableElasticDiskTokens: unlimited
# Another write stall, and the flush utilization target fraction drops to 1.325.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=9 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈156 B], flushed 73 KiB [≈69 KiB]; admitting 92 KiB (rate 6.1 KiB/s) due to memtable flush (multiplier 1.325) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:9 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:156 smoothedCompactionByteTokens:156.25 smoothedNumFlushTokens:70781.25 flushUtilTargetFraction:1.3250000000000006 totalNumByteTokens:93785 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1564
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1564 setAvailableElasticDiskTokens: unlimited
# Set a lower bound of 1.3 on the flush utilization target fraction.
set-min-flush-util percent=130
@@ -386,20 +386,20 @@ set-min-flush-util percent=130
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=10 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈78 B], flushed 73 KiB [≈71 KiB]; admitting 92 KiB (rate 6.2 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:10 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:78 smoothedCompactionByteTokens:78.125 smoothedNumFlushTokens:72890.625 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:94757 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1580
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1580 setAvailableElasticDiskTokens: unlimited
# Despite another write stall, the flush utilization target fraction does not
# decrease since it is already at the lower bound.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=11 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈39 B], flushed 73 KiB [≈72 KiB]; admitting 94 KiB (rate 6.3 KiB/s) due to memtable flush (multiplier 1.300) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:11 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:39 smoothedCompactionByteTokens:39.0625 smoothedNumFlushTokens:73945.3125 flushUtilTargetFraction:1.3000000000000007 totalNumByteTokens:96128 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1603
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1603 setAvailableElasticDiskTokens: unlimited
# Bump up the lower bound to 1.35, which is greater than the current flush
# utilization target fraction.
@@ -411,65 +411,107 @@ set-min-flush-util percent=135
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=12 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈19 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.5 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:12 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:19 smoothedCompactionByteTokens:19.53125 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:100538 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1676
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1676 setAvailableElasticDiskTokens: unlimited
# The flush utilization is too low, so there is no limit on flush tokens.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=100 write-stall-count=13 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈9 B], flushed 73 KiB [≈73 KiB]; admitting all
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:9 smoothedCompactionByteTokens:9.765625 smoothedNumFlushTokens:74472.65625 flushUtilTargetFraction:1.35 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.0196078431372549 intWriteStalls:1 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: unlimited
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
# Flush utilization is high enough, so flush tokens are again limited.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈4 B], flushed 73 KiB [≈73 KiB]; admitting 98 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:4 smoothedCompactionByteTokens:4.8828125 smoothedNumFlushTokens:74736.328125 flushUtilTargetFraction:1.35 totalNumByteTokens:100894 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1682
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1682 setAvailableElasticDiskTokens: unlimited
# No write stalls, and token utilization is high, which will have an effect
# in the next pebbleMetricsTick.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈2 B], flushed 73 KiB [≈73 KiB]; admitting 99 KiB (rate 6.6 KiB/s) due to memtable flush (multiplier 1.350) (used 0 B)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:2 smoothedCompactionByteTokens:2.44140625 smoothedNumFlushTokens:74868.1640625 flushUtilTargetFraction:1.35 totalNumByteTokens:101072 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:0 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1685
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1685 setAvailableElasticDiskTokens: unlimited
# No write stalls, and token utilization was high, so flush utilization
# target fraction is increased to 1.375.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈1 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 197 KiB)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:1 smoothedCompactionByteTokens:1.220703125 smoothedNumFlushTokens:74934.08203125 flushUtilTargetFraction:1.375 totalNumByteTokens:103034 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:202144 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1718
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1718 setAvailableElasticDiskTokens: unlimited
# No write stalls, and token utilization was high, so flush utilization
# target fraction is increased to 1.4.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=13 all-tokens-used=true print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 102 KiB (rate 6.8 KiB/s) due to memtable flush (multiplier 1.400) (used 201 KiB)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:13 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.6103515625 smoothedNumFlushTokens:74967.041015625 flushUtilTargetFraction:1.4 totalNumByteTokens:104953 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:0 prevTokensUsed:206068 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1750
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1750 setAvailableElasticDiskTokens: unlimited
# There is a write stall, so even though token utilization is high, we
# decrease flush utilization target fraction to 1.375.
set-state l0-bytes=10000 l0-added-write=11000 l0-files=1 l0-sublevels=1 flush-bytes=10000 flush-work-sec=2 flush-idle-sec=10 write-stall-count=14 all-tokens-used=true print-only-first-tick=true
----
compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 73 KiB [≈73 KiB]; admitting 101 KiB (rate 6.7 KiB/s) due to memtable flush (multiplier 1.375) (used 205 KiB)
-{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 tokensAllocated:0 tokensUsed:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true} ioThreshold:}
+{ioLoadListenerState:{cumL0AddedBytes:11000 curL0Bytes:10000 cumWriteStallCount:14 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:11000} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0.30517578125 smoothedNumFlushTokens:74983.5205078125 flushUtilTargetFraction:1.375 totalNumByteTokens:103102 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:75000 intFlushUtilization:0.16666666666666666 intWriteStalls:1 prevTokensUsed:209906 tokenKind:1 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:true diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:11000 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
store-request-estimates: writeTokens: 1
-tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1
-setAvailableIOTokens: 1719
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: 1719 setAvailableElasticDiskTokens: unlimited
+
+# Test disk bandwidth tokens.
+init
+----
+
+set-state l0-bytes=100 l0-added-write=0 bytes-read=0 bytes-written=0 provisioned-bandwidth=10 l0-files=1 l0-sublevels=1 print-only-first-tick=true
+----
+compaction score 0.000 (1 ssts, 1 sub-levels), L0 growth 0 B (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 0 B [≈0 B], flushed 0 B [≈0 B]; admitting all
+{ioLoadListenerState:{cumL0AddedBytes:0 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:0 bytesWritten:0 incomingLSMBytes:0} smoothedIntL0CompactedBytes:0 smoothedCompactionByteTokens:0 smoothedNumFlushTokens:0 flushUtilTargetFraction:0 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:0 intL0CompactedBytes:0 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:0 writeBandwidth:0 provisionedBandwidth:0} intervalLSMInfo:{incomingBytes:0 regularTokensUsed:0 elasticTokensUsed:0}}} ioThreshold:}
+tick: 0, setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
+
+set-state l0-bytes=100 l0-added-write=100000 bytes-read=1000000 bytes-written=2000000 provisioned-bandwidth=10 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true
+----
+compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 0 B ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈49 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): work model 1.75x+1 B, disk bw read 65 KiB write 130 KiB provisioned 10 B
+{ioLoadListenerState:{cumL0AddedBytes:100000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:1000000 bytesWritten:2000000 incomingLSMBytes:100000} smoothedIntL0CompactedBytes:50000 smoothedCompactionByteTokens:50000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:66666 writeBandwidth:133333 provisionedBandwidth:10} intervalLSMInfo:{incomingBytes:100000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:}
+store-request-estimates: writeTokens: 1
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105
+
+set-state l0-bytes=100 l0-added-write=200000 bytes-read=2000000 bytes-written=4000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true
+----
+compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈73 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): work model 1.75x+1 B, disk bw read 130 KiB write 260 KiB provisioned 3.8 MiB
+{ioLoadListenerState:{cumL0AddedBytes:200000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:2000000 bytesWritten:4000000 incomingLSMBytes:200000} smoothedIntL0CompactedBytes:75000 smoothedCompactionByteTokens:75000 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:100000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:133333 writeBandwidth:266666 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:200000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:}
+store-request-estimates: writeTokens: 1
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105
+
+set-state l0-bytes=100 l0-added-write=300000 bytes-read=3000000 bytes-written=6000000 provisioned-bandwidth=4000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true
+----
+compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈85 KiB], flushed 0 B [≈0 B]; admitting all; elastic tokens 6.1 KiB (used 100 B, regular used 100 B): work model 1.75x+1 B, disk bw read 195 KiB write 391 KiB provisioned 3.8 MiB
+{ioLoadListenerState:{cumL0AddedBytes:300000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:3000000 bytesWritten:6000000 incomingLSMBytes:300000} smoothedIntL0CompactedBytes:87500 smoothedCompactionByteTokens:87500 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:6250 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:100000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:200000 writeBandwidth:400000 provisionedBandwidth:4000000} intervalLSMInfo:{incomingBytes:300000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:}
+store-request-estimates: writeTokens: 1
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: 105
+
+set-state l0-bytes=100 l0-added-write=400000 bytes-read=4000000 bytes-written=8000000 provisioned-bandwidth=5000000 disk-bw-tokens-used=(100,100) l0-files=1 l0-sublevels=1 print-only-first-tick=true
+----
+compaction score 0.050 (1 ssts, 1 sub-levels), L0 growth 98 KiB (write 98 KiB ingest 0 B ignored 0 B): requests 0 (0 bypassed) with 0 B acc-write (0 B bypassed) + 0 B acc-ingest (0 B bypassed) + write-model 0.00x+0 B (smoothed 1.75x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 1 B, compacted 98 KiB [≈92 KiB], flushed 0 B [≈0 B]; admitting all
+{ioLoadListenerState:{cumL0AddedBytes:400000 curL0Bytes:100 cumWriteStallCount:0 diskBW:{bytesRead:4000000 bytesWritten:8000000 incomingLSMBytes:400000} smoothedIntL0CompactedBytes:93750 smoothedCompactionByteTokens:93750 smoothedNumFlushTokens:0 flushUtilTargetFraction:1.5 totalNumByteTokens:9223372036854775807 byteTokensAllocated:0 byteTokensUsed:0 elasticDiskBWTokens:9223372036854775807 elasticDiskBWTokensAllocated:0} requestEstimates:{writeTokens:1} writeLM:{multiplier:1.75 constant:1} ingestedLM:{multiplier:0.7505 constant:1} diskBWLM:{multiplier:1.75 constant:1} aux:{intL0AddedBytes:100000 intL0CompactedBytes:100000 intFlushTokens:0 intFlushUtilization:0 intWriteStalls:0 prevTokensUsed:0 tokenKind:0 perWorkTokensAux:{intWorkCount:0 intL0WriteBytes:100000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:100000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0} doLogFlush:false diskBW:{intervalDiskLoadInfo:{readBandwidth:266666 writeBandwidth:533333 provisionedBandwidth:5000000} intervalLSMInfo:{incomingBytes:400000 regularTokensUsed:100 elasticTokensUsed:100}}} ioThreshold:}
+store-request-estimates: writeTokens: 1
+tick: 0, setAdmittedDoneModelsLocked: write-lm: 1.75x+1 ingested-lm: 0.75x+1 disk-bw-lm: 1.75x+1
+setAvailableIOTokens: unlimited setAvailableElasticDiskTokens: unlimited
diff --git a/pkg/util/admission/testdata/store_per_work_token_estimator b/pkg/util/admission/testdata/store_per_work_token_estimator
index 62b8a5a32236..cfcc76b07d2b 100644
--- a/pkg/util/admission/testdata/store_per_work_token_estimator
+++ b/pkg/util/admission/testdata/store_per_work_token_estimator
@@ -4,71 +4,79 @@ init
# First call initializes.
update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0
----
-interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:0 intL0WriteBytes:0 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:0 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 1
write-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1
ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1
+disk-bw-tokens: int: 0.00x+0 smoothed: 1.75x+1 per-work-accounted: 1
# Writes account for ~1/2 of what is written, reflecting what can happen with
# application to the state machine. No ingests.
update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0
----
-interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:1000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:1.98 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 50
write-tokens: int: 1.98x+1 smoothed: 1.86x+1 per-work-accounted: 25
ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1
+disk-bw-tokens: int: 1.98x+1 smoothed: 1.86x+1 per-work-accounted: 25
# Same as previous, except some of these are bypassed. Will not affect the model.
update flushed=1000 ingested=0 admitted=10 write-accounted=500 ingested-accounted=0 bypassed-count=4 bypassed-write=300 bypassed-ingested=0
----
-interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:1000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:0 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0 constant:0} intDiskBWLinearModel:{multiplier:1.98 constant:1} intBypassedWorkCount:4 intL0WriteBypassedAccountedBytes:300 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 75
write-tokens: int: 1.98x+1 smoothed: 1.92x+1 per-work-accounted: 37
ingest-tokens: int: 0.00x+0 smoothed: 0.75x+1 per-work-accounted: 1
+disk-bw-tokens: int: 1.98x+1 smoothed: 1.92x+1 per-work-accounted: 37
# Ingestion also happens. Bumps up the at-admission-tokens since at that time
# we can't differentiate between writes and ingests. The constants in the
# linear models stays 1, since we can fit effectively using the multipliers.
# This means a mix of regular writes and sstable ingests (say index
# backfills), will not effect the cost attributed to regular writes.
-update flushed=1000 ingested=1000 admitted=10 write-accounted=500 ingested-accounted=4000 bypassed-count=2 bypassed-write=0 bypassed-ingested=2000
+update flushed=1000 ingested=1000 other-levels-ingested=9000 admitted=10 write-accounted=500 ingested-accounted=4000 bypassed-count=2 bypassed-write=0 bypassed-ingested=2000
----
-interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intL0WriteAccountedBytes:500 intL0IngestedAccountedBytes:4000 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0.2475 constant:1} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:2000 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000 intLSMWriteAndIngestedBytes:11000 intL0WriteAccountedBytes:500 intIngestedAccountedBytes:4000 intWriteLinearModel:{multiplier:1.98 constant:1} intIngestedLinearModel:{multiplier:0.2475 constant:1} intDiskBWLinearModel:{multiplier:2.442222222222222 constant:1} intBypassedWorkCount:2 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:2000 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 137
write-tokens: int: 1.98x+1 smoothed: 1.95x+1 per-work-accounted: 43
ingest-tokens: int: 0.25x+1 smoothed: 0.50x+1 per-work-accounted: 200
+disk-bw-tokens: int: 2.44x+1 smoothed: 2.18x+1 per-work-accounted: 243
-# No ingestion observed by LSM, though ingested-accounted is non-zero -- this
-# updates the model since all these ingested bytes could have gone to levels
+# No ingestion into L0 observed by LSM, though ingested-accounted is non-zero
+# -- this updates the model since all these ingested bytes have gone to levels
# lower than L0.
-update flushed=1000 ingested=0 admitted=10 write-accounted=450 ingested-accounted=500
+update flushed=1000 ingested=0 other-levels-ingested=500 admitted=10 write-accounted=450 ingested-accounted=500
----
-interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intL0WriteAccountedBytes:450 intL0IngestedAccountedBytes:500 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.001 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:0 intLSMWriteAndIngestedBytes:1500 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:500 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.001 constant:1} intDiskBWLinearModel:{multiplier:1.568421052631579 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 118
write-tokens: int: 2.20x+1 smoothed: 2.08x+1 per-work-accounted: 44
ingest-tokens: int: 0.00x+1 smoothed: 0.25x+1 per-work-accounted: 125
+disk-bw-tokens: int: 1.57x+1 smoothed: 1.88x+1 per-work-accounted: 169
# Large amount of ingestion. Bumps up at-admission-tokens.
-update flushed=1000 ingested=1000000 admitted=10 write-accounted=450 ingested-accounted=2000000
+update flushed=1000 ingested=1000000 other-levels-ingested=2000000 admitted=10 write-accounted=450 ingested-accounted=2000000
----
-interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:450 intL0IngestedAccountedBytes:2000000 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.499995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
+interval state: {intWorkCount:10 intL0WriteBytes:1000 intL0IngestedBytes:1000000 intLSMWriteAndIngestedBytes:3001000 intL0WriteAccountedBytes:450 intIngestedAccountedBytes:2000000 intWriteLinearModel:{multiplier:2.2 constant:1} intIngestedLinearModel:{multiplier:0.499995 constant:1} intDiskBWLinearModel:{multiplier:1.5001574645704716 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:0}
at-admission-tokens: 50109
write-tokens: int: 2.20x+1 smoothed: 2.14x+1 per-work-accounted: 44
ingest-tokens: int: 0.50x+1 smoothed: 0.37x+1 per-work-accounted: 100062
+disk-bw-tokens: int: 1.50x+1 smoothed: 1.69x+1 per-work-accounted: 100107
# Large number of bytes ingested into L0, but only 2000 are not to be ignored.
# So we can fit a reasonable model.
update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000
----
-interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000}
+interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMWriteAndIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intDiskBWLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000}
at-admission-tokens: 25154
write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44
ingest-tokens: int: 0.99x+1 smoothed: 0.68x+1 per-work-accounted: 50131
+disk-bw-tokens: int: 0.99x+1 smoothed: 1.34x+1 per-work-accounted: 50153
# Repeat of the previous.
update flushed=0 ingested=1000000 admitted=10 write-accounted=0 ingested-accounted=2000 ignore-ingested-into-L0=998000
----
-interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intL0WriteAccountedBytes:0 intL0IngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intL0IngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000}
+interval state: {intWorkCount:10 intL0WriteBytes:0 intL0IngestedBytes:1000000 intLSMWriteAndIngestedBytes:1000000 intL0WriteAccountedBytes:0 intIngestedAccountedBytes:2000 intWriteLinearModel:{multiplier:0 constant:0} intIngestedLinearModel:{multiplier:0.995 constant:1} intDiskBWLinearModel:{multiplier:0.995 constant:1} intBypassedWorkCount:0 intL0WriteBypassedAccountedBytes:0 intIngestedBypassedAccountedBytes:0 intL0IgnoredIngestedBytes:998000}
at-admission-tokens: 12677
write-tokens: int: 0.00x+0 smoothed: 2.14x+1 per-work-accounted: 44
ingest-tokens: int: 0.99x+1 smoothed: 0.84x+1 per-work-accounted: 25165
+disk-bw-tokens: int: 0.99x+1 smoothed: 1.17x+1 per-work-accounted: 25176
diff --git a/pkg/util/admission/testdata/store_work_queue b/pkg/util/admission/testdata/store_work_queue
index a3584af56438..ae615d87acd6 100644
--- a/pkg/util/admission/testdata/store_work_queue
+++ b/pkg/util/admission/testdata/store_work_queue
@@ -3,7 +3,8 @@ init
print
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:0 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:1}
@@ -12,35 +13,37 @@ set-try-get-return-value v=true
admit id=1 tenant=53 priority=0 create-time-millis=1 bypass=false
----
-tryGet: returning true
-id 1: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:1 admissionEnabled:true}
+tryGet regular: returning true
+id 1: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:1 workClass:0 admissionEnabled:true}
work-done id=1
----
-storeWriteDone: originalTokens 1, doneBytes(write 0,ingested 0) returning 0
+storeWriteDone regular: originalTokens 1, doneBytes(write 0,ingested 0) returning 0
set-store-request-estimates write-tokens=100
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 1, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:1 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:100}
admit id=2 tenant=55 priority=0 create-time-millis=1 bypass=false
----
-tryGet: returning true
-id 2: admit succeeded with handle {tenantID:{InternalValue:55} writeTokens:100 admissionEnabled:true}
+tryGet regular: returning true
+id 2: admit succeeded with handle {tenantID:{InternalValue:55} writeTokens:100 workClass:0 admissionEnabled:true}
admit id=3 tenant=53 priority=0 create-time-millis=1 bypass=false
----
-tryGet: returning true
-id 3: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:100 admissionEnabled:true}
+tryGet regular: returning true
+id 3: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:100 workClass:0 admissionEnabled:true}
print
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 101, w: 1, fifo: -128
tenant-id: 55 used: 100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:1 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:100}
@@ -49,85 +52,176 @@ set-try-get-return-value v=false
admit id=4 tenant=57 priority=0 create-time-millis=1 bypass=false
----
-tryGet: returning false
+tryGet regular: returning false
work-done id=2 additional-tokens=500
----
-storeWriteDone: originalTokens 100, doneBytes(write 0,ingested 0) returning 500
+storeWriteDone regular: originalTokens 100, doneBytes(write 0,ingested 0) returning 500
print
----
-closed epoch: 0 tenantHeap len: 1 top tenant: 57
+regular workqueue: closed epoch: 0 tenantHeap len: 1 top tenant: 57
tenant-id: 53 used: 101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 0, w: 1, fifo: -128 waiting work heap: [0: pri: 0, ct: 1, epoch: 0, qt: 0]
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:2 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:100}
granted
----
-continueGrantChain 0
-id 4: admit succeeded with handle {tenantID:{InternalValue:57} writeTokens:100 admissionEnabled:true}
-granted: returned 100
+continueGrantChain regular 0
+id 4: admit succeeded with handle {tenantID:{InternalValue:57} writeTokens:100 workClass:0 admissionEnabled:true}
+granted regular: returned 100
print
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:2 writeAccountedBytes:0 ingestedAccountedBytes:0 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:100}
work-done id=3 ingested-bytes=1000000 additional-tokens=50000
----
-storeWriteDone: originalTokens 100, doneBytes(write 0,ingested 1000000) returning 50000
+storeWriteDone regular: originalTokens 100, doneBytes(write 0,ingested 1000000) returning 50000
print
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 50101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:3 writeAccountedBytes:0 ingestedAccountedBytes:1000000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:100}
set-store-request-estimates write-tokens=10000
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 50101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:3 writeAccountedBytes:0 ingestedAccountedBytes:1000000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:10000}
work-done id=4 write-bytes=2000 ingested-bytes=1000 additional-tokens=2000
----
-storeWriteDone: originalTokens 100, doneBytes(write 2000,ingested 1000) returning 2000
+storeWriteDone regular: originalTokens 100, doneBytes(write 2000,ingested 1000) returning 2000
print
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 50101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:4 writeAccountedBytes:2000 ingestedAccountedBytes:1001000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:0 writeBypassedAccountedBytes:0 ingestedBypassedAccountedBytes:0}}
estimates:{writeTokens:10000}
bypassed-work-done work-count=10 write-bytes=1000 ingested-bytes=1000000
----
-closed epoch: 0 tenantHeap len: 0
+storeWriteDone regular: originalTokens 0, doneBytes(write 1000,ingested 1000000) returning 2000
+
+print
+----
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 50101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:0 ApproxIngestedIntoL0Bytes:0}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
estimates:{writeTokens:10000}
stats-to-ignore ingested-bytes=12000 ingested-into-L0-bytes=9000
----
-closed epoch: 0 tenantHeap len: 0
+regular workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 50101, w: 1, fifo: -128
+ tenant-id: 55 used: 600, w: 1, fifo: -128
+ tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
+stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
+estimates:{writeTokens:10000}
+
+# Elastic work.
+admit id=5 tenant=53 priority=-30 create-time-millis=1 bypass=false
+----
+tryGet elastic: returning false
+
+# Noop since the only waiting work is elastic and this is granting to regular.
+# work.
+granted
+----
+granted regular: returned 0
+
+granted elastic=true
+----
+continueGrantChain elastic 0
+id 5: admit succeeded with handle {tenantID:{InternalValue:53} writeTokens:10000 workClass:1 admissionEnabled:true}
+granted elastic: returned 10000
+
+print
+----
+regular workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 50101, w: 1, fifo: -128
+ tenant-id: 55 used: 600, w: 1, fifo: -128
+ tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 10000, w: 1, fifo: -128
+stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
+estimates:{writeTokens:10000}
+
+set-try-get-return-value v=true elastic=true
+----
+
+admit id=6 tenant=54 priority=-40 create-time-millis=3 bypass=false
+----
+tryGet elastic: returning true
+id 6: admit succeeded with handle {tenantID:{InternalValue:54} writeTokens:10000 workClass:1 admissionEnabled:true}
+
+print
+----
+regular workqueue: closed epoch: 0 tenantHeap len: 0
tenant-id: 53 used: 50101, w: 1, fifo: -128
tenant-id: 55 used: 600, w: 1, fifo: -128
tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 10000, w: 1, fifo: -128
+ tenant-id: 54 used: 10000, w: 1, fifo: -128
stats:{admittedCount:14 writeAccountedBytes:3000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
estimates:{writeTokens:10000}
+
+work-done id=5 write-bytes=1000 additional-tokens=200
+----
+storeWriteDone elastic: originalTokens 10000, doneBytes(write 1000,ingested 0) returning 200
+
+print
+----
+regular workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 50101, w: 1, fifo: -128
+ tenant-id: 55 used: 600, w: 1, fifo: -128
+ tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 10200, w: 1, fifo: -128
+ tenant-id: 54 used: 10000, w: 1, fifo: -128
+stats:{admittedCount:15 writeAccountedBytes:4000 ingestedAccountedBytes:2001000 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
+estimates:{writeTokens:10000}
+
+work-done id=6 ingested-bytes=500 additional-tokens=500
+----
+storeWriteDone elastic: originalTokens 10000, doneBytes(write 0,ingested 500) returning 500
+
+print
+----
+regular workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 50101, w: 1, fifo: -128
+ tenant-id: 55 used: 600, w: 1, fifo: -128
+ tenant-id: 57 used: 2100, w: 1, fifo: -128
+elastic workqueue: closed epoch: 0 tenantHeap len: 0
+ tenant-id: 53 used: 10200, w: 1, fifo: -128
+ tenant-id: 54 used: 10500, w: 1, fifo: -128
+stats:{admittedCount:16 writeAccountedBytes:4000 ingestedAccountedBytes:2001500 statsToIgnore:{IngestOperationStats:{Bytes:12000 ApproxIngestedIntoL0Bytes:9000}} aux:{bypassedCount:10 writeBypassedAccountedBytes:1000 ingestedBypassedAccountedBytes:1000000}}
+estimates:{writeTokens:10000}
diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go
index dbf7cd258df5..c51e42167bc4 100644
--- a/pkg/util/admission/work_queue.go
+++ b/pkg/util/admission/work_queue.go
@@ -1559,11 +1559,11 @@ type StoreWriteWorkInfo struct {
// StoreWorkQueue is responsible for admission to a store.
type StoreWorkQueue struct {
- q WorkQueue
+ q [numWorkClasses]WorkQueue
// Only calls storeWriteDone. The rest of the interface is used by
// WorkQueue.
- granter granterWithStoreWriteDone
- mu struct {
+ granters [numWorkClasses]granterWithStoreWriteDone
+ mu struct {
syncutil.RWMutex
estimates storeRequestEstimates
stats storeAdmissionStats
@@ -1577,6 +1577,7 @@ type StoreWorkHandle struct {
tenantID roachpb.TenantID
// The writeTokens acquired by this request. Must be > 0.
writeTokens int64
+ workClass workClass
admissionEnabled bool
}
@@ -1593,15 +1594,21 @@ func (h StoreWorkHandle) AdmissionEnabled() bool {
func (q *StoreWorkQueue) Admit(
ctx context.Context, info StoreWriteWorkInfo,
) (handle StoreWorkHandle, err error) {
+ // For now, we compute a workClass based on priority.
+ wc := regularWorkClass
+ if info.Priority < admissionpb.NormalPri {
+ wc = elasticWorkClass
+ }
h := StoreWorkHandle{
- tenantID: info.TenantID,
+ tenantID: info.TenantID,
+ workClass: wc,
}
q.mu.RLock()
estimates := q.mu.estimates
q.mu.RUnlock()
h.writeTokens = estimates.writeTokens
info.WorkInfo.requestedCount = h.writeTokens
- enabled, err := q.q.Admit(ctx, info.WorkInfo)
+ enabled, err := q.q[wc].Admit(ctx, info.WorkInfo)
if err != nil {
return StoreWorkHandle{}, err
}
@@ -1616,13 +1623,6 @@ type StoreWorkDoneInfo struct {
// the write-batch is empty, which happens when all the bytes are being
// added via sstable ingestion. NB: it is possible for both WriteBytes and
// IngestedBytes to be 0 if nothing was actually written.
- //
- // TODO(sumeer): WriteBytes will under count the actual effect on the Pebble
- // store shared by the raft log and the state machine, since this only
- // reflects the changes to the raft log. We compensate for this with an
- // additive adjustment which is the same across all writes regardless of
- // bytes. We should consider using an adjustment that is proportional to the
- // WriteBytes.
WriteBytes int64
// The size of the sstables, for ingests. Zero if there were no ingests.
IngestedBytes int64
@@ -1635,8 +1635,8 @@ func (q *StoreWorkQueue) AdmittedWorkDone(h StoreWorkHandle, doneInfo StoreWorkD
return nil
}
q.updateStoreAdmissionStats(1, doneInfo, false)
- additionalTokens := q.granter.storeWriteDone(h.writeTokens, doneInfo)
- q.q.adjustTenantTokens(h.tenantID, additionalTokens)
+ additionalTokens := q.granters[h.workClass].storeWriteDone(h.writeTokens, doneInfo)
+ q.q[h.workClass].adjustTenantTokens(h.tenantID, additionalTokens)
return nil
}
@@ -1645,7 +1645,9 @@ func (q *StoreWorkQueue) AdmittedWorkDone(h StoreWorkHandle, doneInfo StoreWorkD
// estimation model.
func (q *StoreWorkQueue) BypassedWorkDone(workCount int64, doneInfo StoreWorkDoneInfo) {
q.updateStoreAdmissionStats(uint64(workCount), doneInfo, true)
- _ = q.granter.storeWriteDone(0, doneInfo)
+ // Since we have no control over such work, we choose to count it as
+ // regularWorkClass.
+ _ = q.granters[regularWorkClass].storeWriteDone(0, doneInfo)
}
// StatsToIgnore is called for range snapshot ingestion -- see the comment in
@@ -1674,19 +1676,24 @@ func (q *StoreWorkQueue) updateStoreAdmissionStats(
// SetTenantWeights passes through to WorkQueue.SetTenantWeights.
func (q *StoreWorkQueue) SetTenantWeights(tenantWeights map[uint64]uint32) {
- q.q.SetTenantWeights(tenantWeights)
-}
-
-func (q *StoreWorkQueue) hasWaitingRequests() bool {
- return q.q.hasWaitingRequests()
+ for i := range q.q {
+ q.q[i].SetTenantWeights(tenantWeights)
+ }
}
-func (q *StoreWorkQueue) granted(grantChainID grantChainID) int64 {
- return q.q.granted(grantChainID)
+// getRequesters implements storeRequester.
+func (q *StoreWorkQueue) getRequesters() [numWorkClasses]requester {
+ var result [numWorkClasses]requester
+ for i := range q.q {
+ result[i] = &q.q[i]
+ }
+ return result
}
func (q *StoreWorkQueue) close() {
- q.q.close()
+ for i := range q.q {
+ q.q[i].close()
+ }
}
func (q *StoreWorkQueue) getStoreAdmissionStats() storeAdmissionStats {
@@ -1703,14 +1710,16 @@ func (q *StoreWorkQueue) setStoreRequestEstimates(estimates storeRequestEstimate
func makeStoreWorkQueue(
ambientCtx log.AmbientContext,
- granter granterWithStoreWriteDone,
+ granters [numWorkClasses]granterWithStoreWriteDone,
settings *cluster.Settings,
opts workQueueOptions,
) storeRequester {
q := &StoreWorkQueue{
- granter: granter,
+ granters: granters,
+ }
+ for i := range q.q {
+ initWorkQueue(&q.q[i], ambientCtx, KVWork, granters[i], settings, opts)
}
- initWorkQueue(&q.q, ambientCtx, KVWork, granter, settings, opts)
// Arbitrary initial value. This will be replaced before any meaningful
// token constraints are enforced.
q.mu.estimates = storeRequestEstimates{
diff --git a/pkg/util/admission/work_queue_test.go b/pkg/util/admission/work_queue_test.go
index 7b43fc7518a1..7e74ab6a4649 100644
--- a/pkg/util/admission/work_queue_test.go
+++ b/pkg/util/admission/work_queue_test.go
@@ -56,29 +56,30 @@ func (b *builderWithMu) stringAndReset() string {
}
type testGranter struct {
+ name string
buf *builderWithMu
r requester
returnValueFromTryGet bool
additionalTokens int64
}
-var _ granter = &testGranter{}
+var _ granterWithStoreWriteDone = &testGranter{}
func (tg *testGranter) grantKind() grantKind {
return slot
}
func (tg *testGranter) tryGet(count int64) bool {
- tg.buf.printf("tryGet: returning %t", tg.returnValueFromTryGet)
+ tg.buf.printf("tryGet%s: returning %t", tg.name, tg.returnValueFromTryGet)
return tg.returnValueFromTryGet
}
func (tg *testGranter) returnGrant(count int64) {
- tg.buf.printf("returnGrant %d", count)
+ tg.buf.printf("returnGrant%s %d", tg.name, count)
}
func (tg *testGranter) tookWithoutPermission(count int64) {
- tg.buf.printf("tookWithoutPermission %d", count)
+ tg.buf.printf("tookWithoutPermission%s %d", tg.name, count)
}
func (tg *testGranter) continueGrantChain(grantChainID grantChainID) {
- tg.buf.printf("continueGrantChain %d", grantChainID)
+ tg.buf.printf("continueGrantChain%s %d", tg.name, grantChainID)
}
func (tg *testGranter) grant(grantChainID grantChainID) {
rv := tg.r.granted(grantChainID)
@@ -89,13 +90,13 @@ func (tg *testGranter) grant(grantChainID grantChainID) {
// concurrency_manager_test.go.
time.Sleep(50 * time.Millisecond)
}
- tg.buf.printf("granted: returned %d", rv)
+ tg.buf.printf("granted%s: returned %d", tg.name, rv)
}
func (tg *testGranter) storeWriteDone(
originalTokens int64, doneInfo StoreWorkDoneInfo,
) (additionalTokens int64) {
- tg.buf.printf("storeWriteDone: originalTokens %d, doneBytes(write %d,ingested %d) returning %d",
- originalTokens, doneInfo.WriteBytes, doneInfo.IngestedBytes, tg.additionalTokens)
+ tg.buf.printf("storeWriteDone%s: originalTokens %d, doneBytes(write %d,ingested %d) returning %d",
+ tg.name, originalTokens, doneInfo.WriteBytes, doneInfo.IngestedBytes, tg.additionalTokens)
return tg.additionalTokens
}
@@ -448,12 +449,24 @@ func TestPriorityStates(t *testing.T) {
})
}
+func tryScanWorkClass(t *testing.T, d *datadriven.TestData) workClass {
+ wc := regularWorkClass
+ if d.HasArg("elastic") {
+ var b bool
+ d.ScanArgs(t, "elastic", &b)
+ if b {
+ wc = elasticWorkClass
+ }
+ }
+ return wc
+}
+
/*
TestStoreWorkQueueBasic is a datadriven test with the following commands:
init
admit id= tenant= priority= create-time-millis= bypass=
-set-try-get-return-value v=
-granted
+set-try-get-return-value v= [elastic=]
+granted [elastic=]
cancel-work id=
work-done id= [write-bytes=] [ingested-bytes=] [additional-tokens=]
print
@@ -469,14 +482,15 @@ func TestStoreWorkQueueBasic(t *testing.T) {
}
}
defer closeFn()
- var tg *testGranter
+ var tg [numWorkClasses]*testGranter
var wrkMap workMap
var buf builderWithMu
var st *cluster.Settings
printQueue := func() string {
q.mu.Lock()
defer q.mu.Unlock()
- return fmt.Sprintf("%s\nstats:%+v\nestimates:%+v", q.q.String(), q.mu.stats,
+ return fmt.Sprintf("regular workqueue: %s\nelastic workqueue: %s\nstats:%+v\nestimates:%+v",
+ q.q[regularWorkClass].String(), q.q[elasticWorkClass].String(), q.mu.stats,
q.mu.estimates)
}
@@ -485,15 +499,18 @@ func TestStoreWorkQueueBasic(t *testing.T) {
switch d.Cmd {
case "init":
closeFn()
- tg = &testGranter{buf: &buf}
+ tg[regularWorkClass] = &testGranter{name: " regular", buf: &buf}
+ tg[elasticWorkClass] = &testGranter{name: " elastic", buf: &buf}
opts := makeWorkQueueOptions(KVWork)
opts.usesTokens = true
opts.timeSource = timeutil.NewManualTime(timeutil.FromUnixMicros(0))
opts.disableEpochClosingGoroutine = true
st = cluster.MakeTestingClusterSettings()
q = makeStoreWorkQueue(log.MakeTestingAmbientContext(tracing.NewTracer()),
- tg, st, opts).(*StoreWorkQueue)
- tg.r = q
+ [numWorkClasses]granterWithStoreWriteDone{tg[regularWorkClass], tg[elasticWorkClass]},
+ st, opts).(*StoreWorkQueue)
+ tg[regularWorkClass].r = q.getRequesters()[regularWorkClass]
+ tg[elasticWorkClass].r = q.getRequesters()[elasticWorkClass]
wrkMap.resetMap()
return ""
@@ -537,7 +554,8 @@ func TestStoreWorkQueueBasic(t *testing.T) {
case "set-try-get-return-value":
var v bool
d.ScanArgs(t, "v", &v)
- tg.returnValueFromTryGet = v
+ wc := tryScanWorkClass(t, d)
+ tg[wc].returnValueFromTryGet = v
return ""
case "set-store-request-estimates":
@@ -549,7 +567,8 @@ func TestStoreWorkQueueBasic(t *testing.T) {
return printQueue()
case "granted":
- tg.grant(0)
+ wc := tryScanWorkClass(t, d)
+ tg[wc].grant(0)
return buf.stringAndReset()
case "cancel-work":
@@ -589,7 +608,7 @@ func TestStoreWorkQueueBasic(t *testing.T) {
if !work.admitted {
return fmt.Sprintf("id not admitted: %d\n", id)
}
- tg.additionalTokens = int64(additionalTokens)
+ tg[work.handle.workClass].additionalTokens = int64(additionalTokens)
require.NoError(t, q.AdmittedWorkDone(work.handle,
StoreWorkDoneInfo{
WriteBytes: int64(writeBytes),
@@ -607,7 +626,7 @@ func TestStoreWorkQueueBasic(t *testing.T) {
WriteBytes: int64(writeBytes),
IngestedBytes: int64(ingestedBytes),
})
- return printQueue()
+ return buf.stringAndReset()
case "stats-to-ignore":
var ingestedBytes, ingestedIntoL0Bytes int