Skip to content
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: be tighter with allocations when TargetBytes is present #84319

Merged
merged 1 commit into from
Jul 22, 2022

Conversation

yuzefovich
Copy link
Member

@yuzefovich yuzefovich commented Jul 12, 2022

Previously, while putting the key into the repr, we could make an
allocation that was too large given the remaining TargetBytes budget.
This is the case since we're exponentially increasing the capacities of
the buffers until 128MiB and because we're only accounting for the
length of the slice even though the whole capacity would have a memory
footprint.

For example, with 10MiB of TargetBytes (which is used by SQL in many
cases) and a ScanResponse that exceeds that limit, we'd allocate
capacities that are powers of two, starting at, say, 256B, and would go
all the way up to 8MiB; however, given that 10MiB limit, we'd only use
2MiB of that last 8MiB repr when we encounter the target bytes limit
and stop. Such behavior is kinda ok if the response is marshalled by the
gRPC to be sent to the other node, but it is quite unfortunate in the
local fast-path cases (meaning the SQL and the KV are part of the same
physical machine). In the latter scenario SQL would only account for the
lengths of slices while keeping the whole slices alive for a while,
leading to significant unaccounted for memory. In the example above, on
the order of 6MiB would be unaccounted for - multiply that by some
concurrency, and we have unaccounted memory on the order of hundreds of
MiBs.

The current behavior seems especially bad for the streamer use case
where we issue many requests with the TargetBytes set and use
ScanResponse.NumBytes field (which tracks the lengths of the slices)
for the memory accounting purposes.

In order to improve here, this commit teaches put method about the
maximum capacity it can use. In the example above, the last slice would
be on the order of 2MiB making everything happy: we stay very close to
TargetBytes limit and don't have any wasted space.

Addresses: #64906.
Addresses: #82160.

Release note: None

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@yuzefovich
Copy link
Member Author

An example of the current behavior, only 10MiB are accounted for whereas the actual footprint is 16MiB:
Screen Shot 2022-07-12 at 1 46 55 PM

@yuzefovich yuzefovich changed the title kv: remove TargetBytesAvoidExcess version gate storage: be tighter with allocations when TargetBytes is present Jul 12, 2022
@yuzefovich yuzefovich force-pushed the target-bytes branch 2 times, most recently from 19f8757 to 8090e93 Compare July 13, 2022 02:03
@yuzefovich yuzefovich marked this pull request as ready for review July 13, 2022 02:04
@yuzefovich yuzefovich requested review from a team as code owners July 13, 2022 02:04
@yuzefovich
Copy link
Member Author

I'm curious to hear what people think and very open to other solutions, but I believe we should fix the oversight in one way or another.

@yuzefovich
Copy link
Member Author

Pulled the first commit into a separate PR so that we can focus here on the main change. PTAL.

Copy link
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

Thanks! This is a nice and simple improvement, but I think it can still underestimate the memory use quite significantly in pathological cases where we're only using half of each buffer. Ideally, I suppose we'd want these to form a continuous buffer where KVs can straddle the bounds to avoid any wasted memory. Maybe write up an issue if that's something we want to pursue.

p.resumeKey = trimmedKey
}
}
p.err = keyNotAdded()
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: found this a bit confusing when skimming the code, since it reads almost like an error constructor. Maybe something like this would be clearer:

p.resumeKey, p.err = maybeTrimResult(key)

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewed all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @yuzefovich)


pkg/storage/pebble_mvcc_scanner.go line 1032 at r4 (raw file):

		}
	}

There is a bunch of targetBytes and maxKeys logic in this function that is making it harder to understand. How about a comment like the following

// We are here due to one of the following cases:
// A. No limits were exceeded
// B. Limits were exceeded, but we need to put a key, so mustPutKey = true
// For B we will never set maxNewSize.
// For A, we may set maxNewSize, but we already know that p.targetBytes > p.results.bytes. So maxNewSize > 0. However, maxNewSize may not be enough to hold the result, in which case added can be false.

Don't we have a bug in this new logic in that case? In that we will call keyNotAdded below without doing the mustPutKey dance, so we could remove everything even if !p.allowEmpty?
If we lifted the following logic above the code that handles p.resumeReason != 0 then it seems correct:

if p.resumeReason == 0 {
	var maxNewSize int
	if p.targetBytes > 0 && p.targetBytes > p.results.bytes {
		maxNewSize = int(p.targetBytes - p.results.bytes)
	}
	if added, err := p.results.put(ctx, rawKey, rawValue, p.memAccount, maxNewSize); err != nil {
		p.err = errors.Wrapf(err, "scan with start key %s", p.start)
		return false
	}
        if added {
           p.advanceKey()
        } else {
           p.resumeReason = roachpb.RESUME_BYTE_LIMIT
        }
}
// INVARIANT: p.resumeReason != 0.

Also the interplay between p.targetBytesAvoidExcess and the maxNewSize is not very clear. It seems we are ok with exceeding the size when !p.targetBytesAvoidExcess, but then this put allocation behavior may prevent an excess. Am I understanding correctly? A code comment would help.

@yuzefovich yuzefovich force-pushed the target-bytes branch 2 times, most recently from 43bba74 to 150b228 Compare July 18, 2022 16:40
Copy link
Member Author

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)


pkg/storage/pebble_mvcc_scanner.go line 1032 at r4 (raw file):

// A. No limits were exceeded

This is not right - we are in this branch only if p.resumeReason != 0 meaning that the limits were exceeded.

We only proceed to try to put() the current key while restricting the new capacity only if the limits were not exceeded. This actually means that adding the current key into the result won't put us over targetBytes limit, so we don't have to worry about !p.allowEmpty case - the put will succeed. This is ensured by

p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) {

I added some more commentary, let me know what you think.

Also the interplay between p.targetBytesAvoidExcess and the maxNewSize is not very clear. It seems we are ok with exceeding the size when !p.targetBytesAvoidExcess, but then this put allocation behavior may prevent an excess. Am I understanding correctly? A code comment would help.

You're right that !p.targetBytesAvoidExcess case needs some attention. However, we are removing this field in #84519 now because p.targetBytesAvoidExcees is always true on 22.2 version (it was only kept around for 22.1-22.2 mixed version cluster setup), so we can actually ignore !p.targetBytesAvoidExcess case. I initially included the corresponding commit into this PR but decided to pull it out.

Copy link
Member Author

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Good point. Filed #84594 to track that, but it seems such a scenario should occur rarely in practice, so I'm ok with that being addressed (if ever) later.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)


pkg/storage/pebble_mvcc_scanner.go line 1032 at r4 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

// A. No limits were exceeded

This is not right - we are in this branch only if p.resumeReason != 0 meaning that the limits were exceeded.

We only proceed to try to put() the current key while restricting the new capacity only if the limits were not exceeded. This actually means that adding the current key into the result won't put us over targetBytes limit, so we don't have to worry about !p.allowEmpty case - the put will succeed. This is ensured by

p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) {

I added some more commentary, let me know what you think.

Also the interplay between p.targetBytesAvoidExcess and the maxNewSize is not very clear. It seems we are ok with exceeding the size when !p.targetBytesAvoidExcess, but then this put allocation behavior may prevent an excess. Am I understanding correctly? A code comment would help.

You're right that !p.targetBytesAvoidExcess case needs some attention. However, we are removing this field in #84519 now because p.targetBytesAvoidExcees is always true on 22.2 version (it was only kept around for 22.1-22.2 mixed version cluster setup), so we can actually ignore !p.targetBytesAvoidExcess case. I initially included the corresponding commit into this PR but decided to pull it out.

I temporarily cherry-picked the commit that removes targetBytesAvoidExcess logic into this PR to ease the discussion. Also note that the existing tests were, in fact, failing for !p.targetBytesAvoidExcess case.

@yuzefovich yuzefovich removed the do-not-merge bors won't merge a PR with this label. label Jul 18, 2022
@yuzefovich
Copy link
Member Author

Rebased on top of latest master which contains the removal of targetBytesAvoidExcess business.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @yuzefovich)


pkg/storage/pebble_mvcc_scanner.go line 1032 at r4 (raw file):

This is not right - we are in this branch only if p.resumeReason != 0 meaning that the limits were exceeded.

I was referring to the code after the if-block, i.e., a comment that would precede all of the code after the if-block.

We only proceed to try to put() the current key while restricting the new capacity only if the limits were not exceeded. This actually means that adding the current key into the result won't put us over targetBytes limit, so we don't have to worry about !p.allowEmpty case - the put will succeed. This is ensured by ...

So we know maxNewSize is sufficient because lenToAdd := p.sizeOf(lenKey, lenValue) used by put is the same computation used in in the avoid-excess case of p.results.sizeOf(len(rawKey), len(rawValue). Relying on such matching code makes me slightly nervous -- there should be at least some invariant checking to ensure that the code in the 2 places doesn't slightly diverge so that the spare capacity in the former doesn't appear slightly larger than the latter and cause the latter (put) to fail.

	if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess &&
		p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) {

You're right that !p.targetBytesAvoidExcess case needs some attention. ...
Also note that the existing tests were, in fact, failing for !p.targetBytesAvoidExcess case.

That's the one where I thought there was a bug since maxNewSize was not guaranteed to hold the key-value pair. So glad to hear that our existing tests covered that case and were failing.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker, @sumeerbhola, and @yuzefovich)


pkg/storage/pebble_mvcc_scanner.go line 131 at r6 (raw file):

				newSize *= 2
			}
		}

The old size code was hard to follow for me, and now it is even harder. How about something like:

// Exponential increase by default, while ensuring that we
// respect a hard lower bound of lenToAdd and the soft upper bound of
// maxSize and the hard upper bound of maxNewSize (if set).
if maxNewSize > 0 && maxNewSize < lenToAdd {
  // Hard upper bound is greater than hard lower bound, so cannot add.
  return false, nil
}
// Exponential growth to ensure newSize >= lenToAdd.
newSize := 2 * cap(p.repr)
if newSize == 0 || newSize > maxSize {
  // If the previous buffer exceeded maxSize, we don't double its capacity
  // for next allocation, and instead reset the exponential increase, in
  // case we had a stray huge key-value.
  newSize = minSize
}
for newSize < lenToAdd {
  newSize *= 2
}
// Respect soft upper-bound before hard lower-bound, since it could be lower than hard lower-bound.
if newSize > maxSize {
  newSize = maxSize
}
// Respect hard upper-bound.
if maxNewSize > 0 && newSize > maxNewSize {
  newSize = maxNewSize
}
// Now respect hard lower-bound.
if newSize < lenToAdd {
  newSize = lenToAdd
}


pkg/storage/pebble_mvcc_scanner.go line 1047 at r6 (raw file):

	var maxNewSize int
	if p.targetBytes > 0 && p.targetBytes > p.results.bytes && !mustPutKey {

// INVARIANT: !mustPutKey => maxNewSize is sufficient for key-value pair.


pkg/storage/pebble_mvcc_scanner.go line 1053 at r6 (raw file):

		p.err = errors.Wrapf(err, "scan with start key %s", p.start)
		return false
	} else if !added {

isn't this !added case a failure of an internal code invariant?

Copy link
Member Author

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)


pkg/storage/pebble_mvcc_scanner.go line 1032 at r4 (raw file):

Previously, sumeerbhola wrote…

This is not right - we are in this branch only if p.resumeReason != 0 meaning that the limits were exceeded.

I was referring to the code after the if-block, i.e., a comment that would precede all of the code after the if-block.

We only proceed to try to put() the current key while restricting the new capacity only if the limits were not exceeded. This actually means that adding the current key into the result won't put us over targetBytes limit, so we don't have to worry about !p.allowEmpty case - the put will succeed. This is ensured by ...

So we know maxNewSize is sufficient because lenToAdd := p.sizeOf(lenKey, lenValue) used by put is the same computation used in in the avoid-excess case of p.results.sizeOf(len(rawKey), len(rawValue). Relying on such matching code makes me slightly nervous -- there should be at least some invariant checking to ensure that the code in the 2 places doesn't slightly diverge so that the spare capacity in the former doesn't appear slightly larger than the latter and cause the latter (put) to fail.

	if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess &&
		p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) {

You're right that !p.targetBytesAvoidExcess case needs some attention. ...
Also note that the existing tests were, in fact, failing for !p.targetBytesAvoidExcess case.

That's the one where I thought there was a bug since maxNewSize was not guaranteed to hold the key-value pair. So glad to hear that our existing tests covered that case and were failing.

I see. I added the comments you were suggesting earlier given that we don't need to return that added boolean after all.


pkg/storage/pebble_mvcc_scanner.go line 131 at r6 (raw file):

Previously, sumeerbhola wrote…

The old size code was hard to follow for me, and now it is even harder. How about something like:

// Exponential increase by default, while ensuring that we
// respect a hard lower bound of lenToAdd and the soft upper bound of
// maxSize and the hard upper bound of maxNewSize (if set).
if maxNewSize > 0 && maxNewSize < lenToAdd {
  // Hard upper bound is greater than hard lower bound, so cannot add.
  return false, nil
}
// Exponential growth to ensure newSize >= lenToAdd.
newSize := 2 * cap(p.repr)
if newSize == 0 || newSize > maxSize {
  // If the previous buffer exceeded maxSize, we don't double its capacity
  // for next allocation, and instead reset the exponential increase, in
  // case we had a stray huge key-value.
  newSize = minSize
}
for newSize < lenToAdd {
  newSize *= 2
}
// Respect soft upper-bound before hard lower-bound, since it could be lower than hard lower-bound.
if newSize > maxSize {
  newSize = maxSize
}
// Respect hard upper-bound.
if maxNewSize > 0 && newSize > maxNewSize {
  newSize = maxNewSize
}
// Now respect hard lower-bound.
if newSize < lenToAdd {
  newSize = lenToAdd
}

I like it, thanks, done.


pkg/storage/pebble_mvcc_scanner.go line 1047 at r6 (raw file):

Previously, sumeerbhola wrote…

// INVARIANT: !mustPutKey => maxNewSize is sufficient for key-value pair.

Done.


pkg/storage/pebble_mvcc_scanner.go line 1053 at r6 (raw file):

Previously, sumeerbhola wrote…

isn't this !added case a failure of an internal code invariant?

Yes, you're totally right, removed.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

:lgtm:

Reviewed 1 of 1 files at r7, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @erikgrinaker and @sumeerbhola)

Previously, while `put`ting the key into the `repr`, we could make an
allocation that was too large given the remaining `TargetBytes` budget.
This is the case since we're exponentially increasing the capacities of
the buffers until 128MiB and because we're only accounting for the
length of the slice even though the whole capacity would have a memory
footprint.

For example, with 10MiB of `TargetBytes` (which is used by SQL in many
cases) and a ScanResponse that exceeds that limit, we'd allocate
capacities that are powers of two, starting at, say, 256B, and would go
all the way up to 8MiB; however, given that 10MiB limit, we'd only use
2MiB of that last 8MiB `repr` when we encounter the target bytes limit
and stop. Such behavior is kinda ok if the response is marshalled by the
gRPC to be sent to the other node, but it is quite unfortunate in the
local fast-path cases (meaning the SQL and the KV are part of the same
physical machine). In the latter scenario SQL would only account for the
lengths of slices while keeping the whole slices alive for a while,
leading to significant unaccounted for memory. In the example above, on
the order of 6MiB would be unaccounted for - multiply that by some
concurrency, and we have unaccounted memory on the order of hundreds of
MiBs.

The current behavior seems especially bad for the streamer use case
where we issue many requests with the `TargetBytes` set and use
`ScanResponse.NumBytes` field (which tracks the lengths of the slices)
for the memory accounting purposes.

In order to improve here, this commit teaches `put` method about the
maximum capacity it can use. In the example above, the last slice would
be on the order of 2MiB making everything happy: we stay very close to
TargetBytes limit and don't have any wasted space.

Release note: None
@yuzefovich
Copy link
Member Author

Adjusted the commit message a bit.

TFTRs!

bors r+

@craig
Copy link
Contributor

craig bot commented Jul 22, 2022

Build succeeded:

@craig craig bot merged commit 3adb070 into cockroachdb:master Jul 22, 2022
@yuzefovich yuzefovich deleted the target-bytes branch July 22, 2022 16:50
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.

4 participants