-
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
storage: backpressure SSTs before other writes if L0 fills up #34258
Conversation
This was free-friday foray a bit outside my familiar territory, so hoping to get some guidance from folks who know this code a bit better as far as if this is the right place/approach, in particular w.r.t. what we need to be worried about during apply, blocking raft, etc. Also, I think ideally we'd only back pressure SSTs that are headed for L0? That's require knowing its bounds though, which we don't have readily available during apply at least unless we open it and read them. Pushing this down to Rocks could make sense -- it is pretty similar to the write backpressure mechanism that motivates this and Rocks is already going to figure out where this SST is going, how many other SSTs it has, etc, That said, for now I figured at least for now I'd like to log when it happens (and maybe even maintain a metric) so doing it separately seemed to make sense? Also doing it outside ingest allows doing it before it is in raft at all which seemed like a good idea? |
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 is interesting. Have you done any experiments to see if it makes a difference?
Do we have metrics on how often sstable ingestion is going into L0 vs lower levels? I'm pretty sure no, but I'd like to know those numbers in order to motivate this work. Adding a few fprintfs to RocksDB and running an ingestion-heavy workload (restore or import) would be one way to gather this.
Reviewed 3 of 3 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten, @petermattis, and @tbg)
I did some tests last month in a three-node cluster and did not trigger the RocksDB slowdown -- I had We don't currently have a problem with this at all: RESTORE and IMPORT should usually ingest only at the lowest level -- since we carve out new key space that isn't seeing any other traffic, the AddSSTAble is usually the first write and overlaps with nothing. With index backfills though this will not be the case -- a table with heavy write traffic will likely have index entries written by live traffic in the ranges by the time the backfill gets to them, so the SST would overlap pretty recent (i.e. not yet compacted) writes. But that isn't the really scary part: every table reader will be generate batches of index entries, chunking them into SSTs and then shipping those off to the appropriate range. Thus a given range should expect, in the worst case, to receive an SSTs from every reader from every batch (i.e. 100k rows of the source table) -- all of which can overlap each other, forcing them all into L0. Since a range should expect, worst case, an SST from every producer, a given range should expect to be hit with num-nodes overlapping SSTs at the same time. In a larger cluster, this could hit the constant limit of 20 that triggers the slowdown by itself. The risk seems exacerbated by the fact the node likely has many ranges in the same situation in that RocksDB, so it could easily spike above even the hard stop limit. |
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.
Adding a few fprintfs to RocksDB and running an ingestion-heavy workload (restore or import) would be one way to gather this.
Before we do a one-off here, it wouldn't be more work to add sensible metrics that also make sense in other contexts. For example, we could add metrics lsm_num_files_l{0,1,2,3,4,5,6}
.
I generally think this change is a good idea, but like @petermattis I think we should find a scenario in which it fires (perhaps the 32 node restore tests?).
Reviewed 3 of 3 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt, @nvanbenschoten, @petermattis, and @tbg)
pkg/storage/engine/rocksdb.go, line 2930 at r2 (raw file):
// SST additions well before then at 10. const l0SlowdownFiles = 10 for i, re := 1, retry.StartWithCtx(ctx, retry.Options{MaxRetries: 6}); re.Next(); i++ {
Could you specify the options a bit more? Now this is using some defaults but those could change (plus I don't know what to expect here)
Because we keep all replicas in the same rocksdb, this stops being true if, after you kick off the restore/import, you make a new table and give it write traffic. |
A customer trying out the latest 19.1 beta hit a rocksdb stall crash and looking at the log they pasted, that node had 326 SSTs added to L0 in the 100s interval prior to the crash, with 536 l0_slowdown stalls and 17k pending compaction byte stalls. It seems like we might want to dust off this change -- we're working on bigger fixes for 19.2 that might make fewer files, or make us able to ingest more files and ingest faster -- but until then (i.e. in 19.1) this change should at least protect the running node from being crashed by excessive AddSSTables. |
@ajkr this topic is probably going to keep us busy in the next release cycle (these stalls have caused node liveness failure and with that, complete temporary cluster failure of course on top of a myriad of other symptoms that tend to really confuse the untrained eye). What's your take on changes like these? Is this a good workaround for now? Is there something else that's not too invasive that we should be looking at? |
I think we need to extend this change to also check |
I added |
That sounds reasonable to me. @tbg I think backpressure is good since it prevents file ingestion from impacting live traffic. I know @dt has some ideas regarding cutting the files before they're ingested in a way that will reduce overlap, allowing us to ingest them directly into a lower level in some cases. But until that's available, backpressure sounds like a good strategy to me. |
I think we should strongly consider backporting this to 19.1.x -- I don't have a reproduction of the stalls the customer saw yet but as it stands, this the the best idea I have for preventing the worse-case ingestion behavior from turning into a stability issue. I think the main downside of this change, as I understand it it, would be that it would slow down SST applications while holding the replica mu? but probably less bad than holding the whole store behind a rocks stall after the ingestion? I'll keep trying to reproduce the stall the customer saw to see if I can confirm this works, but I don't have any better ideas right now. |
22e9acd
to
bc07980
Compare
Rebased this and moved the proposer delay to also be outside Eval, so we don't need to hold latches either. The application delay still needs to block raft, but there isn't much we can do about that one short of somehow gossiping follower health info up to the leader. |
2240a68
to
4ad3442
Compare
@petermattis @ajkr we're pretty confident we need something like this since we're seeing rocks stalls even with the L0 file limit bumped up to 200. |
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, just had a question about adding new settings vs. deriving values from existing ones.
var ingestDelayPerFile = settings.RegisterDurationSetting( | ||
"rocksdb.ingest_backpressure.delay_l0_file", | ||
"delay to add to ingestions per file in L0 over the configured limit", | ||
time.Millisecond*200, |
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.
Have you considered deriving these values from RocksDB's rates/limits? For example the Rocks slowdown rate is currently configured at 16MB/s, so if we had a 64MB file to ingest, we'd calculate a four second delay.
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 5 of 12 files at r3, 3 of 3 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
pkg/storage/engine/rocksdb.go, line 104 at r4 (raw file):
"rocksdb.ingest_backpressure.max_delay", "maximum amount of time to backpressure ingestions", time.Second*5,
Why is this one configurable but the 15s in Store.Send is not? I think I'd prefer to hard-code both and have fewer knobs to turn.
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 @bdarnell, @dt, @lucy-zhang, and @nvanbenschoten)
pkg/storage/replica_proposal.go, line 394 at r3 (raw file):
Previously, lucy-zhang (Lucy Zhang) wrote…
I think this needs to be
engine.PreIngestDefaultDelayLimit
Done.
pkg/storage/engine/rocksdb.go, line 104 at r4 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Why is this one configurable but the 15s in Store.Send is not? I think I'd prefer to hard-code both and have fewer knobs to turn.
I made it configurable as a cop-out since I wasn't sure of the best value to pick for a constant. I'm conflicted on what to pick here since it blocks raft if we hit this one, but if we don't, rocks later will when it gets swapped.
I could make both settings, but I also felt like I might be making too many knobs.
I guess making one a setting and the other not is similarly strange though, so maybe the Store.Send call should just pass a multiplier to apply to the setting-read timeout?
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 @ajkr, @bdarnell, @lucy-zhang, @nvanbenschoten, and @tbg)
pkg/storage/engine/rocksdb.go, line 2930 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Could you specify the options a bit more? Now this is using some defaults but those could change (plus I don't know what to expect here)
The retry loop went away.
pkg/storage/engine/rocksdb.go, line 92 at r4 (raw file):
Previously, ajkr (Andrew Kryczka) wrote…
Have you considered deriving these values from RocksDB's rates/limits? For example the Rocks slowdown rate is currently configured at 16MB/s, so if we had a 64MB file to ingest, we'd calculate a four second delay.
That sounds like it could work though I think currently I'm going the opposite direction and making these runtime-tunable knobs -- (which, as it stands, most of our rocks options are not since they're not hooked up to any knobs a user can get to).
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 @ajkr, @bdarnell, @lucy-zhang, @nvanbenschoten, and @tbg)
pkg/storage/engine/rocksdb.go, line 104 at r4 (raw file):
Previously, dt (David Taylor) wrote…
I made it configurable as a cop-out since I wasn't sure of the best value to pick for a constant. I'm conflicted on what to pick here since it blocks raft if we hit this one, but if we don't, rocks later will when it gets swapped.
I could make both settings, but I also felt like I might be making too many knobs.
I guess making one a setting and the other not is similarly strange though, so maybe the Store.Send call should just pass a multiplier to apply to the setting-read timeout?
I make them both just call PreIngestDelay(ctx)
and they both just use the setting value.
I'm hesitant to switch to a constant that hasn't had weeks or month of time on master to bake, because if we get it wrong, I'd rather have the escape hatch of telling people to change the setting as opposed to deploy a new binary.
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 overall approach looks good.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajkr, @bdarnell, @dt, @lucy-zhang, @nvanbenschoten, and @tbg)
c-deps/libroach/engine.cc, line 208 at r5 (raw file):
std::string l0_file_count_str; rep->GetProperty("rocksdb.num-files-at-level0", &l0_file_count_str);
Nit: the indentation of the second line seems off. It looks reasonable for this to be on a single line.
pkg/storage/replica_proposal.go, line 393 at r5 (raw file):
} if r, ok := eng.(*engine.RocksDB); ok {
Is there a reason you didn't add PreIngestDelay
is a method on the Engine
interface? All of these type-assertions to *engine.RocksDB
will have to go away if we support another storage engine. (I realize there are similar type-assertions already in the code, I'm just reluctant to see more added).
pkg/storage/engine/rocksdb.go, line 2930 at r2 (raw file):
Previously, dt (David Taylor) wrote…
The retry loop went away.
Heh, I was expecting some sort of loop here waiting for the number of L0 files to fall below a target count. I'm curious why you moved away from that approach. If something is horking up L0 compactions, I think we'd want to continue to wait until the number of L0 files fell below a threshold (up to some maximum wait time).
pkg/storage/engine/rocksdb.go, line 86 at r5 (raw file):
"rocksdb.ingest_backpressure.l0_file_count_threshold", "number of L0 files after which to backpressure SST ingestions", 20,
That's a lot of L0 files. Is there a rationale behind setting this to 20 vs something lower?
pkg/storage/engine/rocksdb.go, line 103 at r5 (raw file):
var ingestDelayTime = settings.RegisterDurationSetting( "rocksdb.ingest_backpressure.max_delay", "maximum amount of time to backpressure ingestions",
to backpressure a single ingestion
pkg/storage/engine/rocksdb.go, line 3025 at r5 (raw file):
} func (r *RocksDB) calculatePreIngestDelay(stats *Stats) time.Duration {
The only member of RocksDB
you're using is RocksDB.cfg
. I think it would be better to pass that as a parameter and make this a global function. That would remove the awkwardness in TestIngestDelayLimit
of creating a RocksDB
and only populating a single field.
Currently we start compacting as soon as there are two or more files in L0. If we reach 20, RocksDB will start slowing all writes in an attempt to help the compactor catch up. This global slowdown will affect foreground traffic, liveness, etc and could significantly impact latency of live traffic or even availability. Directly ingesting large numbers of small files could make it easier to hit this limit than it usually is with normal writes -- normal writes are buffered in the memtable and flushed all at once but direct ingestion could add potentially lots of files rather quickly. Additionally, SST ingetions are currently only done by bulk operations like RESTORE, IMPORT or index backfills. These are all less latency sensitive then foreground traffic, so we'd prefer to backpressure them before we risk approaching the global slowdown trigger. Release note (performance improvement): backpressure bulk operations before other traffic.
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 @ajkr, @bdarnell, @lucy-zhang, @nvanbenschoten, @petermattis, and @tbg)
c-deps/libroach/engine.cc, line 208 at r5 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Nit: the indentation of the second line seems off. It looks reasonable for this to be on a single line.
Done.
pkg/storage/replica_proposal.go, line 393 at r5 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Is there a reason you didn't add
PreIngestDelay
is a method on theEngine
interface? All of these type-assertions to*engine.RocksDB
will have to go away if we support another storage engine. (I realize there are similar type-assertions already in the code, I'm just reluctant to see more added).
It seemed like it was working around a RocksDB-specific problem, so the type-specific call-sites didn't bother me, but I could also see other engines just nooping.
Done.
pkg/storage/engine/rocksdb.go, line 2930 at r2 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Heh, I was expecting some sort of loop here waiting for the number of L0 files to fall below a target count. I'm curious why you moved away from that approach. If something is horking up L0 compactions, I think we'd want to continue to wait until the number of L0 files fell below a threshold (up to some maximum wait time).
Yeah, I had a loop like that, but it made the code a tad more complicated and made the service order / queue discipline a bit murky.
pkg/storage/engine/rocksdb.go, line 86 at r5 (raw file):
Previously, petermattis (Peter Mattis) wrote…
That's a lot of L0 files. Is there a rationale behind setting this to 20 vs something lower?
20 was the old slowdown limit (before we bumped it to 200) which is why I picked it.
While that would be a lot of L0 files if they were normal L0 files -- i.e. the size of full memtable and covering the whole key span and thus adding a full read amplification multiple -- when we're ingesting, we have reason to believe that many of these files are not normal L0 files. Ingested files probably only cover a small subspan of the full key span, so the majority of traffic can likely skip then via the bounds check and never end up seeking though the file itself.
Of course some of the files will be ingest-triggered memtable flushes, which likely still cover the whole active keyspan, so but even those are probably smaller than usual since we're forcing frequent flushes during ingest.
If we back-pressused at a much smaller number, we'd likely bring the ingestion to a crawl if it hits some data distribution that results in many small SSTs. Indeed, generally when we see larger numbers of L0 files, they're usually quite small (<1mb) and we actually kind of want the compactor to get as many of these in a single pass as it can (indeed, I'd like to raise the compaction trigger during ingest, but that should happen separately), so letting a few more files into L0 should help the ingest continue to make progress.
All of that said, I don't actually know if 20 is the right number, which is part of why I made it a dynamically adjustable setting.
pkg/storage/engine/rocksdb.go, line 103 at r5 (raw file):
Previously, petermattis (Peter Mattis) wrote…
to backpressure a single ingestion
Done.
pkg/storage/engine/rocksdb.go, line 3025 at r5 (raw file):
Previously, petermattis (Peter Mattis) wrote…
The only member of
RocksDB
you're using isRocksDB.cfg
. I think it would be better to pass that as a parameter and make this a global function. That would remove the awkwardness inTestIngestDelayLimit
of creating aRocksDB
and only populating a single field.
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 7 of 7 files at r6.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @ajkr, @lucy-zhang, @nvanbenschoten, @petermattis, and @tbg)
pkg/storage/engine/rocksdb.go, line 92 at r4 (raw file):
Previously, dt (David Taylor) wrote…
That sounds like it could work though I think currently I'm going the opposite direction and making these runtime-tunable knobs -- (which, as it stands, most of our rocks options are not since they're not hooked up to any knobs a user can get to).
Note that we do allow arbitrary rocksdb options to be passed as --store=path=/foo,rocksdb=key1=val1;key2=val2
.
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 @ajkr, @dt, @lucy-zhang, @nvanbenschoten, and @tbg)
pkg/storage/engine/rocksdb.go, line 86 at r5 (raw file):
Ack. Let's leave as-is for now.
so the majority of traffic can likely skip then via the bounds check and never end up seeking though the file itself
Iterators still need to consider each of these files, even though they are relatively quick checks. I'm not sure how that plays out in practice, though.
bors r+ |
bors r+ |
34258: storage: backpressure SSTs before other writes if L0 fills up r=dt a=dt Currently we start compacting as soon as there are two or more files in L0. If we reach 20, RocksDB will start slowing all writes in an attempt to help the compactor catch up. This global slowdown will affect foreground traffic, liveness, etc and could significantly impact latency of live traffic or even availability. Directly ingesting large numbers of small files could make it easier to hit this limit than it usually is with normal writes -- normal writes are buffered in the memtable and flushed all at once but direct ingestion could add potentially lots of files rather quickly. Additionally, SST ingetions are currently only done by bulk operations like RESTORE, IMPORT or index backfills. These are all less latency sensitive then foreground traffic, so we'd prefer to backpressure them before we risk approaching the global slowdown trigger. Release note (performance improvement): backpressure bulk operations before other traffic. 36689: opt: Fix and improve zone-based index selection r=andy-kimball a=andy-kimball This PR fixes a couple of related issues with zone-based index selection. First, placeholder zones were incorrectly discarded. Placeholder zones are used when there are indexes with Constraints/Leaseholder Preferences on a table that doesn't have either of those things. The fix is to merge the indexes placeholder into the table zone. Second, the optimizer required that a prefix of the locality tiers match constraints. That is no longer required after this commit. For example, if locality=region=us;dc=east, then it now matches any of these constraint sets equally well: [+locality=region=us;+dc=east] [+dc=east] [+dc=east,+dc=west] A missing constraint match is OK (i.e. region=us), as long as a more specific locality tier matches (i.e. dc=east). Fixes #36642 Fixes #36644 Release note: None Co-authored-by: David Taylor <[email protected]> Co-authored-by: Andrew Kimball <[email protected]>
Build succeeded |
This PR adds an unused setting |
@dt Any idea about |
Fixes cockroachdb#102683. Part of cockroachdb#104154. These were added way back in cockroachdb#36403 and cockroachdb#73904, pre-dating much of IO admission control for leaseholder writes. With cockroachdb#95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (cockroachdb#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in cockroachdb#34258 for RocksDB and in \cockroachdb#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action Release note: None
Fixes cockroachdb#102683. Part of cockroachdb#104154. These were added way back in cockroachdb#36403 and cockroachdb#73904, pre-dating much of IO admission control for leaseholder writes. With cockroachdb#95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (cockroachdb#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in cockroachdb#34258 for RocksDB and in \cockroachdb#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action Release note: None
104861: kvserver: disable pre-AC above-raft AddSST throttling r=irfansharif a=irfansharif Fixes #102683. Part of #104154. These were added way back in #36403 and #73904, pre-dating much of IO admission control for leaseholder writes. With #95563, we now have IO admission control for follower writes. Put together, have ample LSM read-amp protection through AC alone. These concurrency limiters are now redundant and oblivious to more sophisticated AC measures. We recently removed the below-raft equivalents of these limiters (#98762), and like mentioned there, these limiters can exacerbate memory pressure. Separately, we're looking to work on speedier restores, and these limiters are starting to get in the way. While here, we also disable the pre-ingest delay mechanism in pebble, which too pre-dates AC, introduced way back in #34258 for RocksDB and in \#41839 for Pebble. IO AC is able to limit the number of L0 files, and this pre-ingest delay with its maximum per-request delay time of 5s can be less than effective. It's worth noting that the L0 file count threshold at which this pre-ingest delay mechanism kicked in was 20, while AC aims for 1000[^1]. This commit doesn't go as far as removing these limiters outright, merely disabling them. This is just out of an overabundance of caution. We can probably remove them once kvflowcontrol.enabled has had >1 release worth of baking time. Until then, it's nice to know we have these old safety hatches. We have ample time in the release to assess fallout from this commit, and also use this increased AddSST concurrency to stress the kvflowcontrol machinery. [^1]: The 1000 file limit exists to bound how long it takes to clear L0 completely. Envelope math cribbed from elsewhere: With 2MiB files, 1000 files is ~2GB, which at 40MB/s of compaction throughput (with a compaction slot consistently dedicated to L0) takes < 60s to clear the backlog. So the 'recovery' time is modest in that operators should not need to take manual action. Release note: None Co-authored-by: irfan sharif <[email protected]>
Currently we start compacting as soon as there are two or more files in
L0. If we reach 20, RocksDB will start slowing all writes in an attempt
to help the compactor catch up.
This global slowdown will affect foreground traffic, liveness, etc and
could significantly impact latency of live traffic or even availability.
Directly ingesting large numbers of small files could make it easier to
hit this limit than it usually is with normal writes -- normal writes
are buffered in the memtable and flushed all at once but direct
ingestion could add potentially lots of files rather quickly.
Additionally, SST ingetions are currently only done by bulk operations
like RESTORE, IMPORT or index backfills. These are all less latency
sensitive then foreground traffic, so we'd prefer to backpressure them
before we risk approaching the global slowdown trigger.
Release note (performance improvement): backpressure bulk operations before other traffic.