Skip to content

engine: add byte limit to MVCCScan#44339

Merged
craig[bot] merged 1 commit intocockroachdb:masterfrom
tbg:pebble-bytes-limit
Feb 3, 2020
Merged

engine: add byte limit to MVCCScan#44339
craig[bot] merged 1 commit intocockroachdb:masterfrom
tbg:pebble-bytes-limit

Conversation

@tbg
Copy link
Copy Markdown
Member

@tbg tbg commented Jan 24, 2020

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

@tbg tbg requested a review from itsbilal January 24, 2020 12:38
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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: :shipit: 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.

@itsbilal
Copy link
Copy Markdown
Contributor

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.

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 27, 2020 via email

@petermattis
Copy link
Copy Markdown
Collaborator

petermattis commented Jan 27, 2020 via email

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 28, 2020

It may not be too much more effort to sprinkle in if p.result.bytes >= p.maxBytes && p.maxBytes > 0

@itsbilal that's what I did initially, but it was... hard, because maxKeys has this statemachine-like property where the behavior is different exactly when result.count == maxKeys (but then you need to be able to get the resume key). A cleaner way to structure this would be nice, though I'm just going to go with the current approach to avoid getting sidetracked.

@tbg tbg force-pushed the pebble-bytes-limit branch 2 times, most recently from bd852dc to da7977d Compare January 30, 2020 11:44
@tbg tbg changed the title [wip] engine: add byte limit to pebbleMVCCScanner engine: add byte limit to MVCCScan Jan 30, 2020
@tbg tbg force-pushed the pebble-bytes-limit branch from da7977d to 4346dac Compare January 30, 2020 11:52
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 30, 2020

This PR is now ready for review in earnest.

@@ -0,0 +1,58 @@
## Test opts.TargetBytes.
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The testing seems a little on the light side to me, accepting suggestions on what else to test.

Copy link
Copy Markdown
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: though @itsbilal should give this a review as well.

Reviewable status: :shipit: 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.

Copy link
Copy Markdown
Contributor

@itsbilal itsbilal left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: thanks!

Reviewed 10 of 10 files at r2.
Reviewable status: :shipit: 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 tbg requested a review from petermattis February 3, 2020 08:49
Copy link
Copy Markdown
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTRs! I souped up the tests with reverse iteration and tombstones (and the combination of both). Everything worked as expected.

Reviewable status: :shipit: 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.

@tbg tbg force-pushed the pebble-bytes-limit branch from 4346dac to 8cc3a87 Compare February 3, 2020 08:50
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
@tbg tbg force-pushed the pebble-bytes-limit branch from 8cc3a87 to 14b6cc3 Compare February 3, 2020 09:36
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Feb 3, 2020

bors r=petermattis,itsbilal

craig bot pushed a commit that referenced this pull request Feb 3, 2020
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>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Feb 3, 2020

Build succeeded

@craig craig bot merged commit 14b6cc3 into cockroachdb:master Feb 3, 2020
@tbg tbg deleted the pebble-bytes-limit branch February 3, 2020 11:58
@jordanlewis
Copy link
Copy Markdown
Member

woo!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants