From 4a8d69052d3e90b80f572b49e7c33f15c2a1b144 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Fri, 24 Jan 2020 13:26:29 +0100 Subject: [PATCH] [wip] engine: add byte limit to pebbleMVCCScanner TODO: due to the RocksDB/pebble modes of operation, there are currently two ways to do anything. Thanks to the two possible response types for batches (KVs vs bytes) there is another factor of two. In short, I expect to have to make three more similar changes to fully be able to implement byte hints for scans. TODO: testing and potentially write the code in a saner way. ---- 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 provides the necessary functionality to pebbleMVCCScanner: The new maxBytes field stops the scan once the size of the result meets or exceeds the threshold (at least one key will be added, regardless of its size). 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 --- pkg/storage/engine/pebble_mvcc_scanner.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/pkg/storage/engine/pebble_mvcc_scanner.go b/pkg/storage/engine/pebble_mvcc_scanner.go index 0c4cdb207fb4..dfaef62b5f37 100644 --- a/pkg/storage/engine/pebble_mvcc_scanner.go +++ b/pkg/storage/engine/pebble_mvcc_scanner.go @@ -33,6 +33,7 @@ const ( // expected by MVCCScanDecodeKeyValue. type pebbleResults struct { count int64 + bytes int64 repr []byte bufs [][]byte } @@ -76,6 +77,7 @@ func (p *pebbleResults) put(key MVCCKey, value []byte) { encodeKeyToBuf(p.repr[startIdx+kvLenSize:startIdx+kvLenSize+lenKey], key, lenKey) copy(p.repr[startIdx+kvLenSize+lenKey:], value) p.count++ + p.bytes += int64(lenToAdd) } func (p *pebbleResults) finish() [][]byte { @@ -98,6 +100,9 @@ type pebbleMVCCScanner struct { ts hlc.Timestamp // Max number of keys to return. maxKeys int64 + // Stop adding keys once p.result.bytes matches or exceeds this threshold, + // if nonzero. + maxBytes int64 // Transaction epoch and sequence number. txn *roachpb.Transaction txnEpoch enginepb.TxnEpoch @@ -524,6 +529,13 @@ func (p *pebbleMVCCScanner) addAndAdvance(val []byte) bool { // to include tombstones in the results. if len(val) > 0 || p.tombstones { p.results.put(p.curMVCCKey(), val) + if p.maxBytes > 0 && p.results.bytes >= p.maxBytes { + // Hacky way to implement maxBytes: once we're above threshold, + // 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). + p.maxKeys = p.results.count + } if p.results.count == p.maxKeys { return false }