engine: add byte limit to MVCCScan#44339
Conversation
petermattis
left a comment
There was a problem hiding this comment.
Both the KVs and Bytes response formats go through the same mvccScanner/pebbleMVCCScanner code paths. We always scan to the Bytes format, and then convert to KVs if necessary. I think you only have to do this work twice, not four times.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @itsbilal and @tbg)
pkg/storage/engine/pebble_mvcc_scanner.go, line 536 at r1 (raw file):
// pretend we reached maxKeys. (The resume key computation requires // fetching an additional key after that, so trying to exit the // iterations based on size alone is not trivial).
That is a bit hacky, though it seems safer than trying to extend the resume key logic.
|
Looks good to me so far. It may not be too much more effort to sprinkle in And yes, like Peter said, you'll have to make the same change only once more, in You should also be able to use Raphael's new datadriven-based test framework to quickly write up some tests for this. Take a look at |
|
Before I write tests, any concerns/ideas around naming this? We're not
really limiting the max size strictly speaking because we "overshoot". OTOH
this behavior makes sense (you wouldn't want to return zero results when
there's a resume key). There must be a "right" name for this sort of thing.
…On Fri, Jan 24, 2020 at 4:26 PM Bilal Akhtar ***@***.***> wrote:
Looks good to me so far. It may not be too much more effort to sprinkle in if
p.result.bytes >= p.maxBytes && p.maxBytes > 0 checks in all 3-4 of the
cases where we already have maxKeys checks, but given how this is a hot
path it might only make things less performant than just updating maxKeys,
so I'm fine with this approach. Possibly with a comment on the dual-use of
maxKeys where it is defined.
And yes, like Peter said, you'll have to make the same change only once
more, in mvcc.h for the rocksdb mvcc scanner. The KV/bytes cases wrap
around these two implementations.
You should also be able to use Raphael's new datadriven-based test
framework to quickly write up some tests for this. Take a look at
TestMVCCHistories and the giant comment there for how to use it.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#44339?email_source=notifications&email_token=ABGXPZBCCOVB5RH2UGLKBBDQ7MCDBA5CNFSM4KLFRT4KYY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEJ3EMMA#issuecomment-578176560>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/ABGXPZEVTZH7ZMVPU2WTAXDQ7MCDBANCNFSM4KLFRT4A>
.
|
|
`TargetBytes`? You can both overshoot and undershoot a target.
On Mon, Jan 27, 2020 at 7:38 AM Tobias Grieger <notifications@github.com>
wrote:
… Before I write tests, any concerns/ideas around naming this? We're not
really limiting the max size strictly speaking because we "overshoot". OTOH
this behavior makes sense (you wouldn't want to return zero results when
there's a resume key). There must be a "right" name for this sort of thing.
On Fri, Jan 24, 2020 at 4:26 PM Bilal Akhtar ***@***.***>
wrote:
> Looks good to me so far. It may not be too much more effort to sprinkle
in if
> p.result.bytes >= p.maxBytes && p.maxBytes > 0 checks in all 3-4 of the
> cases where we already have maxKeys checks, but given how this is a hot
> path it might only make things less performant than just updating
maxKeys,
> so I'm fine with this approach. Possibly with a comment on the dual-use
of
> maxKeys where it is defined.
>
> And yes, like Peter said, you'll have to make the same change only once
> more, in mvcc.h for the rocksdb mvcc scanner. The KV/bytes cases wrap
> around these two implementations.
>
> You should also be able to use Raphael's new datadriven-based test
> framework to quickly write up some tests for this. Take a look at
> TestMVCCHistories and the giant comment there for how to use it.
>
> —
> You are receiving this because you were mentioned.
> Reply to this email directly, view it on GitHub
> <
#44339?email_source=notifications&email_token=ABGXPZBCCOVB5RH2UGLKBBDQ7MCDBA5CNFSM4KLFRT4KYY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEJ3EMMA#issuecomment-578176560
>,
> or unsubscribe
> <
https://github.com/notifications/unsubscribe-auth/ABGXPZEVTZH7ZMVPU2WTAXDQ7MCDBANCNFSM4KLFRT4A
>
> .
>
—
You are receiving this because you commented.
Reply to this email directly, view it on GitHub
<#44339?email_source=notifications&email_token=ABPJ753QRCKPLT46LYKJCE3Q73IWHA5CNFSM4KLFRT4KYY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEJ7LDQI#issuecomment-578728385>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/ABPJ7566AG22B5I67DJPAZ3Q73IWHANCNFSM4KLFRT4A>
.
|
@itsbilal that's what I did initially, but it was... hard, because |
bd852dc to
da7977d
Compare
da7977d to
4346dac
Compare
|
This PR is now ready for review in earnest. |
| @@ -0,0 +1,58 @@ | |||
| ## Test opts.TargetBytes. | |||
There was a problem hiding this comment.
The testing seems a little on the light side to me, accepting suggestions on what else to test.
petermattis
left a comment
There was a problem hiding this comment.
though @itsbilal should give this a review as well.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @itsbilal and @tbg)
c-deps/libroach/chunked_buffer.cc, line 32 at r2 (raw file):
put(value.data(), value.size(), 0); count_++; bytes_ += 8 + key.size() + value.size(); // see (*pebbleResults).put
s/8/sizeof(size_buf)/g
pkg/storage/engine/testdata/mvcc_histories/target_bytes, line 1 at r2 (raw file):
Previously, tbg (Tobias Grieger) wrote…
The testing seems a little on the light side to me, accepting suggestions on what else to test.
Reverse scans?
Perhaps a test where keys are skipped. That is, show that the target bytes is only affected by keys that are added to the result, not keys that are skipped because they are too new.
pkg/storage/engine/testdata/mvcc_histories/target_bytes, line 7 at r2 (raw file):
put k=a v=abcdef put k=c v=ghijkl put k=e v=mnopqr
Might want to vary the value size. Right now, every key and value has the same size. This does make the results line up nicely, but is perhaps a bit too uniform for testing purposes.
itsbilal
left a comment
There was a problem hiding this comment.
Reviewed 10 of 10 files at r2.
Reviewable status:complete! 2 of 0 LGTMs obtained (waiting on @tbg)
c-deps/libroach/mvcc.h, line 728 at r2 (raw file):
const rocksdb::Slice end_key_; int64_t max_keys_; const int64_t target_bytes_; // see pebbleMVCCScanner
Might be worth saying see MVCCScanOptions instead, since the big comment explaining this is there?
pkg/storage/engine/mvcc_history_test.go, line 700 at r2 (raw file):
max = int64(imax) } if key := "tb"; e.hasArg(key) {
This works but I'm wondering if we can use a more descriptive arg name for this, given my first instinct was "tb? sounds like TomBstones". targetbytes probably works better and is less ambiguous.
pkg/storage/engine/pebble_mvcc_scanner.go, line 539 at r2 (raw file):
// number of keys. // // TODO(bilal): see if this can be implemented more transparently.
👍 on the todo
pkg/storage/engine/testdata/mvcc_histories/target_bytes, line 1 at r2 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Reverse scans?
Perhaps a test where keys are skipped. That is, show that the target bytes is only affected by keys that are added to the result, not keys that are skipped because they are too new.
👍 on the reverse scan suggestion.
Also, another possibility is zero-sized values (basically tombstones) and scans with tombstones=true
tbg
left a comment
There was a problem hiding this comment.
TFTRs! I souped up the tests with reverse iteration and tombstones (and the combination of both). Everything worked as expected.
Reviewable status:
complete! 2 of 0 LGTMs obtained (waiting on @petermattis)
c-deps/libroach/chunked_buffer.cc, line 32 at r2 (raw file):
Previously, petermattis (Peter Mattis) wrote…
s/8/sizeof(size_buf)/g
Done.
pkg/storage/engine/testdata/mvcc_histories/target_bytes, line 7 at r2 (raw file):
Previously, petermattis (Peter Mattis) wrote…
Might want to vary the value size. Right now, every key and value has the same size. This does make the results line up nicely, but is perhaps a bit too uniform for testing purposes.
Done.
4346dac to
8cc3a87
Compare
A fledgling step towards cockroachdb#19721 is allowing incoming KV requests to bound the size of the response in terms of bytes rather than rows. This commit adds a TargetBytes field to MVCCScanOptions to address this need: scans stop once the size of the result meets or exceeds the threshold (at least one key will be added, regardless of its size), and returns a ResumeSpan as appropriate. The classic example of the problem this addresses is a table in which each row is, say, ~1mb in size. A full table scan will currently fetch data from KV in batches of [10k], causing at least 10GB of data held in memory at any given moment. This sort of thing does happen in practice; we have a long-failing roachtest cockroachdb#33660 because of just that, and anecdotally OOMs in production clusters are with regularity caused by individual queries consuming excessive amounts of memory at the KV level. Plumbing this limit into a header field on BatchRequest and down to the engine level will allow the batching in [10k] to become byte-sized in nature, thus avoiding one obvious source OOMs. This doesn't solve cockroachdb#19721 in general (many such requests could work together to consume a lot of memory after all), but it's a sane baby step that might just avoid a large portion of OOMs already. [10k]: https://github.com/cockroachdb/cockroach/blob/0a658c19cd164e7c021eaff7f73db173f0650e8c/pkg/sql/row/kv_batch_fetcher.go#L25-L29 Release note: None
8cc3a87 to
14b6cc3
Compare
|
bors r=petermattis,itsbilal |
44339: engine: add byte limit to MVCCScan r=petermattis,itsbilal a=tbg A fledgling step towards #19721 is allowing incoming KV requests to bound the size of the response in terms of bytes rather than rows. This commit adds a TargetBytes field to MVCCScanOptions to address this need: scans stop once the size of the result meets or exceeds the threshold (at least one key will be added, regardless of its size), and returns a ResumeSpan as appropriate. The classic example of the problem this addresses is a table in which each row is, say, ~1mb in size. A full table scan will currently fetch data from KV in batches of [10k], causing at least 10GB of data held in memory at any given moment. This sort of thing does happen in practice; we have a long-failing roachtest #33660 because of just that, and anecdotally OOMs in production clusters are with regularity caused by individual queries consuming excessive amounts of memory at the KV level. Plumbing this limit into a header field on BatchRequest and down to the engine level will allow the batching in [10k] to become byte-sized in nature, thus avoiding one obvious source OOMs. This doesn't solve #19721 in general (many such requests could work together to consume a lot of memory after all), but it's a sane baby step that might just avoid a large portion of OOMs already. [10k]: https://github.com/cockroachdb/cockroach/blob/0a658c19cd164e7c021eaff7f73db173f0650e8c/pkg/sql/row/kv_batch_fetcher.go#L25-L29 Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
Build succeeded |
|
woo! |
A fledgling step towards #19721 is allowing incoming KV requests to
bound the size of the response in terms of bytes rather than rows.
This commit adds a TargetBytes field to MVCCScanOptions to address
this need: scans stop once the size of the result meets or exceeds the
threshold (at least one key will be added, regardless of its size),
and returns a ResumeSpan as appropriate.
The classic example of the problem this addresses is a table in which
each row is, say, ~1mb in size. A full table scan will currently fetch
data from KV in batches of 10k, causing at least 10GB of data held in
memory at any given moment. This sort of thing does happen in practice;
we have a long-failing roachtest #33660 because of just that, and
anecdotally OOMs in production clusters are with regularity caused by
individual queries consuming excessive amounts of memory at the KV
level.
Plumbing this limit into a header field on BatchRequest and down to the
engine level will allow the batching in 10k to become byte-sized in
nature, thus avoiding one obvious source OOMs. This doesn't solve #19721
in general (many such requests could work together to consume a lot of
memory after all), but it's a sane baby step that might just avoid a
large portion of OOMs already.
Release note: None