-
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: error accounting for disk reads and writes #133310
admission: error accounting for disk reads and writes #133310
Conversation
3537d77
to
4cdddda
Compare
1ab57ec
to
80a17cd
Compare
80a17cd
to
bf20f9e
Compare
bf20f9e
to
3c6d66a
Compare
4942021
to
815c33a
Compare
Changes in |
adaac96
to
75ac944
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.
sorry for the delay
Reviewed 4 of 10 files at r2, 1 of 2 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aadityasondhi and @itsbilal)
pkg/util/admission/granter.go
line 306 at r3 (raw file):
diskTokensAvailable diskTokens diskTokensError struct { prevObservedWrites uint64
Can you add a comment
// prevObserved{Writes,Reads} is the observed disk metrics in the last call to adjustDiskTokenErrorLocked. These are used to compute the delta.
pkg/util/admission/granter.go
line 309 at r3 (raw file):
prevObservedReads uint64 diskWriteTokensAlreadyDeducted int64 diskReadTokensAlreadyDeducted int64
I realize this will change in the future, but I think it is important to document what these are now.
// diskWriteTokensAlreadyDeducted are the deductions we have seen over the last error adjustment interval.
// diskReadTokensAlreadyDeducted are the expected deductions over an error adjustment interval. We compute the expected deductions once every 15s and then dole them out in setAvailableTokens.
I think there is a problem here. Say we observed R reads over the last 15s interval. The elasticDiskReadTokensCapacity
will be R/60. But our error adjustment interval is 1s. So even if there is no error, the observed reads will be R/15 and we will deduct more. The easiest fix would be to make the error adjustment interval equal to 250ms so that these align. And write a detailed code comment on why we are aligning these.
Also, there may be a bit of misalignment risk with the shouldAdjustForError()
logic working independently of the ticks. If the error interval is 250ms and it fires at 3ms and 253ms, while the diskReadTokensAlreadyDeducted
was also being updated at the unloadedDuration
of 250ms and got written at 2ms and 254ms, then the second error adjustment will find diskReadTokensAlreadyDeducted
equal to zero. I don't think the write tokens suffer from this problem since the deductions are live deductions. We may need to rework the tokenAllocatorTicker
logic to introduce coordination.
pkg/util/admission/granter.go
line 522 at r3 (raw file):
if writeError > 0 { if sg.coordMu.diskTokensAvailable.writeByteTokens < 0 { sg.coordMu.diskTokensAvailable.writeByteTokens = 0
I suspect this is motivated by https://docs.google.com/document/d/1KelFCIUd9jaBkAev5G6CReI\_kA-u\_pv6CHHNjjD4-lk/edit?tab=t.0#bookmark=id.czsnhcjsmyia
I suspect the approach mentioned in that doc is not quite right. It is possible that writeByteTokens
are negative because of a huge write and not due to previous error correction. This needs some more thought.
pkg/util/admission/granter.go
line 538 at r3 (raw file):
if sg.coordMu.diskTokensAvailable.writeByteTokens < 0 { sg.coordMu.diskTokensAvailable.writeByteTokens = 0 }
why this resetting to 0 after subtracting the readError
?
pkg/util/admission/io_load_listener.go
line 249 at r3 (raw file):
totalNumElasticByteTokens int64 elasticByteTokensAllocated int64
I think we should add a comment stating that even though these are prefixed with elastic, these are tokens for both regular and elastic work. These are only prefixed with elastic as only elastic work waits for them. Technically, elastic reads don't wait for elasticDiskReadTokens either, so perhaps we should just drop the elastic prefix on these fields.
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 review. I haven't adressed anything, just want to discuss the synchronization thing before I make changes.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/granter.go
line 309 at r3 (raw file):
Previously, sumeerbhola wrote…
I realize this will change in the future, but I think it is important to document what these are now.
// diskWriteTokensAlreadyDeducted are the deductions we have seen over the last error adjustment interval.
// diskReadTokensAlreadyDeducted are the expected deductions over an error adjustment interval. We compute the expected deductions once every 15s and then dole them out in setAvailableTokens.
I think there is a problem here. Say we observed R reads over the last 15s interval. The
elasticDiskReadTokensCapacity
will be R/60. But our error adjustment interval is 1s. So even if there is no error, the observed reads will be R/15 and we will deduct more. The easiest fix would be to make the error adjustment interval equal to 250ms so that these align. And write a detailed code comment on why we are aligning these.Also, there may be a bit of misalignment risk with the
shouldAdjustForError()
logic working independently of the ticks. If the error interval is 250ms and it fires at 3ms and 253ms, while thediskReadTokensAlreadyDeducted
was also being updated at theunloadedDuration
of 250ms and got written at 2ms and 254ms, then the second error adjustment will finddiskReadTokensAlreadyDeducted
equal to zero. I don't think the write tokens suffer from this problem since the deductions are live deductions. We may need to rework thetokenAllocatorTicker
logic to introduce coordination.
Re: The error coordination issue you brought up. I was initially debating the synchronization vs no synchronization thing. I landed on this because I think it works fine (but I see a bug now after rethinking). Here is my thought processes. And let's disregard what the code currently does since I think there is a minor issue there.
- let
I
represent the allocation interval, letJ
represent the error accounting interval - let's assume we have decided to deduct 100*60 (6000B) for this interval.
I0 = 0
I1 = 100
I2 = 200
I3 = 300
I4 = 400
J1 => we find 300B read from disk over the interval. We subtract and set to 0. We don't deduct additional tokens since 300 < 400.
I5 = 100
I6 = 200
I7 = 300
I8 = 400
J2 => we find 600B read from disk over the interval. We subtract and set to 0. We deduct 200 additional tokens.
The issue in my code being I currently don't let diskReadTokensAlreadyDeducted
go above the hypothetical 100B here on every allocation tick. If I remove that limitation, I think this should work. The issue is in pkg/util/admission/granter.go:703
:
sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted += elasticDiskReadTokens
if sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted > elasticDiskReadTokensCapacity {
sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted = elasticDiskReadTokensCapacity
}
With the setup described above, I think regardless on when J ticks, we should still be accurately adjusting the error. It basically works the same way as writes do, but the deduction (which happens at admission for writes) happens during token allocation, and doesn't need to be limited to the capacity
i.e. we keep allocating until the error accounting tick adjusts for the observed error.
I could be missing something and it might be helpful to discuss with a whiteboard in-person.
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 @aadityasondhi and @itsbilal)
pkg/util/admission/granter.go
line 522 at r3 (raw file):
Previously, sumeerbhola wrote…
I suspect this is motivated by https://docs.google.com/document/d/1KelFCIUd9jaBkAev5G6CReI\_kA-u\_pv6CHHNjjD4-lk/edit?tab=t.0#bookmark=id.czsnhcjsmyia
I suspect the approach mentioned in that doc is not quite right. It is possible that
writeByteTokens
are negative because of a huge write and not due to previous error correction. This needs some more thought.
I was thinking a bit more about this, and trying to abstract out the problem. Assume for a moment that the error correction interval is 250ms. So we have 60 such errors to forecast. Let's call these e_f(0), ..., e_f(59). There is also the actual observed error for an interval at e_o(0), ..., e_0(59). We have to forecast e_f(i) and then 250ms later we observe e_o(i).
Roughly what we want is to:
minimize |e_o(i)-e_f(i)|
for all i.
Additionally, we want e_f(i) >= 0
(assuming we are subtracting the error from tokens), since we want to err on the side of having slightly fewer tokens than to give out more tokens.
What we are currently doing is setting e_f(i) = max(0, e_o(i-1))
. And for e_f(0), using the last observed error from the previous 60 ticks.
The idea from that doc was to additionally correct previous mistakes. In some sense trying to reduce |sum_i e_o(i) - sum_i e_f(i)|
. This can be in opposition to the previously stated minimization goal. And the hack suggested in the doc isn't quite the right way to correct either. So I am leaning towards thinking that we shouldn't bother with this at all. That is, just subtract e_f(i) and don't care if the accumulation of error forecast have made tokens too few. But we should use the lastTick
logic in setAvailableTokens
to also adjust diskTokensAvailable.writeByteTokens
, so that every 15s we restore it to 0.
Just to summarize what we discussed in-person re: the error accounting scheme.
|
75ac944
to
c48782e
Compare
Your pull request contains more than 1000 changes. It is strongly encouraged to split big PRs into smaller chunks. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf. |
de75793
to
e390d83
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.
Resolved your outstanding comments. And changed the implementation to reflect what we discussed in person. This should ready for a final review.
PTAL @sumeerbhola
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/granter.go
line 306 at r3 (raw file):
Previously, sumeerbhola wrote…
Can you add a comment
// prevObserved{Writes,Reads} is the observed disk metrics in the last call to adjustDiskTokenErrorLocked. These are used to compute the delta.
Done.
pkg/util/admission/granter.go
line 309 at r3 (raw file):
Previously, aadityasondhi (Aaditya Sondhi) wrote…
Re: The error coordination issue you brought up. I was initially debating the synchronization vs no synchronization thing. I landed on this because I think it works fine (but I see a bug now after rethinking). Here is my thought processes. And let's disregard what the code currently does since I think there is a minor issue there.
- let
I
represent the allocation interval, letJ
represent the error accounting interval- let's assume we have decided to deduct 100*60 (6000B) for this interval.
I0 = 0
I1 = 100
I2 = 200
I3 = 300
I4 = 400
J1 => we find 300B read from disk over the interval. We subtract and set to 0. We don't deduct additional tokens since 300 < 400.
I5 = 100
I6 = 200
I7 = 300
I8 = 400
J2 => we find 600B read from disk over the interval. We subtract and set to 0. We deduct 200 additional tokens.The issue in my code being I currently don't let
diskReadTokensAlreadyDeducted
go above the hypothetical 100B here on every allocation tick. If I remove that limitation, I think this should work. The issue is inpkg/util/admission/granter.go:703
:sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted += elasticDiskReadTokens if sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted > elasticDiskReadTokensCapacity { sg.coordMu.diskTokensError.diskReadTokensAlreadyDeducted = elasticDiskReadTokensCapacity }
With the setup described above, I think regardless on when J ticks, we should still be accurately adjusting the error. It basically works the same way as writes do, but the deduction (which happens at admission for writes) happens during token allocation, and doesn't need to be limited to the
capacity
i.e. we keep allocating until the error accounting tick adjusts for the observed error.I could be missing something and it might be helpful to discuss with a whiteboard in-person.
Changed implementation to reflect what we discussed.
pkg/util/admission/granter.go
line 522 at r3 (raw file):
Previously, sumeerbhola wrote…
I was thinking a bit more about this, and trying to abstract out the problem. Assume for a moment that the error correction interval is 250ms. So we have 60 such errors to forecast. Let's call these e_f(0), ..., e_f(59). There is also the actual observed error for an interval at e_o(0), ..., e_0(59). We have to forecast e_f(i) and then 250ms later we observe e_o(i).
Roughly what we want is to:
minimize|e_o(i)-e_f(i)|
for all i.
Additionally, we wante_f(i) >= 0
(assuming we are subtracting the error from tokens), since we want to err on the side of having slightly fewer tokens than to give out more tokens.What we are currently doing is setting
e_f(i) = max(0, e_o(i-1))
. And for e_f(0), using the last observed error from the previous 60 ticks.The idea from that doc was to additionally correct previous mistakes. In some sense trying to reduce
|sum_i e_o(i) - sum_i e_f(i)|
. This can be in opposition to the previously stated minimization goal. And the hack suggested in the doc isn't quite the right way to correct either. So I am leaning towards thinking that we shouldn't bother with this at all. That is, just subtract e_f(i) and don't care if the accumulation of error forecast have made tokens too few. But we should use thelastTick
logic insetAvailableTokens
to also adjustdiskTokensAvailable.writeByteTokens
, so that every 15s we restore it to 0.
Changed implementation to reflect what we discussed.
pkg/util/admission/granter.go
line 538 at r3 (raw file):
Previously, sumeerbhola wrote…
why this resetting to 0 after subtracting the
readError
?
Changed implementation to reflect what we discussed.
pkg/util/admission/io_load_listener.go
line 249 at r3 (raw file):
Previously, sumeerbhola wrote…
I think we should add a comment stating that even though these are prefixed with elastic, these are tokens for both regular and elastic work. These are only prefixed with elastic as only elastic work waits for them. Technically, elastic reads don't wait for elasticDiskReadTokens either, so perhaps we should just drop the elastic prefix on these fields.
Done.
e390d83
to
14b299d
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.
Reviewed all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aadityasondhi and @itsbilal)
pkg/util/admission/io_load_listener.go
line 462 at r5 (raw file):
tickDur = loadedDuration } errorTickThreshold := uint64((int64(errorAdjustmentDuration) * tickDur.ticksInAdjustmentInterval()) / int64(adjustmentInterval*time.Second))
I don't think this is robust to delayed ticks in the way that remainingTicks
is robust.
If we are ticking at 1ms, the errorTickThreshold=1000
. Which is reasonable since we expect 15000 ticks under perfect conditions. But we can easily jump from 14002 remaining ticks to 13998 remaining ticks (making something up). The caller of remainingTicks
adjusts to these jumps by apportioning the remaining tokens over the remaining ticks. But the logic here is expecting the modulus to be 0. But since we never say remainingTicks=14000 it will not be 0, even though we crossed 14000. I think we need some additional state in tokenAllocationTicker
to compensate for these jumps.
pkg/util/admission/granter.go
line 699 at r5 (raw file):
// NB: We don't cap the disk read tokens as they are only deducted during the // error accounting loop. So essentially, we give reads the "burst" capacity // of the error accounting interval.
Say these ticks are happening every 250ms (to make the example more concise), and each tick is adding 1MiB (i.e., diskReadTokens=1MiB). So at 1s we are at 4MiB. Then say the observed reads is 1MiB. So the diskReadTokensAlreadyDeducted=3MiB
, and this repeats for the first 10s, which gives us diskReadTokensAlreadyDeducted=30MiB
. So we've accumulated a "burst capacity" of 30MiB. If the next 1s sees 10MiB of reads, the burst capacity can absorb multiple seconds of such behavior, without deducting it from the write tokens.
This seems wrong. The provisioned disk bandwidth, especially on cloud virtual disks, doesn't have such multi-second burst capacity. Enforcement of the provisioned bandwidth/IOPS can happen at a fine granularity (some notes from a meeting suggest 1s interval in EBS).
We may need to go back to the idea of making the error accounting run at 250ms and use a read bandwidth capacity like the others.
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.
Haven't made any changes. Will wait on agreement on the error ticking logic.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/granter.go
line 699 at r5 (raw file):
That large burst capacity thing would not happen. That is because we reset diskReadTokensAlreadyDeducted
every time we adjust error. See adjustDiskTokenErrorLocked
. Maybe worth adding a comment here. So at most, we have a burst capacity of 1s (error interval). It has worked fine in experiments, granted I was not specifically looking for this type of issue.
We may need to go back to the idea of making the error accounting run at 250ms and use a read bandwidth capacity like the others.
We could do this but it would also need to adapt to the 1ms ticking, otherwise we can't add a cap to the read tokens at 250ms. This would mean basically account for error, every allocation loop. I am worried that 1ms might be too short of an interval to account for reads and writes and we may start doing too much.
We could alternatively, decouple the allocation of read tokens from the rest and allocate them at a fixed 250ms interval, which would match the error accounting interval, but it is pretty much the same thing as what we do now (1s burst). But it make this code more confusing.
I can work on making this code easier to digest, add more documentation etc., if you think the logic I described in my first statement is sound.
pkg/util/admission/io_load_listener.go
line 462 at r5 (raw file):
Previously, sumeerbhola wrote…
I don't think this is robust to delayed ticks in the way that
remainingTicks
is robust.If we are ticking at 1ms, the
errorTickThreshold=1000
. Which is reasonable since we expect 15000 ticks under perfect conditions. But we can easily jump from 14002 remaining ticks to 13998 remaining ticks (making something up). The caller ofremainingTicks
adjusts to these jumps by apportioning the remaining tokens over the remaining ticks. But the logic here is expecting the modulus to be 0. But since we never say remainingTicks=14000 it will not be 0, even though we crossed 14000. I think we need some additional state intokenAllocationTicker
to compensate for these jumps.
Nice callout. I did not consider that. I will add a field for last error accounting "tick".
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.
Reviewed 1 of 7 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aadityasondhi and @itsbilal)
pkg/util/admission/granter.go
line 699 at r5 (raw file):
That large burst capacity thing would not happen. That is because we reset
diskReadTokensAlreadyDeducted
every time we adjust error. SeeadjustDiskTokenErrorLocked
. Maybe worth adding a comment here. So at most, we have a burst capacity of 1s (error interval).
Ah yes. Definitely worth a code comment. It's easy to forget how these things interlock.
510e7b3
to
61eea9f
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.
Should be good for another look.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/granter.go
line 699 at r5 (raw file):
Previously, sumeerbhola wrote…
That large burst capacity thing would not happen. That is because we reset
diskReadTokensAlreadyDeducted
every time we adjust error. SeeadjustDiskTokenErrorLocked
. Maybe worth adding a comment here. So at most, we have a burst capacity of 1s (error interval).Ah yes. Definitely worth a code comment. It's easy to forget how these things interlock.
Done.
pkg/util/admission/io_load_listener.go
line 462 at r5 (raw file):
Previously, aadityasondhi (Aaditya Sondhi) wrote…
Nice callout. I did not consider that. I will add a field for last error accounting "tick".
Done.
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.
Reviewed 1 of 10 files at r2, 4 of 7 files at r4, 3 of 3 files at r6.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aadityasondhi and @itsbilal)
pkg/util/admission/grant_coordinator.go
line 148 at r6 (raw file):
storeLoaded := gc.pebbleMetricsTick(ctx, m) // TODO(aaditya): Verify we go back to the unloaded state. I don't // think we do.
Good catch. But given the pain 250ms ticks have caused for CPU, I think we should leave the accidental current behavior and simply change the comment where systemLoaded
is declared saying that we start as unloaded mainly for tests, and do a one-way transition to do 1ms ticks once load happens.
pkg/util/admission/testdata/granter
line 837 at r6 (raw file):
# Tokens already deducted is 0. Any writes that occur will be considered error. adjust-disk-error actual-write-bytes=10 actual-read-bytes=10
did this deduct 20 from disk-write-tokens-avail? Seems to have deducted 10.
pkg/util/admission/testdata/granter
line 845 at r6 (raw file):
---- GrantCoordinator: (chain: id: 0 active: false index: 5) io-avail: 50(50), disk-write-tokens-avail: 70
can we also print the 20 for the disk read tokens?
pkg/util/admission/testdata/granter
line 849 at r6 (raw file):
# Tokens already deducted is 0. Any writes that occur will be considered error. # We accounted for 20B of reads, so reads in excess of that will be error. err = # (10-0) + (40-20) = 30. We expect 70-30=40 tokens available.
I didn't understand the equation. Disk read error = 20 -50 = -30
Disk write error = 0 - 20 = -20.
So we have a total error of -50? Should that make the avail tokens = 70-50?
pkg/util/admission/io_load_listener.go
line 469 at r6 (raw file):
t.lastErrorAdjustmentTick = uint64(tickDur.ticksInAdjustmentInterval()) } errorTickThreshold := uint64((int64(errorAdjustmentDuration) * tickDur.ticksInAdjustmentInterval()) / int64(adjustmentInterval*time.Second))
I think it would be good to separate the const part of this computation into a declared const for (adjustmentInterval*time.Second)/errorAdjustmentDuration
. It makes it easier to understand.
Also, an example of how t.lastErrorAdjustmentTick will evolve over an adjustmentInterval would help the reader. It took me some time to understand what was going on here.
pkg/util/admission/io_load_listener.go
line 475 at r6 (raw file):
if !shouldAdjust { return false }
Is TestTokenAllocationTicker
updated to test the new logic?
61eea9f
to
5c3f94f
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.
Sorry for taking this long in getting back to this. Still have one more thing to address.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/grant_coordinator.go
line 148 at r6 (raw file):
Previously, sumeerbhola wrote…
Good catch. But given the pain 250ms ticks have caused for CPU, I think we should leave the accidental current behavior and simply change the comment where
systemLoaded
is declared saying that we start as unloaded mainly for tests, and do a one-way transition to do 1ms ticks once load happens.
Done.
pkg/util/admission/io_load_listener.go
line 469 at r6 (raw file):
Previously, sumeerbhola wrote…
I think it would be good to separate the const part of this computation into a declared const for
(adjustmentInterval*time.Second)/errorAdjustmentDuration
. It makes it easier to understand.Also, an example of how t.lastErrorAdjustmentTick will evolve over an adjustmentInterval would help the reader. It took me some time to understand what was going on here.
Done.
pkg/util/admission/io_load_listener.go
line 475 at r6 (raw file):
Previously, sumeerbhola wrote…
Is
TestTokenAllocationTicker
updated to test the new logic?
Nice catch. Added a separate test. Look at TestTokenAllocationTickerErrorAdjustmentThreshold
.
pkg/util/admission/testdata/granter
line 837 at r6 (raw file):
Previously, sumeerbhola wrote…
did this deduct 20 from disk-write-tokens-avail? Seems to have deducted 10.
Since this was the first iteration. We had unlimited
read tokens. And because of that we don't deduct them. This is a little unfortunate setup, since I wanted to use the unlimited
value to denote "unset" tokens.
Now that I think of it, maybe we should just leave it at 0 if unset since disk write tokens will be unlimited. I will put up a change in a bit to address that, flushing what I have for now.
pkg/util/admission/testdata/granter
line 845 at r6 (raw file):
Previously, sumeerbhola wrote…
can we also print the 20 for the disk read tokens?
Done.
pkg/util/admission/testdata/granter
line 849 at r6 (raw file):
Previously, sumeerbhola wrote…
I didn't understand the equation. Disk read error = 20 -50 = -30
Disk write error = 0 - 20 = -20.
So we have a total error of -50? Should that make the avail tokens = 70-50?
I added more commentary to explain what happened here.
Basically, the bytes passed into the function are cumulative, so we look at the delta. The delta here is, write = 10, read = 40.
5c3f94f
to
148e1d2
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.
Should be ready for another look. Thanks for your patience with this one!
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @sumeerbhola)
pkg/util/admission/testdata/granter
line 837 at r6 (raw file):
Previously, aadityasondhi (Aaditya Sondhi) wrote…
Since this was the first iteration. We had
unlimited
read tokens. And because of that we don't deduct them. This is a little unfortunate setup, since I wanted to use theunlimited
value to denote "unset" tokens.Now that I think of it, maybe we should just leave it at 0 if unset since disk write tokens will be unlimited. I will put up a change in a bit to address that, flushing what I have for now.
Done.
I set the default value for these tokens to be 0 rather than unlimited. Added some commentary for the reasoning of doing so.
8ef71c9
to
2904c11
Compare
Previously, we would ignore errors in disk token accounting hoping that the next adjustment interval would capture them. This caused issues when async compactions ran within an adjustment interval. These could spike the bandwidth for a section of the 15 second interval without moving the average too much (write-amp estimation). This patch introduces a higher frequency error accounting mechanism. It runs every 1s to account for any additional reads and writes that could have occurred. Errors are only deducted if the read and/or write bandwidth in the interval is greater than the number of tokens already deducted in that interval. Fixes cockroachdb#132332. Release note: None
2904c11
to
460d4b0
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.
Reviewed 1 of 7 files at r4, 3 of 5 files at r7, 2 of 2 files at r8, all commit messages.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @itsbilal)
TFTR! bors r=sumeerbhola |
With cockroachdb#133310 merged, and promising results from internal experimentation, we can turn this setting on by default. Informs: cockroachdb#121779. Release note (ops change): `kvadmission.store.snapshot_ingest_bandwidth_control.enabled` is now 'true' by default. This will enable disk bandwdith based Admission Control for range snapshot ingests. It requires the provisioned bandwidth to be set using `kvadmission.store.provisioned_bandwidth`.
With cockroachdb#133310 merged, and promising results from internal experimentation, we can turn this setting on by default. Informs: cockroachdb#121779. Release note (ops change): `kvadmission.store.snapshot_ingest_bandwidth_control.enabled` is now 'true' by default. This will enable disk bandwidth based Admission Control for range snapshot ingests. It requires the provisioned bandwidth to be set using `kvadmission.store.provisioned_bandwidth`.
With cockroachdb#133310 merged, and promising results from internal experimentation, we can turn this setting on by default. Informs: cockroachdb#121779. Release note (ops change): `kvadmission.store.snapshot_ingest_bandwidth_control.enabled` is now 'true' by default. This will enable disk bandwidth based Admission Control for range snapshot ingests. It requires the provisioned bandwidth to be set using `kvadmission.store.provisioned_bandwidth`.
137618: admission: turn on snapshot ingest bandwidth control by default r=sumeerbhola a=aadityasondhi With #133310 merged, and promising results from internal experimentation, we can turn this setting on by default. Informs: #121779. Release note (ops change): `kvadmission.store.snapshot_ingest_bandwidth_control.enabled` is now 'true' by default. This will enable disk bandwidth based Admission Control for range snapshot ingests. It requires the provisioned bandwidth to be set using `kvadmission.store.provisioned_bandwidth`. 137697: build: add configuration to opt into PGO r=rail a=rickystewart Epic: CRDB-41952 Part of: CRDB-44693 Release note: None Co-authored-by: Aaditya Sondhi <[email protected]> Co-authored-by: Ricky Stewart <[email protected]>
Previously, we would ignore errors in disk token accounting hoping that
the next adjustment interval would capture them. This caused issues when
async compactions ran within an adjustment interval. These could spike
the bandwidth for a section of the 15 second interval without moving the
average too much (write-amp estimation).
This patch introduces a higher frequency error accounting mechanism. It
runs every 1s to account for any additional reads and writes that could
have occurred. Errors are only deducted if the read and/or write
bandwidth in the interval is greater than the number of tokens already
deducted in that interval.
Fixes #132332.
Release note: None