release-22.1: kv/bulk: parallelize sending SSTs due to range bounds#80386
release-22.1: kv/bulk: parallelize sending SSTs due to range bounds#80386dt merged 6 commits intocockroachdb:release-22.1from
Conversation
Release note: none.
Release note: none.
the buffering adder took over making this split so this was dead code. Release note: none.
Previously the batcher, when it determined it needed to finish one SST and send it before starting another, would wait for it to be sent before moving one. When flushing a buffer that contained data that mapped to many ranges, this meant many serial flushes, e.g. flushing 512MB of data from a buffer that had keys uniformly distributed over a table which was split into 2000 ranges meant waiting for roughly 2000 sequential AddSSTable requests. When those requests were slow, for example sometimes taking as much as 1s each or more, this became a major bottleneck. This change switches the batcher to send files that are ended due to a round boundary asynchronously, queuing up the request to send and then starting the next file while it sends, as long as memory capacity in the monitor allows holding the extra file in memory (as these async sends could result in using an entire extra buffer's worth of memory if they all end up in-flight at once, which they easily could if the receivers are queuing). Release note (performance improvement): Bulk ingestion of unsorted data during IMPORT and schema changes uses a higher level of parallelism to send produced data to the storage layer.
|
Thanks for opening a backport. Please check the backport criteria before merging:
If some of the basic criteria cannot be satisfied, ensure that the exceptional criteria are satisfied within.
Add a brief release justification to the body of your PR to justify this backport. Some other things to consider:
|
shermanCRL
left a comment
There was a problem hiding this comment.
Approved for extraordinary backport, pending code review from @nvanbenschoten.
nvb
left a comment
There was a problem hiding this comment.
Reviewed 5 of 5 files at r1, 9 of 9 files at r2, 1 of 1 files at r3, 7 of 7 files at r4, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @dt)
pkg/kv/bulk/sst_batcher.go, line 483 at r4 (raw file):
} if reason == rangeFlush {
I'm struggling to understand how this interacts with the b.mu.lastRange state. When we issue multiple AddSSTables concurrently, the information added to b.mu.lastRange will be non-deterministic (the previous request may not have finished, or its result may have been cloberred). Is this ok because we're only using concurrency in the rangeFlush case, and we only want to perform manual splits in the rangeSize case where we still issue requests serially? If so, I wonder whether we could make that more clear in the code. If the state in b.mu.lastRange is meaningless because it's non-deterministic, should we even set it in those cases?
pkg/kv/bulk/sst_batcher.go, line 692 at r4 (raw file):
} b.mu.Lock() b.stats.splitRetries += files - 1
Should b.stats be in the mu struct?
dt
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/bulk/sst_batcher.go, line 483 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
I'm struggling to understand how this interacts with the
b.mu.lastRangestate. When we issue multiple AddSSTables concurrently, the information added tob.mu.lastRangewill be non-deterministic (the previous request may not have finished, or its result may have been cloberred). Is this ok because we're only using concurrency in therangeFlushcase, and we only want to perform manual splits in therangeSizecase where we still issue requests serially? If so, I wonder whether we could make that more clear in the code. If the state inb.mu.lastRangeis meaningless because it's non-deterministic, should we even set it in those cases?
async requests will indeed overwrite each other's lastRange, but i think last-write-wins is probably going to be fine, since as you say, if we're sending async flushes at all, we're probably already in the small-ssts-to-many-ranges case where we don't send our own splits anyway.
I could just make only size-flushes update lastRange if you'd prefer?
pkg/kv/bulk/sst_batcher.go, line 692 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should
b.statsbe in themustruct?
Hm, I'd tried to avoid moving it, since most of the fields are accessed without holding mu. Indeed, the reader of this field doesn't lock it, since reads are done only by the log statement that is called after a flush() which has called .Wait(), so the locking is only for write/write races.
I decided that was too confusing with the other ones and just made them atomics to avoid commenting all that; I could do the same here if you feel strongly.
nvb
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @dt)
pkg/kv/bulk/sst_batcher.go, line 483 at r4 (raw file):
Previously, dt (David Taylor) wrote…
async requests will indeed overwrite each other's
lastRange, but i think last-write-wins is probably going to be fine, since as you say, if we're sending async flushes at all, we're probably already in the small-ssts-to-many-ranges case where we don't send our own splits anyway.I could just make only size-flushes update lastRange if you'd prefer?
If you did that then we could pull b.mu.lastRange out of the mutex, right? I would find that much easier to reason about.
pkg/kv/bulk/sst_batcher.go, line 692 at r4 (raw file):
Previously, dt (David Taylor) wrote…
Hm, I'd tried to avoid moving it, since most of the fields are accessed without holding mu. Indeed, the reader of this field doesn't lock it, since reads are done only by the log statement that is called after a flush() which has called .Wait(), so the locking is only for write/write races.
I decided that was too confusing with the other ones and just made them atomics to avoid commenting all that; I could do the same here if you feel strongly.
I feel less strongly about this one. I'll defer to you.
dt
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @dt and @nvanbenschoten)
pkg/kv/bulk/sst_batcher.go, line 483 at r4 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
If you did that then we could pull
b.mu.lastRangeout of the mutex, right? I would find that much easier to reason about.
added a commit (will forward port it to master too if it looks like what you were thinking)
nvb
left a comment
There was a problem hiding this comment.
Reviewed 2 of 2 files at r5, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @dt)
pkg/kv/bulk/sst_batcher.go, line 542 at r5 (raw file):
sstBytes []byte, stats enginepb.MVCCStats, reason int,
Should this be the reason, or should this be a bool indicating that the addSSTable is being run concurrently? Or should this be a trackLastRange bool? In other words, whose responsibility should it be to make the decision about whether setting b.lastRange is needed and is thread-safe?
c46b95d to
bcb710a
Compare
dt
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/bulk/sst_batcher.go line 542 at r5 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should this be the reason, or should this be a bool indicating that the addSSTable is being run concurrently? Or should this be a
trackLastRange bool? In other words, whose responsibility should it be to make the decision about whether settingb.lastRangeis needed and is thread-safe?
Good point, makes sense.
Done.
nvb
left a comment
There was a problem hiding this comment.
Reviewed 1 of 1 files at r6, 3 of 3 files at r7, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @dt)
Release note: none.
Release note: none.
bcb710a to
c0d7f68
Compare
Fixed in second commit of #80467 and picked up here too now. |
|
verified restore/nodeShutdown/worker passes on master with the fix mentioned above. so think this is good to go |
|
blathers backport 22.1.0 |
|
Encountered an error creating backports. Some common things that can go wrong:
You might need to create your backport manually using the backport tool. error creating backport branch refs/heads/blathers/backport-release-22.1.0-80386: POST https://api.github.com/repos/cockroachdb/cockroach/git/refs: 422 Reference already exists [] Backport to branch 22.1.0 failed. See errors above. 🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan. |
Backport:
Please see individual PRs for details.
/cc @cockroachdb/release
Addresses #79615
Release justification: high impact change motivated by customer request.