-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
admission: add support for disk bandwidth as a bottleneck resource #85722
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The first 2 commits are from #85059 and #85127
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @tbg)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @tbg)
pkg/util/admission/disk_bandwidth_test.go
line 11 at r3 (raw file):
// licenses/APL.txt. package admission
I owe a test for the (simple) code in disk_bandwidth.go.
6baa969
to
c0fc3cd
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks good. I am a little lost reviewing these changes once it moves away from the core logic, to be honest. I can always understand (with some work) the denser parts of the PR but then there is always a bunch of nontrivial plumbing and putting things together where I can't really follow any more. Maybe this is just me not spending enough time on the review, being tired, etc, but I am worried that nobody but you can currently make meaningful changes to IO admission control as a whole or really confidently review you. Now is not the time to fix that but it is an area of concern and we should try to involve others in authorship in these parts of the codebase with a focus on improving clarity in the future. For now, a large doc comment and being exhaustive with inline comments is probably the best we can do, I've tried to highlight some areas where I was struggling.
Reviewed 4 of 17 files at r1, 11 of 20 files at r2, 12 of 15 files at r3, 3 of 4 files at r4, 1 of 1 files at r5, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @sumeerbhola)
-- commits
line 200 at r5:
It's now present (unless I'm misreading what this means), just make sure to update before merge.
-- commits
line 211 at r5:
Still true?
pkg/util/admission/disk_bandwidth.go
line 36 at r5 (raw file):
// 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.
If we use direct IO (or whatever option skips the page cache), we do know. Is there a chance this is a useful pattern for compactions? I assume not (if something's in memory, probably better to use it) but wanted to ask.
pkg/util/admission/disk_bandwidth.go
line 40 at r5 (raw file):
// - We don't shape incoming reads. // // - There can be a huge lag between the shaping of incoming writes, and when
Can we quantify "huge"? My intuition says "hours" (based on overloading clusters) and I would like to know if that's right.
pkg/util/admission/disk_bandwidth.go
line 54 at r5 (raw file):
using an enum,
diskLoadLevel
.
pkg/util/admission/disk_bandwidth.go
line 64 at r5 (raw file):
// we start by limiting its application to two kinds of writes: // // - Incoming writes that are deemed "elastic": This can be done by
Make clearer what is done in this PR and what is future work.
pkg/util/admission/disk_bandwidth.go
line 85 at r5 (raw file):
const ( // diskLoadLow implies no need to shape anything. diskLoadLow diskLoadLevel = iota
The I/O admission control code is in general fairly light on metrics (are there any at all other than tokens exhausted duration?). Let's not perpetuate that pattern here but add a IOMetrics
(name tbd) struct and plumbing for it. I think we want kvserver.StoreMetrics
to embed IOMetrics
and pass &storeMetrics.IOMetrics
to makeDiskBandwidthLimiter
, which in turn requires passing it to SetPebbleMetricsProvider
. (Note that *Store
is already backing the IOThresholdConsumer
, so this is almost free).
We want at least the enum in the metrics, and the IOThresholdScore, but I suspect that once we have metrics plumbing we'll end up adding more (like number of regular tokens, elastic tokens, etc, I know we already have some admission control metrics so maybe they're already there). It might even make sense to have all the numbers we print in the log messages for IO overload as metrics, though I'm less sure about that.
I'm ok with this being done as a follow-up, to keep things moving, in which case I can file an issue.
pkg/util/admission/disk_bandwidth.go
line 116 at r5 (raw file):
ctx context.Context, load intervalDiskLoadInfo, storeID int32, ) { d.lastInterval = load
Could you avoid updating the struct in place over the course of the method by moving this to the very end?
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
d.lastInterval = load util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth) log.Infof(logtags.AddTag(ctx, "s", storeID),
Is it appropriate to log on each call?
pkg/util/admission/disk_bandwidth.go
line 131 at r5 (raw file):
// small changes can have outsize influence if a higher number of // compactions start happening. if util < 0.3 {
Do you want to have consts for these numeric values? I'm noticing, for example, that 0.3
shows up twice and it refers to the same constant, and it would be helpful to have that be explicit.
pkg/util/admission/disk_bandwidth.go
line 142 at r5 (raw file):
// moderate. Give it more time at moderate, where we will gradually // increase tokens. d.loadLevel = diskLoadModerate
I have a preference for a local defined before the outermost if
so that we can have, at the end of this method, a complete update of d
in one place (d.loadLevel = loadLevel
). Not a hill I am willing to die on but I consider it an error-prone pattern to incrementally update a struct over multiple lines, I prefer structuring the code such that it is clear that the state is replaced wholesale.
pkg/util/admission/disk_bandwidth.go
line 161 at r5 (raw file):
// lower value of provisioned bandwidth, if they want to further reduce // the probability of hitting the real provisioned bandwidth due to // elastic work.
I was a bit surprised to not see smoothing here. Are you basically finding that waiting out "two intervals" to make more drastic changes is enough "smoothing"? I see that there is some smoothing in the higher layers. (diskBandwidthLimiter
), so it's a bit weird that we're using two different mechanisms for determining the load level vs actually adjusting the tokens based on the load level. If we feel like the token adjustment needs to be smoothed, why doesn't the load level transition (which determines the direction of token adjustment) need to be? Maybe the answer is just "this works and is simple".
pkg/util/admission/disk_bandwidth.go
line 164 at r5 (raw file):
d.loadLevel = diskLoadHigh } else { d.loadLevel = diskLoadOverload
Add a comment that in this regime we (presumably, haven't seen it yet) reduce tokens.
pkg/util/admission/disk_bandwidth.go
line 168 at r5 (raw file):
d.lastUtil = util // TODO(sumeer): Use history of fsync latency and the value in the current // interval to bump up the load level computed earlier based on bandwidth.
What do you mean by bump up? Upgrade a moderate to a high or overload if syncs are slow?
pkg/util/admission/disk_bandwidth.go
line 178 at r5 (raw file):
func (d *diskLoadWatcher) getLoadLevel() (level diskLoadLevel, unusedBandwidth int64) { return d.loadLevel, d.lastInterval.provisionedBandwidth - d.lastInterval.readBandwidth - d.lastInterval.writeBandwidth
Are we ok with this being negative?
pkg/util/admission/disk_bandwidth.go
line 186 at r5 (raw file):
// 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.
Doesn't ignoring this sort of introduce an error of 2x? That is, aren't we consuming twice the tokens for compactions? Maybe this is ok since compactions are a read-write cycle and assuming the read part isn't coming from the cache, they actually consume 2x the bandwidth.
pkg/util/admission/disk_bandwidth.go
line 193 at r5 (raw file):
// regularTokensUsed+elasticTokensUsed do not need to sum up to // incomingBytes, since these stats are produced by different sources. regularTokensUsed int64
Might it be worth naming these {regular,elastic}WriteTokens
to make clear what these are?
pkg/util/admission/disk_bandwidth.go
line 197 at r5 (raw file):
} // diskBandwidthLimiter produces tokens for elastic work.
As an aside, I am wondering if "elastic" is the right adjective for this kind of work. Isn't it more akin to "optional" or "deferrable" work? "Elastic" invokes the concept of supply scaling up to meet demand, but here we are scaling up demand when there is sufficient supply. I can live with the current naming but wanted to bring it up in case it strikes a chord.
pkg/util/admission/disk_bandwidth.go
line 220 at r5 (raw file):
d.diskLoadWatcher.setIntervalInfo(ctx, id, d.storeID) const alpha = 0.5 d.smoothedIncomingBytes = alpha*float64(il.incomingBytes) + (1-alpha)*d.smoothedIncomingBytes
Would you also move this to the pattern where everything goes into locals and then we update d
once, at the end?
pkg/util/admission/disk_bandwidth.go
line 246 at r5 (raw file):
d.elasticTokens = math.MaxInt64 if d.elasticTokens != lastElasticTokens { log.Infof(logtags.AddTag(ctx, "s", d.storeID), "diskBandwidthLimiter low load")
Could you follow a similar pattern that we use in the IO granter, where we populate a struct and then that struct can render itself at the end (and includes the determination of whether something noteworthy happened)?
pkg/util/admission/disk_bandwidth.go
line 250 at r5 (raw file):
// else we stay in the common case of low bandwidth usage. case diskLoadModerate: tokensFullyUtilized := func() bool {
why is this a func rather than tokensFullyUtilized := a || b
?
pkg/util/admission/disk_bandwidth.go
line 253 at r5 (raw file):
// elasticTokens == MaxInt64 is also considered fully utilized since we // can never fully utilize unlimited tokens. return d.elasticTokens == 0 || d.elasticTokens == math.MaxInt64 ||
in which case is the == 0
hit?
pkg/util/admission/disk_bandwidth.go
line 258 at r5 (raw file):
, i.e. roughly increase token count by 10%.
pkg/util/admission/disk_bandwidth.go
line 269 at r5 (raw file):
// 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)
You could omit this line and instead update the computation above to use math.Max(d.smoothedIncomingBytes, float64(il.incomingBytes))
. I think that is more concise and also somewhat more readable since it would avoid the conditional. I would even go all the way:
elasticTokens := int64(math.Max(
(d.smoothedElasticFraction + 0.1) * math.Max(d.smoothedIncomingBytes, float64(il.incomingBytes)),
1.1 * float64(il.elasticTokensUsed),
))
if elasticTokens == 0 {
// Don't get stuck in a situation where the incoming byte counts are all zero.
elasticTokens = math.MaxInt64
}
pkg/util/admission/disk_bandwidth.go
line 311 at r5 (raw file):
if smoothedElasticBytes/2 < d.elasticTokens { d.elasticTokens = smoothedElasticBytes / 2 }
How about
d.elasticTokens := int64(0.5*math.Min(
float64(intElasticBytes)),
d.smoothedElasticFraction * d.smoothedIncomingBytes,
)
pkg/util/admission/doc.go
line 51 at r5 (raw file):
// // TODO(sumeer): update with all the recent changes.
Just adding a comment here to have reviewable help us keep track of this.
pkg/util/admission/granter.go
line 383 at r5 (raw file):
// 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
It would be helpful to break the abstraction (in the comment) and explain why we need a demuxHandle
, i.e. that in practice it is a work class.
pkg/util/admission/granter.go
line 649 at r5 (raw file):
} func (tg *tokenGranter) tryGrantLocked(grantChainID grantChainID) grantResult {
interesting that this code seems to be shared, exactly, with slotGranter
. Rule of three, I know, but still, maybe this is a candidate to pull out into a helper.
pkg/util/admission/granter.go
line 654 at r5 (raw file):
tokens := tg.requester.granted(grantChainID) if tokens == 0 { // Did not accept grant
nit: . here and in the copy.
pkg/util/admission/granter.go
line 681 at r5 (raw file):
// two granter-requester pairs, for the two workClasses. The granter in these // pairs is implemented by kvStoreTokenChildGranter, and the requester by // WorkQueue.
It would be nice to explain how they interact. There is something about elastic requests acquiring from both, but elastic requests returning tokens (???), anyway I'm not really sure how it works from this point out and I don't think it is explained anywhere.
pkg/util/admission/granter.go
line 782 at r5 (raw file):
sg.subtractTokens(-count, false) if wc == elasticWorkClass { sg.elasticDiskBWTokensAvailable += count
interesting, so elastic tokens are returned when the work is done? So they're really more like slots? Feels like there' something to explain in the commit message (or code docs) on how this is all architected. I think the bus factor of all of this code is very close to one at the moment. I see that you have a TODO to update the package docs, looking forward to that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I made through all the code, just have to look at the tests next. Flushing out all my comments (I don't expect more from the tests round, I just need to read them). Like Tobi, this all LGTM (and so do the experimental results), but I'll be a much better reviewer once I'm able to write code in this package myself or try editing some doc text. I understand what this PR is doing but it would've taken me a lot longer to do it myself. Reviewable's acting a little funky so once this is rebased + I look at the tests + another sanity pass, I'll be able to stamp. Tomorrow morning.
Reviewed 3 of 17 files at r1, 11 of 20 files at r2, 5 of 15 files at r3, 2 of 4 files at r4, 1 of 1 files at r5.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif, @sumeerbhola, and @tbg)
-- commits
line 170 at r5:
You mention two previous objects, but there's just one above.
pkg/util/admission/disk_bandwidth.go
line 36 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
If we use direct IO (or whatever option skips the page cache), we do know. Is there a chance this is a useful pattern for compactions? I assume not (if something's in memory, probably better to use it) but wanted to ask.
We could also periodically query the OS for its page cache hit ratios (https://www.brendangregg.com/blog/2014-12-31/linux-page-cache-hit-ratio.html), and use it to approximate what % of reads across compactions + foreground reads hit the disk. From the little I know about eBPF probes and perf, they need root privileges (or overridden paranoia levels: https://unix.stackexchange.com/questions/14227/do-i-need-root-admin-permissions-to-run-userspace-perf-tool-perf-events-ar), so could be a non-starter. We could also compare the outgoing read bandwidth from the CRDB process (so excluding block cache hits) and compare it to disk stats to understand the % of reads served from the page cache. This is all speculative and I'm not sure if we can buy ourselves some leeway in managed deployments.
pkg/util/admission/disk_bandwidth.go
line 50 at r5 (raw file):
// is non-trivial to approach full utilization without risking high latency. // // Due to these challenges, and previous design attempts that were quite
[nit] Drop references to previous design attempts unless we're also spelling out what they were here, and explain why they didn't work and/or were incomplete. It may trip up readers, make them feel like they're missing important context, even though this comment block is pretty sufficient.
pkg/util/admission/disk_bandwidth.go
line 85 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
The I/O admission control code is in general fairly light on metrics (are there any at all other than tokens exhausted duration?). Let's not perpetuate that pattern here but add a
IOMetrics
(name tbd) struct and plumbing for it. I think we wantkvserver.StoreMetrics
to embedIOMetrics
and pass&storeMetrics.IOMetrics
tomakeDiskBandwidthLimiter
, which in turn requires passing it toSetPebbleMetricsProvider
. (Note that*Store
is already backing theIOThresholdConsumer
, so this is almost free).We want at least the enum in the metrics, and the IOThresholdScore, but I suspect that once we have metrics plumbing we'll end up adding more (like number of regular tokens, elastic tokens, etc, I know we already have some admission control metrics so maybe they're already there). It might even make sense to have all the numbers we print in the log messages for IO overload as metrics, though I'm less sure about that.
I'm ok with this being done as a follow-up, to keep things moving, in which case I can file an issue.
There's #82743, can also pick it up if Sumeer doesn't here. (I need to work with this code more directly.)
pkg/util/admission/disk_bandwidth.go
line 117 at r5 (raw file):
) { d.lastInterval = load util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth)
Constraining util
to acceptable bounds here (just incase our stats are incorrect) would reduce some FUD from all the math below.
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
d.lastInterval = load util := float64(load.readBandwidth+load.writeBandwidth) / float64(load.provisionedBandwidth) log.Infof(logtags.AddTag(ctx, "s", storeID),
Ideally this logtag was plumbed down automatically, invoked using a ctx derived from the relevant store's ambient context, and this ticker goroutine done under the some stopper:
cockroach/pkg/util/admission/granter.go
Line 1502 in 7c18668
go func() { |
Not for now, just noting the structure I'd have typically expected.
pkg/util/admission/disk_bandwidth.go
line 153 at r5 (raw file):
// 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
Can we make it less lumpy? By acquiring a gradually increased rate of elastic write tokens over time while guaranteeing some minimum rate (necessary to in turn generate compaction tokens for foreground writes)? I'm not asking for changes in this PR (and what I'm saying probably doesn't make sense) -- but I want to understand what's possible and how to think about this elastic bandwidth use. I also have a general question about how to think about the other resource used during compactions -- CPU. Would you have additional compaction threads acquire both elastic write tokens and elastic CPU tokens? Be subject to shaping from both sides? How do you make sure you're not grabbing the write tokens and then getting queued for the CPU tokens? Maybe I'm asking the same things you were alluding to in your commit message "resource dimensions to the granter-requester interaction but found it too complicated".
pkg/util/admission/disk_bandwidth.go
line 184 at r5 (raw file):
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.
Is the bandwidth use for these side-loaded SSTables not important? (That's what we're talking about here, right?) Or just something we'll look into later on?
pkg/util/admission/disk_bandwidth.go
line 185 at r5 (raw file):
// 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
Still partway through the PR, so perhaps this is clearer later on: but are we needing to capture the bandwidth use as part of the WAL writes? Why or why not (too small a fraction to care about, and we don't care about IOPs, yet)? If we're doing it, where are we doing it? Are the effects of the WAL bandwidth use captured by the linear model fitter we're fitting to observed write bandwidth/writes AC knows about?
pkg/util/admission/disk_bandwidth.go
line 198 at r5 (raw file):
// diskBandwidthLimiter produces tokens for elastic work. type diskBandwidthLimiter struct {
s/diskBandwidthLimiter/elasticDiskbandwidthLimiter? Since it scales up or down the bandwidth used by elastic work.
pkg/util/admission/disk_bandwidth.go
line 246 at r5 (raw file):
d.elasticTokens = math.MaxInt64 if d.elasticTokens != lastElasticTokens { log.Infof(logtags.AddTag(ctx, "s", d.storeID), "diskBandwidthLimiter low load")
Pull a ctx out once with the right logtags instead of initializing one at every log.Infof site.
pkg/util/admission/disk_bandwidth.go
line 257 at r5 (raw file):
} if tokensFullyUtilized() { // Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to
The +10% is not of smoothedIncomingBytes, right? It's +10% over the smoothed elastic fraction.
pkg/util/admission/disk_bandwidth.go
line 279 at r5 (raw file):
if d.elasticTokens == 0 { // Don't get stuck in a situation where smoothedIncomingBytes are 0. d.elasticTokens = math.MaxInt64
Can we push some of this defensive validation out of the main control loop? Or just, at the very top?
pkg/util/admission/granter.go
line 383 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
It would be helpful to break the abstraction (in the comment) and explain why we need a
demuxHandle
, i.e. that in practice it is a work class.
How do you feel about making this explicitly in terms of work classes instead? Granters that don't need to distinguish between multiple classes ("elastic work doesn't apply) can ignore it, granters that do can consider it. If in the future we need the opaqueness this handle can provide (segmenting along something other than work classes), we can add it.
pkg/util/admission/granter.go
line 782 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
interesting, so elastic tokens are returned when the work is done? So they're really more like slots? Feels like there' something to explain in the commit message (or code docs) on how this is all architected. I think the bus factor of all of this code is very close to one at the moment. I see that you have a TODO to update the package docs, looking forward to that.
There's the sg.subtractTokens(-count)
above; we're returning availableIOTokens for both regular and elastic work, but since elastic work additionally takes from elasticDiskBWTokensAvailable, we have to special case this here.
pkg/util/admission/granter.go
line 910 at r5 (raw file):
wasExhausted := exhaustedFunc() actualTokens := int64(float64(doneInfo.WriteBytes)*sg.writeLM.multiplier) + sg.writeLM.constant +
Worth pulling into a method on the linear model type? We do this math in several places.
pkg/util/admission/granter.go
line 2195 at r5 (raw file):
// totalNumByteTokens represents the tokens to give out until the next call to // adjustTokens. They are parceled out in small intervals. byteTokensAllocated // represents what has been given out.
Mention that this is cumulative (ditto for elasticDiskBWTokensAllocated below).
pkg/util/admission/granter.go
line 2241 at r5 (raw file):
// 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
I'm struggling with this sentence, I don't entirely understand the distinction between "incoming bytes into the LSM" that do consume these tokens, and "compactions" (grouped as part of "every write to the disk"). Aren't compactions going to consume "elastic disk bandwidth tokens"? Or are you trying to say that compactions don't consume the "flush/compaction tokens" we generate for foreground writes to not overwhelm the LSM/cause memtable stalls?
pkg/util/admission/granter.go
line 2378 at r5 (raw file):
// allocateTokensTick gives out 1/ticksInAdjustmentInterval of the // various okens every 250ms.
Type: okens.
pkg/util/admission/granter.go
line 2427 at r5 (raw file):
readBandwidth: int64((diskStats.BytesRead - prevCumBytesRead) / adjustmentInterval), writeBandwidth: int64((diskStats.BytesWritten - prevCumBytesWritten) / adjustmentInterval), provisionedBandwidth: diskStats.ProvisionedBandwidth,
General question: how are operators going to opt-in to elastic bandwidth shaping, i.e. how is this provisioned bandwidth going to be provided?
pkg/util/admission/store_token_estimation.go
line 298 at r5 (raw file):
e.atDoneIngestTokensLinearModel.updateModelUsingIntervalStats( intIngestedAccountedBytes, adjustedIntL0IngestedBytes, intWorkCount) // Disk bandwidth model
[nit] Missing period.
pkg/util/admission/store_token_estimation.go
line 302 at r5 (raw file):
int64(e.cumLSMWriteAndIngestedBytes) adjustedIntLSMWriteAndIngestedBytes := intLSMWriteAndIngestedBytes - (int64(admissionStats.statsToIgnore.Bytes) - int64(e.cumStoreAdmissionStats.statsToIgnore.Bytes))
The anonymous struct in statsToIgnore makes this harder to read (as opposed to admissionStats.statsToIgnore.Bytes.IngestOperationStats.Bytes
). Introducing a variable for int64(admissionStats.statsToIgnore.Bytes) - int64(e.cumStoreAdmissionStats.statsToIgnore.Bytes)
will also help. It's intIgnoredIngestedBytes or something, right?
pkg/util/admission/work_queue.go
line 1595 at r5 (raw file):
) (handle StoreWorkHandle, err error) { // For now, we compute a workClass based on priority. wc := regularWorkClass
Could you explain the reason behind the segmenting of work queues by priority in a bit more detail? Or point me to somewhere you already have. I guess it makes sense, we don't want regular work to be queued behind elastic work, but here -- wouldn't that be taken care of by priority levels already? Are you saying that later on we'll be able to plumb in the work class directly as part of StoreWriteWorkInfo or similar?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM.
Reviewed 1 of 17 files at r1, 7 of 15 files at r3, 1 of 4 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif, @sumeerbhola, and @tbg)
pkg/util/admission/testdata/granter
line 381 at r3 (raw file):
---- GrantCoordinator: (chain: id: 0 active: false index: 5) io-avail: 153722867280912930, elastic-disk-bw-tokens-avail: 153722867280912930
[nit] A set-elastic-disk-bw-tokens tokens=1000
(ditto for io-avail) or something early on will make changes to those values easier to identify when skimming through this test file.
pkg/util/admission/testdata/granter
line 554 at r3 (raw file):
(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
Print out the underlying models through some directive?
pkg/util/admission/testdata/io_load_listener
line 484 at r5 (raw file):
---- 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:<nil>}
I'll echo something from an earlier review -- having this test only print out some diff of changes would make these, to me, easier to read and understand. After looking at the internals I know what fields to look out for, but if just the diffs were rendered, I could use these tests to actually play around with the code and see how things evolve.
pkg/util/admission/testdata/store_per_work_token_estimator
line 36 at r5 (raw file):
# 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.
Add a sentence here explaining the other-levels-ingested and how it affects the disk-bw-tokens linear model.
01b72a4
to
5670269
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TFTRs!
I am still working through the comments. Just flushing the ones I have addressed. I will ping when all are addressed.
Also, this PR is now rebased on master.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif, @sumeerbhola, and @tbg)
Previously, irfansharif (irfan sharif) wrote…
You mention two previous objects, but there's just one above.
Done
Previously, tbg (Tobias Grieger) wrote…
It's now present (unless I'm misreading what this means), just make sure to update before merge.
The plumbing is missing, i.e., DiskStats are not populated yet.
Previously, tbg (Tobias Grieger) wrote…
Still true?
Yes, still true.
pkg/util/admission/disk_bandwidth.go
line 36 at r5 (raw file):
If we use direct IO (or whatever option skips the page cache), we do know. Is there a chance this is a useful pattern for compactions? I assume not (if something's in memory, probably better to use it) but wanted to ask.
I am not sure if it is a useful pattern for compactions. I have seen some kv0 runs where the read bandwidth is close to 0 because compaction reads are getting hits in the write-back page cache. If it turns out to be useful for other reasons, we would of course incorporate it here, but I don't think admission control would motivate that change.
We could also periodically query the OS for its page cache hit ratios (https://www.brendangregg.com/blog/2014-12-31/linux-page-cache-hit-ratio.html), and use it to approximate what % of reads across compactions + foreground reads hit the disk.
We can approximate this even without querying the OS. We know how many bytes Pebble read from the filesystem (that missed its block cache) and we know how many reads were issued due to misses in the page cache, so we know the hit bytes in the page cache. The problem here is that we don't split the reads that missed the page cache into two categories: reads due to compactions and reads due to user-facing Pebble reads.
pkg/util/admission/disk_bandwidth.go
line 40 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Can we quantify "huge"? My intuition says "hours" (based on overloading clusters) and I would like to know if that's right.
This was a poorly phrased code comment. I was thinking huge initially, and then experiments indicated that it is large enough to extend across multiple 15s intervals. I've changed it to say 1+min. I've observed instances like: we cut down on elastic tokens and then the next 3 or so intervals of 15s each show gradual declines in bandwidth, and then there is a steep fall.
pkg/util/admission/disk_bandwidth.go
line 50 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
[nit] Drop references to previous design attempts unless we're also spelling out what they were here, and explain why they didn't work and/or were incomplete. It may trip up readers, make them feel like they're missing important context, even though this comment block is pretty sufficient.
Good point. They didn't make it past my handwritten notes, so I've removed this mention of previous design attempts.
pkg/util/admission/disk_bandwidth.go
line 54 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
using an enum,
diskLoadLevel
.
Done
pkg/util/admission/disk_bandwidth.go
line 64 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Make clearer what is done in this PR and what is future work.
Done
pkg/util/admission/disk_bandwidth.go
line 85 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
There's #82743, can also pick it up if Sumeer doesn't here. (I need to work with this code more directly.)
@irfansharif thanks for volunteering! I've added a TODO where GranterMetrics
is declared. I think as part of this we should cleanup how this metrics plumbing works, since currently it all starts in NewGrantCoordinators
which calls makeGranterMetrics
which is premature for per-store metrics.
// 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.
pkg/util/admission/disk_bandwidth.go
line 116 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Could you avoid updating the struct in place over the course of the method by moving this to the very end?
Done. btw, I have mixed feelings about this pattern. It's good when the data-structure is a "real" struct and all fields are being updated, since it gives assurance that everything was updated. But in other cases introducing these local variables seems more prone to overlooking something to update.
pkg/util/admission/disk_bandwidth.go
line 117 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Constraining
util
to acceptable bounds here (just incase our stats are incorrect) would reduce some FUD from all the math below.
I think we need to leave it unconstrained since someone could set a provisioned bandwidth to be much lower than what we can actually utilize, because they want to keep headroom or some other reason. Knowing that the utilization is 2.1 (and not 1.0) is useful to know.
There is some commentary in the logic below about this.
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Ideally this logtag was plumbed down automatically, invoked using a ctx derived from the relevant store's ambient context, and this ticker goroutine done under the some stopper:
cockroach/pkg/util/admission/granter.go
Line 1502 in 7c18668
go func() { Not for now, just noting the structure I'd have typically expected.
I've removed these calls by moving the addtag
to the beginning of ioLoadListener.pebbleMetricsTick
.
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Is it appropriate to log on each call?
I've removed this and folded this into the logging done by diskBandwidthLimiter.computeElasticTokens
. That is a bit less verbose since it only logs at diskLoadLow
if the number of elastic tokens have changed.
In general, I am somewhat inclined to log more from the ioLoadListener
and supporting structs, since it happens only once every 15s. Our current logic to avoid logging in ioLoadListener
is a bit annoying when looking at logs during experiments since one often has an overload interval followed by one or two intervals of underload (with no logs) and then overload again, and it is not possible to figure out what happened with work size modeling and number of admitted requests and total byte size during those underloaded intervals (which would actually be useful to know).
pkg/util/admission/disk_bandwidth.go
line 131 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Do you want to have consts for these numeric values? I'm noticing, for example, that
0.3
shows up twice and it refers to the same constant, and it would be helpful to have that be explicit.
Done
pkg/util/admission/disk_bandwidth.go
line 142 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I have a preference for a local defined before the outermost
if
so that we can have, at the end of this method, a complete update ofd
in one place (d.loadLevel = loadLevel
). Not a hill I am willing to die on but I consider it an error-prone pattern to incrementally update a struct over multiple lines, I prefer structuring the code such that it is clear that the state is replaced wholesale.
Done
pkg/util/admission/disk_bandwidth.go
line 153 at r5 (raw file):
We may be able to make it less lumpy, and that may mean reintroducing the compaction pacing stuff in Pebble that we ripped out. It is complicated by the fact that we don't know the read bandwidth usage of a running compaction.
Would you have additional compaction threads acquire both elastic write tokens and elastic CPU tokens?
Yes. Just like the additional compression threads for existing compactions have to ask for soft slots.
How do you make sure you're not grabbing the write tokens and then getting queued for the CPU tokens?
For low volume background work like compactions one can poll, at say 1s intervals, and not queue.
Maybe I'm asking the same things you were alluding to in your commit message "resource dimensions to the granter-requester interaction but found it too complicated".
Perhaps, though I don't think we want to unify all resources into one with a single WorkQueue. The issue you brought up is CPU and IO as separate resources and we do maintain separate queues for those and have write requests queue twice, first for writes and then for cpu. So we already have the risk that once they get the write tokens they will block on a cpu slot. Given that we are adjusting write tokens at 15s granularity and most of the work is wanting to complete with lower latency, maybe this is fine. I don't have good ideas on how to address this issue.
pkg/util/admission/disk_bandwidth.go
line 161 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I was a bit surprised to not see smoothing here. Are you basically finding that waiting out "two intervals" to make more drastic changes is enough "smoothing"? I see that there is some smoothing in the higher layers. (
diskBandwidthLimiter
), so it's a bit weird that we're using two different mechanisms for determining the load level vs actually adjusting the tokens based on the load level. If we feel like the token adjustment needs to be smoothed, why doesn't the load level transition (which determines the direction of token adjustment) need to be? Maybe the answer is just "this works and is simple".
Yes, this is mostly "this works and is simple". The diskLoadWatcher
is doing very coarse smoothing and spitting out a diskLoadLevel
enum. I wanted some modularity in now wanting diskBandwidthLimiter
also tracking the history of diskLoadLevel
s.
Regarding why diskLoadWatcher
doesn't smooth the load more, (a) 15s is a long time interval, so we are seeing some smoothing just due to that, (b) we do want to react very quickly to higher utilization.
Again there is a bunch of subjectivity here. I expect that over time we will fully rewrite this file.
pkg/util/admission/disk_bandwidth.go
line 164 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Add a comment that in this regime we (presumably, haven't seen it yet) reduce tokens.
Done
pkg/util/admission/disk_bandwidth.go
line 168 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
What do you mean by bump up? Upgrade a moderate to a high or overload if syncs are slow?
Correct. I've rephrased the comment.
pkg/util/admission/disk_bandwidth.go
line 178 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Are we ok with this being negative?
Yes, it's used for logging. I added a comment in the caller.
5670269
to
b02940e
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All comments addressed (or added TODOs for the future). I owe tests for the code in disk_bandwidth.go -- will ping when it is ready.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @tbg)
pkg/util/admission/disk_bandwidth.go
line 184 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Is the bandwidth use for these side-loaded SSTables not important? (That's what we're talking about here, right?) Or just something we'll look into later on?
It is important. What this is trying to note is why we just work with the sum of ingested and flushed bytes and do not treat them separately. What we may look into later on is actually distinguishing them since ingests don't incur WAL writes. Such refinement will be needed if we want to start getting closer to 100% utilization of disk bandwidth. I've rephrased the comment
// Flushed bytes + Ingested bytes seen by the LSM. Ingested bytes incur the
// cost of writing a sstable, even though that is done outside Pebble, so
// ingestion is similar in cost to flushing. Ingested bytes don't cause WAL
// writes, but we ignore that difference for simplicity, and just work with
// the sum of flushed and ingested bytes.
pkg/util/admission/disk_bandwidth.go
line 185 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Still partway through the PR, so perhaps this is clearer later on: but are we needing to capture the bandwidth use as part of the WAL writes? Why or why not (too small a fraction to care about, and we don't care about IOPs, yet)? If we're doing it, where are we doing it? Are the effects of the WAL bandwidth use captured by the linear model fitter we're fitting to observed write bandwidth/writes AC knows about?
We are capturing all the bandwidth in the DiskStats
, including the WAL writes. We are not capturing IOPS at all yet.
What we are not doing is saying that the cost of 1 byte of regular write is different from 1 byte of ingest, i.e., we are treating the tokens as interchangeable. One could argue that the cost is different because regular writes are also written to the WAL. But if we start making such distinctions we have a bit of a slippery slope in that then we should also differentiate based on future cost incurred in the system, so we would also need to know which level each ingest is going into.
pkg/util/admission/disk_bandwidth.go
line 186 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Doesn't ignoring this sort of introduce an error of 2x? That is, aren't we consuming twice the tokens for compactions? Maybe this is ok since compactions are a read-write cycle and assuming the read part isn't coming from the cache, they actually consume 2x the bandwidth.
It can introduce a 2x error for ingests. But see my response to the comment by @irfansharif -- each token here in some sense is about its immediate and future cost and estimating the future cost is hard (ingests can go into many levels).
Compactions won't consume these tokens. The tokens here are only for incoming writes to the LSM. Compactions don't have a future cost. What we want compactions to do is to consume directly some part of the disk bandwidth (in the prototype this was done coarsely via adjusting the number of compaction slots, though one can imagine more granular bandwidth tokens
type compactionLimiter struct { |
Eventually, the code in granter should stop calling these disk bandwidth tokens since these are just tokens for all incoming writes into the LSM (this is already noted in the "misnomer" code comments -- I want to delay that refactor given the time pressure).
pkg/util/admission/disk_bandwidth.go
line 193 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Might it be worth naming these
{regular,elastic}WriteTokens
to make clear what these are?
We have a naming problem that needs some cleanup. I'd like to do that later.
We have writes versus ingests, where the former are flowing through memtable and get flushed. We distinguish them in many cases (though not in this file). We have tokens for bytes "incoming into L0" and tokens for bytes "incoming into the LSM". The latter is the tokens here. Then we have resource constraints that result in these token values. Compactions out of L0 and flushes into L0 affect the "incoming into L0" tokens. Disk bandwidth is what affects the "incoming into the LSM" tokens, though only for elastic traffic. Disk bandwidth may also be used to later compute "disk bandwidth" tokens for doing extra compactions.
I don't want to rush the renaming, because I think it will need reviewing walltime.
pkg/util/admission/disk_bandwidth.go
line 197 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
As an aside, I am wondering if "elastic" is the right adjective for this kind of work. Isn't it more akin to "optional" or "deferrable" work? "Elastic" invokes the concept of supply scaling up to meet demand, but here we are scaling up demand when there is sufficient supply. I can live with the current naming but wanted to bring it up in case it strikes a chord.
I'm using/abusing it in the classic networking sense of "elastic traffic". I don't have a good reference, but the following (copy-pasted from a web search) serves as a good summary: "We observe that elastic traffic does not have an intrinsic temporal behavior, and can be transported at arbitrary transfer rates. Transfer delay and delay variability can be tolerated. An elastic transfer can be performed over a wide range of transfer rates, and the rate can even vary over the duration of the transfer."
pkg/util/admission/disk_bandwidth.go
line 198 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
s/diskBandwidthLimiter/elasticDiskbandwidthLimiter? Since it scales up or down the bandwidth used by elastic work.
See my earlier response to @tbg about naming. That said, I think this will still be the diskBandwidthLimiter
since that is its objective -- to protect disk bandwidth consumption. To achieve its objective it will produce tokensForAllIncomingElasticWritesToTheLSM
and compactionDiskBandwidthTokens
. These are not interchangeable tokens since the former has future cost (and we can pretend that we are paying that multiplier now, since what we admitted in the past is charging us that cost now).
pkg/util/admission/disk_bandwidth.go
line 220 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Would you also move this to the pattern where everything goes into locals and then we update
d
once, at the end?
diskBandwidthLimiter
is not a pure-struct -- it has an embedded diskLoadWatcher
. And there is an if-block here that does not update d.smoothedElasticFraction
, so to follow that pattern we need to first copy out the member variable then optionally update it, and then reassign at the end. It doesn't seem a clear improvement from what is here. I'll change it if you really need me to.
pkg/util/admission/disk_bandwidth.go
line 246 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Pull a ctx out once with the right logtags instead of initializing one at every log.Infof site.
Done
pkg/util/admission/disk_bandwidth.go
line 246 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Could you follow a similar pattern that we use in the IO granter, where we populate a struct and then that struct can render itself at the end (and includes the determination of whether something noteworthy happened)?
btw, the ioLoadListener
pattern with adjustTokensResult
is creaking and needs to be revisited.
In this particular case (a) we are printing state in these log statements that does not get stored in diskBandwidthLimiter
(like unusedBW
), (b) these log statements are rather short, (c) I found it clearer to see why we are printing something in the context of the preceding computation logic.
So I've left this alone for now, but added a TODO
// TODO(sumeer): look into extracting the state being logged, so that it is
// a union across all the cases, and move the logging into one spot at the
// end.
pkg/util/admission/disk_bandwidth.go
line 250 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
why is this a func rather than
tokensFullyUtilized := a || b
?
Forgot to cleanup after some code iteration. Done.
pkg/util/admission/disk_bandwidth.go
line 253 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
in which case is the
== 0
hit?
Good catch. This was leftover from code iteration. Removed.
pkg/util/admission/disk_bandwidth.go
line 257 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
The +10% is not of smoothedIncomingBytes, right? It's +10% over the smoothed elastic fraction.
It is over all incoming bytes. One can expand this to
elasticBytes := d.smoothedElasticFraction*d.smoothedIncomingBytes + 0.1*d.smoothedIncomingBytes
The first term is what we are already giving to elastic traffic. The second term is 10% of all incoming bytes.
I've also changed the comment here, based on @tbg's suggestion below.
pkg/util/admission/disk_bandwidth.go
line 258 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
, i.e. roughly increase token count by 10%.
Changed to
// Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to
// elastic work. That is, we are increasing the total incoming bytes by
// 10% (not just the elastic bytes by 10%).
pkg/util/admission/disk_bandwidth.go
line 269 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
You could omit this line and instead update the computation above to use
math.Max(d.smoothedIncomingBytes, float64(il.incomingBytes))
. I think that is more concise and also somewhat more readable since it would avoid the conditional. I would even go all the way:elasticTokens := int64(math.Max( (d.smoothedElasticFraction + 0.1) * math.Max(d.smoothedIncomingBytes, float64(il.incomingBytes)), 1.1 * float64(il.elasticTokensUsed), )) if elasticTokens == 0 { // Don't get stuck in a situation where the incoming byte counts are all zero. elasticTokens = math.MaxInt64 }
Done
pkg/util/admission/disk_bandwidth.go
line 279 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Can we push some of this defensive validation out of the main control loop? Or just, at the very top?
This 0 value is a result of the computation done here, so I don't see how it can be moved out. We don't want to generally change 0 to math.MaxInt64 -- we are doing this only because we are in the diskLoadModerate
case.
pkg/util/admission/disk_bandwidth.go
line 311 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
How about
d.elasticTokens := int64(0.5*math.Min( float64(intElasticBytes)), d.smoothedElasticFraction * d.smoothedIncomingBytes, )
Done (I've developed bad code habits due to golang, instead of chaining things in c++ with std::min,max and the conditional operator).
pkg/util/admission/doc.go
line 51 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Just adding a comment here to have reviewable help us keep track of this.
Ack. Planning to do this in a later PR. This PR is not the only to blame for staleness here.
pkg/util/admission/granter.go
line 383 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
How do you feel about making this explicitly in terms of work classes instead? Granters that don't need to distinguish between multiple classes ("elastic work doesn't apply) can ignore it, granters that do can consider it. If in the future we need the opaqueness this handle can provide (segmenting along something other than work classes), we can add it.
Done. I've explicitly said in the code comment that, when used, it is a workClass.
I feel a bit odd about passing a fake workClass to the GrantCoordinator, so I haven't made that change, but I am likely overthinking this.
pkg/util/admission/granter.go
line 649 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
interesting that this code seems to be shared, exactly, with
slotGranter
. Rule of three, I know, but still, maybe this is a candidate to pull out into a helper.
Yes, it is indeed a copy. Templates would make sharing easier -- we are accessing both methods and member variables here.
pkg/util/admission/granter.go
line 654 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
nit: . here and in the copy.
Done
pkg/util/admission/granter.go
line 681 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
It would be nice to explain how they interact. There is something about elastic requests acquiring from both, but elastic requests returning tokens (???), anyway I'm not really sure how it works from this point out and I don't think it is explained anywhere.
Done
pkg/util/admission/granter.go
line 782 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
There's the
sg.subtractTokens(-count)
above; we're returning availableIOTokens for both regular and elastic work, but since elastic work additionally takes from elasticDiskBWTokensAvailable, we have to special case this here.
It's as @irfansharif said. I've added a couple of code comments.
pkg/util/admission/granter.go
line 910 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Worth pulling into a method on the linear model type? We do this math in several places.
Good point. Done.
pkg/util/admission/granter.go
line 2195 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Mention that this is cumulative (ditto for elasticDiskBWTokensAllocated below).
We are using the term cumulative for stats that are "from the beginning of time" (or more practically from server start). These are just the tokens to give out over the next 15s interval.
pkg/util/admission/granter.go
line 2241 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
I'm struggling with this sentence, I don't entirely understand the distinction between "incoming bytes into the LSM" that do consume these tokens, and "compactions" (grouped as part of "every write to the disk"). Aren't compactions going to consume "elastic disk bandwidth tokens"? Or are you trying to say that compactions don't consume the "flush/compaction tokens" we generate for foreground writes to not overwhelm the LSM/cause memtable stalls?
Incoming bytes into the LSM are from user-facing operations: writes and ingests. Compactions are not going to consume these tokens. These tokens have their own life in that we increase and decrease based on what the disk bandwidth load level looks like, but they are not in terms of disk bandwidth, since the incoming bytes do not account for the unknown write amplification that will occur. In contrast, compactions are more exact in what they are doing (module our lack of visibility into how much they are benefiting from the page cache) -- they will consume write disk bandwidth proportional to the size of the compaction.
I've said something similar in response to a comment on disk_bandwidth.go -- I realize this can be quite confusing, so am very open to ideas on how to explain this better.
pkg/util/admission/granter.go
line 2378 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Type: okens.
Done.
pkg/util/admission/granter.go
line 2427 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
General question: how are operators going to opt-in to elastic bandwidth shaping, i.e. how is this provisioned bandwidth going to be provided?
My current plan is to add a cluster setting for provisioned bandwidth that applies cluster wide on each store, with a per store override via flag/env variables.
pkg/util/admission/store_token_estimation.go
line 298 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
[nit] Missing period.
Done.
pkg/util/admission/store_token_estimation.go
line 302 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
The anonymous struct in statsToIgnore makes this harder to read (as opposed to
admissionStats.statsToIgnore.Bytes.IngestOperationStats.Bytes
). Introducing a variable forint64(admissionStats.statsToIgnore.Bytes) - int64(e.cumStoreAdmissionStats.statsToIgnore.Bytes)
will also help. It's intIgnoredIngestedBytes or something, right?
Done
pkg/util/admission/work_queue.go
line 1595 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Could you explain the reason behind the segmenting of work queues by priority in a bit more detail? Or point me to somewhere you already have. I guess it makes sense, we don't want regular work to be queued behind elastic work, but here -- wouldn't that be taken care of by priority levels already? Are you saying that later on we'll be able to plumb in the work class directly as part of StoreWriteWorkInfo or similar?
Yes, later we may want to plumb the work class explicitly so it may not be a function of priority. But more importantly, the issue is that priority is not the first factor in ordering -- inter-tenant fairness takes precedence over intra-tenant priority. Which means tenant1 that has used fewer tokens can have its elastic work queued ahead of tenant2's regular work (if they shared the same WorkQueue). So if elastic tokens are exhausted, tenant1's work will block tenant2 from making progress. I have added a comment where kvStoreTokenGranter
is declared.
pkg/util/admission/testdata/granter
line 381 at r3 (raw file):
Previously, irfansharif (irfan sharif) wrote…
[nit] A
set-elastic-disk-bw-tokens tokens=1000
(ditto for io-avail) or something early on will make changes to those values easier to identify when skimming through this test file.
Done
pkg/util/admission/testdata/granter
line 554 at r3 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Print out the underlying models through some directive?
I made a note here that this test hard codes the models.
pkg/util/admission/testdata/io_load_listener
line 484 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
I'll echo something from an earlier review -- having this test only print out some diff of changes would make these, to me, easier to read and understand. After looking at the internals I know what fields to look out for, but if just the diffs were rendered, I could use these tests to actually play around with the code and see how things evolve.
I've added a todo
// TODO(sumeer): the output printed by set-state is hard to follow. It
// prints the internal fields which are hard to interpret, and it prints
// a properly formatted ioLoadListenerState. The latter is supposed to
// be easier to understand, but reviewers have noted that it is still
// challenging to understand whether the output is correct. Come up with
// more easily consumable output. Additionally, the input uses
// cumulative values, so one has to look at the preceding testdata -- we
// could instead accept the interval delta as input.
pkg/util/admission/testdata/store_per_work_token_estimator
line 36 at r5 (raw file):
Previously, irfansharif (irfan sharif) wrote…
Add a sentence here explaining the other-levels-ingested and how it affects the disk-bw-tokens linear model.
Done
b02940e
to
502cbe1
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added some simple testing for the logic in disk_bandwidth.go.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @irfansharif and @tbg)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Was sort of hesitant to approve since there are just a few too many things that are still foggy in my mind (see inline comments). But I am fairly certain that what will result from my comments is just more explanations, and there is no reason to hold up the PR for that. Besides, if I had chanced on anything, it would still be better to have merged this and to then change it.
Reviewed 2 of 20 files at r6, 5 of 6 files at r8, 4 of 4 files at r9, all commit messages.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @irfansharif, @sumeerbhola, and @tbg)
pkg/util/admission/disk_bandwidth.go
line 36 at r5 (raw file):
and we know how many reads were issued due to misses in the page cache
... because the number of page faults is included in rusage
(field ru_majflt
)?
I think we can somehow finagle our way to more causality here, too. If we make pebble use "extra threads" (i.e. goroutines that we start at process start and then pin to the OS thread) for compactions, then we can get the page faults for that thread, which will then reflect only those for the pebble compactions. We can then subtract that from the general number of page faults, giving those due to foreground traffic.
Anyway, this isn't for now, but it's an interesting problem to solve at some point.
pkg/util/admission/disk_bandwidth.go
line 116 at r5 (raw file):
Previously, sumeerbhola wrote…
Done. btw, I have mixed feelings about this pattern. It's good when the data-structure is a "real" struct and all fields are being updated, since it gives assurance that everything was updated. But in other cases introducing these local variables seems more prone to overlooking something to update.
Thanks! For my understanding, was this instance for a "real" struct? I agree that if we're not wholesale replacing the struct there is a potential problem. Though I still think that the ideal solution is to group the things we update in a wrapped struct which we can then update at once (in a way that forces anything we forget to reset to zero). Whenever there is a set of variables that are interconnected and replaced wholesale, I am worried that by doing it piecemeal we'll carry over pieces of old computation into the new state by accident, which is much harder to spot than a perpetually zeroed field. Also, as a reviewer I find the increased amount of aliasing that tends to occur with in-place edits (is this operation using the new or old value? You have to backtrack and check) tricky.
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
Previously, sumeerbhola wrote…
I've removed this and folded this into the logging done by
diskBandwidthLimiter.computeElasticTokens
. That is a bit less verbose since it only logs atdiskLoadLow
if the number of elastic tokens have changed.
In general, I am somewhat inclined to log more from theioLoadListener
and supporting structs, since it happens only once every 15s. Our current logic to avoid logging inioLoadListener
is a bit annoying when looking at logs during experiments since one often has an overload interval followed by one or two intervals of underload (with no logs) and then overload again, and it is not possible to figure out what happened with work size modeling and number of admitted requests and total byte size during those underloaded intervals (which would actually be useful to know).
I think it would be fine to do something where we log for the next 10 minutes after an overload was hit. But I do feel rather strongly that if nothing is overloaded and hasn't been for hours, that the log messages shouldn't be there. But I also understand that now is not the time to go on lengthy tangents to rework logging; if it is better to log every 15s for now then so be it as far as I'm concerned.
pkg/util/admission/disk_bandwidth.go
line 220 at r5 (raw file):
Previously, sumeerbhola wrote…
diskBandwidthLimiter
is not a pure-struct -- it has an embeddeddiskLoadWatcher
. And there is an if-block here that does not updated.smoothedElasticFraction
, so to follow that pattern we need to first copy out the member variable then optionally update it, and then reassign at the end. It doesn't seem a clear improvement from what is here. I'll change it if you really need me to.
I'm fine leaving as is, though after this small diff it should be straightforward:
diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go
index 276c3ddd94..1971666d2e 100644
--- a/pkg/util/admission/disk_bandwidth.go
+++ b/pkg/util/admission/disk_bandwidth.go
@@ -222,10 +222,7 @@ type intervalLSMInfo struct {
elasticTokensUsed int64
}
-// diskBandwidthLimiter produces tokens for elastic work.
-type diskBandwidthLimiter struct {
- diskLoadWatcher diskLoadWatcher
-
+type diskBandwidthLimiterState struct {
smoothedIncomingBytes float64
smoothedElasticFraction float64
elasticTokens int64
@@ -233,6 +230,12 @@ type diskBandwidthLimiter struct {
prevElasticTokensUsed int64
}
+// diskBandwidthLimiter produces tokens for elastic work.
+type diskBandwidthLimiter struct {
+ diskLoadWatcher diskLoadWatcher
+ diskBandwidthLimiterState
+}
+
func makeDiskBandwidthLimiter() diskBandwidthLimiter {
return diskBandwidthLimiter{
elasticTokens: math.MaxInt64,
pkg/util/admission/disk_bandwidth.go
line 59 at r9 (raw file):
// - 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).
Is it even additive increase? I might be getting this wrong, but when we increase the tokens, we add 10% of the incoming bytes, and we do so only if the tokens were mostly exhausted, so presumably the incoming bytes is ~ the number of elastic tokens we had in the first place. So aren't we multiplying the tokens by 1.1 roughly?
pkg/util/admission/disk_bandwidth.go
line 150 at r9 (raw file):
// are trying to carefully narrow this case since not limiting tokens can // blow up bandwidth. if d.loadLevel <= diskLoadModerate && util < d.lastUtil+smallDelta &&
I was trying to explain to myself why it makes sense to "stop limiting tokens" (vs leaving them were they ended up) and wasn't quite sure, have some words for a comment here?
pkg/util/admission/disk_bandwidth.go
line 268 at r9 (raw file):
// start happening, or the compaction backlog is cleared. // // TODO(sumeer): experiment with a PID controller.
Just curious, PID?
pkg/util/admission/disk_bandwidth.go
line 287 at r9 (raw file):
if tokensFullyUtilized { // Smoothed elastic bytes plus 10% of smoothedIncomingBytes is given to // elastic work. That is, we are increasing the total incoming bytes by
Aren't we increasing the elastic tokens by 10% of the total incoming (not just elastic) bytes?
pkg/util/admission/disk_bandwidth.go
line 301 at r9 (raw file):
intBasedElasticTokens := (d.smoothedElasticFraction + 0.1) * float64(il.incomingBytes) elasticBytes = math.Max(elasticBytes, intBasedElasticTokens) elasticBytes = math.Max(elasticBytes, 1.1*float64(il.elasticTokensUsed))
Per my earlier comment about whether this is additive increase, isn't this basically bumping the tokens with a factor 1.1, since elasticTokensUsed should be pretty close to the old tokens here, and the left-hand side becomes the new tokens? My impression is that we're increasing by a factor of 1.1, and decreasing by a factor of 0.5.
pkg/util/admission/disk_bandwidth.go
line 336 at r9 (raw file):
ib := humanizeutil.IBytes level := d.diskLoadWatcher.getLoadLevel() p.Printf("diskBandwidthLimiter %s (%v): elastic-fr: %.2f, incoming: %s, "+
Maybe elastic-frac
? That auto-completes the elastic fraction for me whereas I didn't initially know what elastic-fr
would've meant.
pkg/util/admission/granter.go
line 782 at r5 (raw file):
Previously, sumeerbhola wrote…
It's as @irfansharif said. I've added a couple of code comments.
Thanks! Now I'm just more confused though, IO tokens aren't returned, right? Is this code only called when tokens were acquired but not used, for example when an operation didn't actually go ahead and execute?
pkg/util/admission/granter.go
line 714 at r9 (raw file):
// compactions out of L0 and flushes into L0) and (b) elastic disk bandwidth // tokens, which are based on disk bandwidth as a constrained resource, and // apply to all the elastic incoming bytes into th LSM.
the
pkg/util/admission/store_token_estimation.go
line 24 at r9 (raw file):
I think some of this might help future readers 💡 a bit faster than I managed to:
We use a model in which the "actual tokens" are computes as a linear function of the bytes written,
i.e. actual_tokens = a*written_bytes + b, and the result of this computation (which can only be done after completion
of the work) is used to acquire (without blocking) additional tokens. The model thus tries to make sure that one token reflects one byte of work.For example, if the model is initially
(a=1, b=0)
and each incoming request acquires 1000 tokens but ends up writing 2000 bytes, the model should update to roughly(a=2, b=0)
, and future requests will, upon completion, acquire an additional 1000 tokens to even the score. The actual fitting is performed on aggregates over recent requests, and the more work is done "outside" of admission control, the less useful the multiplier becomes; the model will degrade into one with a larger constant term and is expected to perform poorly.
pkg/util/admission/store_token_estimation.go
line 259 at r9 (raw file):
atDoneWriteTokensLinearModel tokensLinearModelFitter atDoneIngestTokensLinearModel tokensLinearModelFitter // Unlike the models above that model bytes into L0, for disk bandwidth
I am not sure what this comment is telling me. So if this file shouldn't ... do any adjustments, why is there a linear model fitter? Am pretty lost on how these pieces fit together. Is it explained anywhere?
I understand that the elastic tokens get scaled up over time in an attempt to mostly, but not fully utilize the disk, but where does the fitter come in? I see we update it in this file.
Is this comment just trying to say that the code that updates elastic tokens is in disk_bandwidth.go (and called from this file via computeElasticTokens
)?
pkg/util/admission/store_token_estimation.go
line 301 at r9 (raw file):
func makeStorePerWorkTokenEstimator() storePerWorkTokenEstimator { return storePerWorkTokenEstimator{
Don't the initial constants for these fitters warrant some explanation?
pkg/util/admission/testdata/store_per_work_token_estimator
line 37 at r9 (raw file):
# This means a mix of regular writes and sstable ingests (say index # backfills), will not effect the cost attributed to regular writes. The # ingest model will be fit based on accounted bytes of 40000, and actual bytes
4000
Also, the fitters coerce a
into a some range, and that range is motivated by the L0-specific tokens, are there any problems here now? For example
// - For ingests, we expect the multiplier a to be <= 1, since some fraction
// of the ingest goes into L0. So it would be reasonable to constrain a to
// [0, 1]. For the same reason as the previous bullet, we use [0.001, 1.5].
// This lower-bound of 0.001 is debatable, since it will cause some token
// consumption even if all ingested bytes are going to levels below L0.
// TODO(sumeer): consider lowering the lower bound, after experimentation.
no longer applies to the disk bandwidth token ingestions, where we care about the entire write, not just the L0 fraction. So at the very least it is difficult to be sure that the constants are right since the comments don't explore the elastic use case.
502cbe1
to
5249222
Compare
We assume that: - There is a provisioned known limit on the sum of read and write bandwidth. This limit is allowed to change. - 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 time lag 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 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, and previous design attempts that were quite complicated (and incomplete), we adopt a goal of simplicity of design, and strong abstraction boundaries. - The disk load is abstracted using an enum. The diskLoadWatcher can be evolved independently. - The approach uses easy to understand small multiplicative increase and large 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: - 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). This prototype does the latter. - 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 cockroachdb/pebble/issues/1329 for motivation. This control on compactions is not currently implemented and is future work (though the prototype in cockroachdb#82813 had code for it). The reader should start with disk_bandwidth.go, consisting of - diskLoadWatcher: which computes load levels. - diskBandwidthLimiter: It used the load level computed by diskLoadWatcher to limit write tokens for elastic writes and in the future will also limit compactions. There is significant refactoring and changes in granter.go and work_queue.go. This is driven by the fact that: - Previously the tokens were for L0 and now we need to support tokens for bytes into L0 and tokens for bytes into the LSM (the former being a subset of the latter). - Elastic work is in a different WorkQueue than regular work, but they are competing for the same tokens. A different WorkQueue is needed to prevent a situation where elastic work for one tenant is queued ahead of regualar work from another tenant, and stops the latter from making progress due to lack of elastic tokens. The latter is handled by allowing kvSlotGranter to multiplex across multiple requesters, via multiple child granters. A number of interfaces are adjusted to make this viable. In general, the GrantCoordinator is now slightly dumber and some of that logic is moved into the granters. For the former (handling two kinds of tokens), I considered adding multiple resource dimensions to the granter-requester interaction but found it too complicated. Instead we rely on the observation that we request tokens based on the total incoming bytes of the request (not just L0), and when the request is completed, tell the granter how many bytes went into L0. The latter allows us to return tokens to L0. So at the time the request is completed, we can account separately for the L0 tokens and these new tokens for all incoming bytes (which we are calling disk bandwidth tokens, since they are constrained based on disk bandwidth). This is a cleaned up version of the prototype in cockroachdb#82813 which contains the experimental results. The plumbing from the KV layer to populate the disk reads, writes and provisioned bandwidth is absent in this PR, and will be added in a subsequent PR. Disk bandwidth bottlenecks are considered only if both the following are true: - DiskStats.ProvisionedBandwidth is non-zero. - The cluster setting admission.disk_bandwidth_tokens.elastic.enabled is true (defaults to true). Informs cockroachdb#82898 Release note: None (the cluster setting mentioned earlier is useless since the integration with CockroachDB will be in a future PR).
5249222
to
88ee320
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the reviews again!
I plan to merge this once CI is green. I am happy to address any new comments in a followup PR.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @irfansharif and @tbg)
pkg/util/admission/disk_bandwidth.go
line 36 at r5 (raw file):
and we know how many reads were issued due to misses in the page cache
... because the number of page faults is included in rusage (field ru_majflt)?
(a) I misspoke, I did not mean to suggest the number of reads -- I meant to say bytes read, (b) yes, the counts are also available, in the libraries we use to implement the logic in getDiskCounters
, (c) I haven't looked at the C code backing these libraries, though maybe not rusage since it doesn't have read/write byte numbers (maybe iostat).
I think we can somehow finagle our way to more causality here, too. ...
good ideas.
pkg/util/admission/disk_bandwidth.go
line 116 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Thanks! For my understanding, was this instance for a "real" struct? I agree that if we're not wholesale replacing the struct there is a potential problem. Though I still think that the ideal solution is to group the things we update in a wrapped struct which we can then update at once (in a way that forces anything we forget to reset to zero). Whenever there is a set of variables that are interconnected and replaced wholesale, I am worried that by doing it piecemeal we'll carry over pieces of old computation into the new state by accident, which is much harder to spot than a perpetually zeroed field. Also, as a reviewer I find the increased amount of aliasing that tends to occur with in-place edits (is this operation using the new or old value? You have to backtrack and check) tricky.
Yes, a "real" struct :)
Good points -- they make a lot of sense.
pkg/util/admission/disk_bandwidth.go
line 118 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I think it would be fine to do something where we log for the next 10 minutes after an overload was hit. But I do feel rather strongly that if nothing is overloaded and hasn't been for hours, that the log messages shouldn't be there. But I also understand that now is not the time to go on lengthy tangents to rework logging; if it is better to log every 15s for now then so be it as far as I'm concerned.
Acknowledged. We need a helper that will track history and we can ask it whether to log and tell it whether we think this log event is interesting by itself. Is that a pattern that we have seen a need for outside the admission package too?
pkg/util/admission/disk_bandwidth.go
line 220 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I'm fine leaving as is, though after this small diff it should be straightforward:
diff --git a/pkg/util/admission/disk_bandwidth.go b/pkg/util/admission/disk_bandwidth.go index 276c3ddd94..1971666d2e 100644 --- a/pkg/util/admission/disk_bandwidth.go +++ b/pkg/util/admission/disk_bandwidth.go @@ -222,10 +222,7 @@ type intervalLSMInfo struct { elasticTokensUsed int64 } -// diskBandwidthLimiter produces tokens for elastic work. -type diskBandwidthLimiter struct { - diskLoadWatcher diskLoadWatcher - +type diskBandwidthLimiterState struct { smoothedIncomingBytes float64 smoothedElasticFraction float64 elasticTokens int64 @@ -233,6 +230,12 @@ type diskBandwidthLimiter struct { prevElasticTokensUsed int64 } +// diskBandwidthLimiter produces tokens for elastic work. +type diskBandwidthLimiter struct { + diskLoadWatcher diskLoadWatcher + diskBandwidthLimiterState +} + func makeDiskBandwidthLimiter() diskBandwidthLimiter { return diskBandwidthLimiter{ elasticTokens: math.MaxInt64,
Done
pkg/util/admission/disk_bandwidth.go
line 59 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Is it even additive increase? I might be getting this wrong, but when we increase the tokens, we add 10% of the incoming bytes, and we do so only if the tokens were mostly exhausted, so presumably the incoming bytes is ~ the number of elastic tokens we had in the first place. So aren't we multiplying the tokens by 1.1 roughly?
You are right! I don't know what I was thinking when I wrote that comment. Fixed in this file and in the commit/PR description.
pkg/util/admission/disk_bandwidth.go
line 150 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I was trying to explain to myself why it makes sense to "stop limiting tokens" (vs leaving them were they ended up) and wasn't quite sure, have some words for a comment here?
There are tradeoffs here, and this is what I went with initially since it was simple. I've added a comment
// An alternative would be to never have unlimited tokens, since that
// ensures there is always some reasonable bound in place. It may mean
// that the initial tokens are insufficient and the tokens catch up to
// what is needed with some lag, and during that time there is unnecessary
// queueing. This downside could be avoided by ramping up faster. This
// alternative is worth investigating.
pkg/util/admission/disk_bandwidth.go
line 268 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Just curious, PID?
Changed to
// TODO(sumeer): experiment with a more sophisticated controller for the
// elastic token adjustment, e.g. a PID (Proportional-Integral-Derivative)
// controller.
pkg/util/admission/disk_bandwidth.go
line 287 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Aren't we increasing the elastic tokens by 10% of the total incoming (not just elastic) bytes?
Correct. I think we are saying the same thing. The comment says "increasing the total incoming bytes by 10%", since it assumes that the regular traffic is stable. Maybe it is the confusion regarding bytes versus tokens. Each token is 1 byte. I've added a sentence to the comment.
pkg/util/admission/disk_bandwidth.go
line 301 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Per my earlier comment about whether this is additive increase, isn't this basically bumping the tokens with a factor 1.1, since elasticTokensUsed should be pretty close to the old tokens here, and the left-hand side becomes the new tokens? My impression is that we're increasing by a factor of 1.1, and decreasing by a factor of 0.5.
Yes, this was silly commenting on my part. Fixed.
pkg/util/admission/disk_bandwidth.go
line 336 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Maybe
elastic-frac
? That auto-completes the elastic fraction for me whereas I didn't initially know whatelastic-fr
would've meant.
Done
pkg/util/admission/granter.go
line 782 at r5 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Thanks! Now I'm just more confused though, IO tokens aren't returned, right? Is this code only called when tokens were acquired but not used, for example when an operation didn't actually go ahead and execute?
Correct, IO tokens are not returned in general, and the only estimation fixing that may result in returning tokens happens via storeWriteDone
and not this path. This path is indeed only for returning when the work did not execute. granter.returnGrant
has the following comment (I've removed a bullet there that is probably the source of confusion).
// returnGrant is called for:
// - returning slots after use.
// - returning either slots or tokens when the grant raced with the work
// being canceled, and the grantee did not end up doing any work.
//
// The last case occurs despite the return value on the requester.granted
// method -- it is possible that the work was not canceled at the time when
// requester.grant was called, and hence returned a count > 0, but later
// 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.
pkg/util/admission/granter.go
line 714 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
the
Done
pkg/util/admission/store_token_estimation.go
line 24 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I think some of this might help future readers 💡 a bit faster than I managed to:
We use a model in which the "actual tokens" are computes as a linear function of the bytes written,
i.e. actual_tokens = a*written_bytes + b, and the result of this computation (which can only be done after completion
of the work) is used to acquire (without blocking) additional tokens. The model thus tries to make sure that one token reflects one byte of work.For example, if the model is initially
(a=1, b=0)
and each incoming request acquires 1000 tokens but ends up writing 2000 bytes, the model should update to roughly(a=2, b=0)
, and future requests will, upon completion, acquire an additional 1000 tokens to even the score. The actual fitting is performed on aggregates over recent requests, and the more work is done "outside" of admission control, the less useful the multiplier becomes; the model will degrade into one with a larger constant term and is expected to perform poorly.
Thanks. I've used this text and rewritten what was already here since some of the lack of integration stuff was stale and it did not talk about the diskBWLM.
pkg/util/admission/store_token_estimation.go
line 259 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
I am not sure what this comment is telling me. So if this file shouldn't ... do any adjustments, why is there a linear model fitter? Am pretty lost on how these pieces fit together. Is it explained anywhere?
I understand that the elastic tokens get scaled up over time in an attempt to mostly, but not fully utilize the disk, but where does the fitter come in? I see we update it in this file.
Is this comment just trying to say that the code that updates elastic tokens is in disk_bandwidth.go (and called from this file via
computeElasticTokens
)?
I cut out the part about compactions. We need a linear model for the same reason we need it for writes and ingests -- what we see coming into the LSM is different from what is claimed. Earlier our models were only for what we saw coming into L0 -- separate model for writes and ingests. Now we need a model for all the incoming bytes into the LSM (note, not compactions). The difference between what we had and what we need to consume the elastic tokens is only a model that accounts for ingested bytes across levels. Then we can compose the existing write model and this all-levels-ingested model for computing the elastic tokens.
But I was stupid and did not see this earlier and instead of doing this model composition had a separate model that was handling both writes and ingests across the LSM. Such a model is not going to be very workload agnostic since we are mixing writes and ingests into the same model. I realized this this morning, when rewriting the comment at the top of store_token_estimation.go. I've now gone ahead and made this change.
pkg/util/admission/store_token_estimation.go
line 301 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Don't the initial constants for these fitters warrant some explanation?
This was in the comment at the top of the file, but it is easy to overlook. I've changed these to refer to consts declared above, and the const declaration has a reference to the comment.
pkg/util/admission/testdata/store_per_work_token_estimator
line 37 at r9 (raw file):
Previously, tbg (Tobias Grieger) wrote…
4000
Also, the fitters coerce
a
into a some range, and that range is motivated by the L0-specific tokens, are there any problems here now? For example// - For ingests, we expect the multiplier a to be <= 1, since some fraction // of the ingest goes into L0. So it would be reasonable to constrain a to // [0, 1]. For the same reason as the previous bullet, we use [0.001, 1.5]. // This lower-bound of 0.001 is debatable, since it will cause some token // consumption even if all ingested bytes are going to levels below L0. // TODO(sumeer): consider lowering the lower bound, after experimentation.
no longer applies to the disk bandwidth token ingestions, where we care about the entire write, not just the L0 fraction. So at the very least it is difficult to be sure that the constants are right since the comments don't explore the elastic use case.
Done: typo fixed; the comments at the top of the file now talk about this elastic use case, including justification for the constants.
bors r=tbg,irfansharif |
Build failed (retrying...): |
Build succeeded: |
A previous PR cockroachdb#85722 added support for disk bandwidth as a bottlneck resource, in the admission control package. To utilize this, admission control needs to be provided the provisioned bandwidth and the observed read and write bytes. This PR adds configuration support for this via the StoreSpec (that uses the --store flag). The StoreSpec now has an optional ProvisionedRateSpec that contains the name of the disk corresponding to the store, and an optional provisioned bandwidth, that are specified as provisioned-rate=name=<disk-name>[:bandwidth=<bandwidth-bytes>]. The disk-name is used to map the DiskStats, retrieved via the existing code in status.GetDiskCounters to the correct Store. These DiskStats contain the read and write bytes. The optional bandwidth is used to override the provisioned bandwidth set via the new cluster setting kv.store.admission.provisioned_bandwidth. Release note (ops change): Disk bandwidth constraint can now be used to control admission of elastic writes. This requires configuration for each store, via the --store flag, that now contains a provisioned-rate field. The provisioned-rate field needs to provide a disk-name for the store and optionally a disk bandwidth. If the disk bandwidth is not provided the cluster setting kv.store.admission.provisioned_bandwidth will be used. The cluster setting defaults to 0. If the effective disk bandwidth, i.e., using the possibly overridden cluster setting is 0, there is no disk bandwidth constraint. Additionally, the admission control cluster setting admission.disk_bandwidth_tokens.elastic.enabled (defaults to true) can be used to turn off enforcement even when all the other configuration has been setup. Turning off enforcement will still output all the relevant information about disk bandwidth usage, so can be used to observe part of the mechanism in action. Fixes cockroachdb#82898
A previous PR cockroachdb#85722 added support for disk bandwidth as a bottlneck resource, in the admission control package. To utilize this, admission control needs to be provided the provisioned bandwidth and the observed read and write bytes. This PR adds configuration support for this via the StoreSpec (that uses the --store flag). The StoreSpec now has an optional ProvisionedRateSpec that contains the name of the disk corresponding to the store, and an optional provisioned bandwidth, that are specified as provisioned-rate=name=<disk-name>[:bandwidth=<bandwidth-bytes>]. The disk-name is used to map the DiskStats, retrieved via the existing code in status.GetDiskCounters to the correct Store. These DiskStats contain the read and write bytes. The optional bandwidth is used to override the provisioned bandwidth set via the new cluster setting kv.store.admission.provisioned_bandwidth. Fixes cockroachdb#82898 Release note (ops change): Disk bandwidth constraint can now be used to control admission of elastic writes. This requires configuration for each store, via the --store flag, that now contains an optional provisioned-rate field. The provisioned-rate field, if specified, needs to provide a disk-name for the store and optionally a disk bandwidth. If the disk bandwidth is not provided the cluster setting kv.store.admission.provisioned_bandwidth will be used. The cluster setting defaults to 0 (which means that the disk bandwidth constraint is disabled). If the effective disk bandwidth, i.e., using the possibly overridden cluster setting is 0, there is no disk bandwidth constraint. Additionally, the admission control cluster setting admission.disk_bandwidth_tokens.elastic.enabled (defaults to true) can be used to turn off enforcement even when all the other configuration has been setup. Turning off enforcement will still output all the relevant information about disk bandwidth usage, so can be used to observe part of the mechanism in action. To summarize, to enable this for a cluster with homogenous disk, provide a disk-name in the provisioned-rate field in the store-spec, and set the kv.store.admission.provisioned_bandwidth cluster setting to the bandwidth limit. To only get information about disk bandwidth usage by elastic traffic (currently via logs, not metrics), do the above but also set admission.disk_bandwidth_tokens.elastic.enabled to false. Release justification: Low risk, high benefit change that allows an operator to enable new functionality (disabled by default).
85392: streamingccl: support sst event in random stream client r=gh-casper a=gh-casper Previously random stream client lacks test coverage for AddSSTable operation, this PR enables random stream client to generate and validate random SSTableEvents. This PR also cleans up the InterceptableStreamClient to avoid unnecessary abstraction, making code cleaner. Release justification: low risk, high benefit changes to existing functionality Release note : None 86063: base,kvserver,server: configuration of provisioned bandwidth for a store r=irfansharif a=sumeerbhola A previous PR #85722 added support for disk bandwidth as a bottlneck resource, in the admission control package. To utilize this, admission control needs to be provided the provisioned bandwidth and the observed read and write bytes. This PR adds configuration support for this via the StoreSpec (that uses the --store flag). The StoreSpec now has an optional ProvisionedRateSpec that contains the name of the disk corresponding to the store, and an optional provisioned bandwidth, that are specified as provisioned-rate=name=<disk-name>[:bandwidth=<bandwidth-bytes>]. The disk-name is used to map the DiskStats, retrieved via the existing code in status.GetDiskCounters to the correct Store. These DiskStats contain the read and write bytes. The optional bandwidth is used to override the provisioned bandwidth set via the new cluster setting kv.store.admission.provisioned_bandwidth. Fixes #82898 Release note (ops change): Disk bandwidth constraint can now be used to control admission of elastic writes. This requires configuration for each store, via the --store flag, that now contains an optional provisioned-rate field. The provisioned-rate field, if specified, needs to provide a disk-name for the store and optionally a disk bandwidth. If the disk bandwidth is not provided the cluster setting kv.store.admission.provisioned_bandwidth will be used. The cluster setting defaults to 0 (which means that the disk bandwidth constraint is disabled). If the effective disk bandwidth, i.e., using the possibly overridden cluster setting is 0, there is no disk bandwidth constraint. Additionally, the admission control cluster setting admission.disk_bandwidth_tokens.elastic.enabled (defaults to true) can be used to turn off enforcement even when all the other configuration has been setup. Turning off enforcement will still output all the relevant information about disk bandwidth usage, so can be used to observe part of the mechanism in action. To summarize, to enable this for a cluster with homogenous disk, provide a disk-name in the provisioned-rate field in the store-spec, and set the kv.store.admission.provisioned_bandwidth cluster setting to the bandwidth limit. To only get information about disk bandwidth usage by elastic traffic (currently via logs, not metrics), do the above but also set admission.disk_bandwidth_tokens.elastic.enabled to false. Release justification: Low risk, high benefit change that allows an operator to enable new functionality (disabled by default). 86558: spanconfigtestutils: copy gc job testing knobs to tenant r=ajwerner a=stevendanna Some of the sqltranslator tests require that we can disable the GC job related to an index creation so that we can observe the span configuration on the temporary indexes. We pause the GC job via a testing knob. Fixes #85507 Release justification: Test-only change Release note: None 86596: ui/cluster-ui: create statements insights api r=xinhaoz a=xinhaoz This commit adds a function to the `cluster-ui` pkg that queries `crdb_internal.cluster_execution_insights` to surface slow running queries. The information retrieved is intended to be used in the insights statement overview and details pages. A new field `statementInsights` is added to the `cachedData` object in the db-console redux store, and the corresponding function to issue the data fetch is also added in `apiReducers`. This commit does not add any reducers or sagas to CC to fetch and store this data. Release justification: non-production code change Release note: None 86684: release: update cockroachdb version to v22.1.6 r=rhu713 a=rail Release justification: not a part of the build Release note: None Co-authored-by: Casper <casper@cockroachlabs.com> Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com> Co-authored-by: Steven Danna <danna@cockroachlabs.com> Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com> Co-authored-by: Rail Aliiev <rail@iqchoice.com>
We assume that:
bandwidth. This limit is allowed to change.
also cause reads, since compactions do reads and writes.
There are multiple challenges:
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.
it affects actual writes in the system, since compaction backlog can
build up in various levels of the LSM store.
queues that can build up due to resource overload. For instance,
different examples of bandwidth saturation exhibit 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, and previous design attempts that were quite
complicated (and incomplete), we adopt a goal of simplicity of design, and strong
abstraction boundaries.
evolved independently.
large 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:
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). This prototype does the latter.
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
db: automatically tune compaction concurrency based on available CPU/disk headroom and read-amp pebble#1329 for motivation. This control on compactions
is not currently implemented and is future work (though the prototype
in [WIP] admission: add support for disk bandwidth as a bottleneck resource #82813 had code for
it).
The reader should start with disk_bandwidth.go, consisting of
to limit write tokens for elastic writes and in the future will also
limit compactions.
There is significant refactoring and changes in granter.go and
work_queue.go. This is driven by the fact that:
bytes into L0 and tokens for bytes into the LSM (the former being a subset
of the latter).
are competing for the same tokens. A different WorkQueue is needed to
prevent a situation where elastic work for one tenant is queued ahead
of regualar work from another tenant, and stops the latter from making
progress due to lack of elastic tokens.
The latter is handled by allowing kvSlotGranter to multiplex across
multiple requesters, via multiple child granters. A number of interfaces
are adjusted to make this viable. In general, the GrantCoordinator
is now slightly dumber and some of that logic is moved into the granters.
For the former (handling two kinds of tokens), I considered adding multiple
resource dimensions to the granter-requester interaction but found it
too complicated. Instead we rely on the observation that we request
tokens based on the total incoming bytes of the request (not just L0),
and when the request is completed, tell the granter how many bytes
went into L0. The latter allows us to return tokens to L0. So at the
time the request is completed, we can account separately for the L0
tokens and these new tokens for all incoming bytes (which we are calling
disk bandwidth tokens, since they are constrained based on disk bandwidth).
This is a cleaned up version of the prototype in
#82813 which contains the
experimental results. The plumbing from the KV layer to populate the
disk reads, writes and provisioned bandwidth is absent in this PR,
and will be added in a subsequent PR.
Disk bandwidth bottlenecks are considered only if both the following
are true:
is true (defaults to true).
Informs #82898
Release note: None (the cluster setting mentioned earlier is useless
since the integration with CockroachDB will be in a future PR).