Join GitHub today
GitHub is home to over 28 million developers working together to host and review code, manage projects, and build software together.
Sign updistsqlrun: optimize merge joiner memory account usage #30924
Conversation
changangela
requested a review
from
jordanlewis
Oct 3, 2018
changangela
requested review from
cockroachdb/distsql-prs
as
code owners
Oct 3, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
changangela
reviewed
Oct 3, 2018
•
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 120 at r1 (raw file):
ret := s.curGroup[:n:n] s.curGroup = s.curGroup[:0] s.memAcc.ResizeTo(evalCtx.Context, int64(0))
Instead of resizing the memAcc to 0, we can achieve significant improvement in speed if we if we reserve a minimum size for the MergeJoiner. ie. we grow the account to merge_joiner_min_size at the start, and then s.memAcc.ResizeTo(evalCtx.Context, int64(merge_joiner_min_size)) here. Thoughts?
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
jordanlewis
Oct 3, 2018
Member
Instead resizing the to a 0, we can achieve significant improvement in speed if we if we reserve a minimum size for the MergeJoiner. ie. we grow the account to merge_joiner_min_size at the start, and then s.memAcc.ResizeTo(evalCtx.Context, int64(merge_joiner_min_size)) here. Thoughts?
That does seem good, but why is that such an improvement?
That does seem good, but why is that such an improvement? |
changangela
reviewed
Oct 3, 2018
•
Since the MergeJoiner is constantly re-requesting for blocks of memory, a lot of the overhead can be reduced if we "allocate" some memory right at the beginning of the merge join and hold onto it until the end. I've updated the code to have the streamGroupAccumulator allocate a single block of memory right at the start of the process, and updated the benchmark above.
Reviewable status:
complete! 0 of 0 LGTMs obtained
changangela
requested a review
from cockroachdb/sql-rest-prs
as a
code owner
Oct 4, 2018
jordanlewis
requested a review
from
solongordon
Oct 4, 2018
solongordon
requested changes
Oct 4, 2018
The general approach makes sense to me. Some questions about implementation details.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 120 at r1 (raw file):
Previously, changangela (Angela Chang) wrote…
Instead resizing the to a 0, we can achieve significant improvement in speed if we if we reserve a minimum size for the MergeJoiner. ie. we grow the account to
merge_joiner_min_sizeat the start, and thens.memAcc.ResizeTo(evalCtx.Context, int64(merge_joiner_min_size))here. Thoughts?
Could this same thing be achieved at the BytesMonitor level by reserving minAllocated up front?
pkg/sql/distsqlrun/stream_group_accumulator.go, line 63 at r2 (raw file):
func (s *streamGroupAccumulator) start(ctx context.Context) { s.src.Start(ctx) s.memAcc.AllocateMinimum(ctx, 1)
Could you explain the choice of 1 here? I'm wondering if this should be related to row size. Otherwise if you have rows larger than poolAllocationSize, your optimization will never kick in.
pkg/util/mon/bytes_usage.go, line 452 at r2 (raw file):
// decreases as used increases (and vice-versa). reserved int64 minAllocated int64
Could use a comment here.
pkg/util/mon/bytes_usage.go, line 481 at r2 (raw file):
} // AllocateMinimum allocates a minimum allocated size (reserved + used) for the
SetMinAllocated might be clearer.
pkg/util/mon/bytes_usage.go, line 484 at r2 (raw file):
// account. The account would not be able to Shrink() unless it has surpassed // this minimum allocated value. func (b *BoundAccount) AllocateMinimum(ctx context.Context, blocks int64) {
I wonder if blocks is the best interface here. It's hard to know what the right blocks value is without knowing what the pool allocation size is. Maybe better to just specify it in bytes?
pkg/util/mon/bytes_usage.go, line 486 at r2 (raw file):
func (b *BoundAccount) AllocateMinimum(ctx context.Context, blocks int64) { if blocks >= 0 { b.minAllocated = blocks * DefaultPoolAllocationSize
Should this be b.mon.poolAllocationSize?
pkg/util/mon/bytes_usage.go, line 501 at r2 (raw file):
b.used = 0 b.reserved = 0 b.minAllocated = 0
I'm not sure if it makes sense to zero this out in Clear since it's more of a configuration setting than a state variable. I guess it depends on how Clear is generally used.
changangela
reviewed
Oct 4, 2018
•
Thanks @solongordon for the review!
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 120 at r1 (raw file):
Previously, solongordon wrote…
Could this same thing be achieved at the
BytesMonitorlevel by reservingminAllocatedup front?
Hmm, the reason I thought it was best to put it into the BoundAccount instead of (*BytesMonitor).releaseBytes was because function calling is quite expensive in this part of the code (ie. one level deeper function call decreases the benchmark speed by ~5%), and so it might take little longer before the Shrink decides to stop shrinking. Is this what you had in mind?
pkg/sql/distsqlrun/stream_group_accumulator.go, line 63 at r2 (raw file):
Previously, solongordon wrote…
Could you explain the choice of
1here? I'm wondering if this should be related to row size. Otherwise if you have rows larger thanpoolAllocationSize, your optimization will never kick in.
Yeah, I was actually just investigating that because using 1 optimized the joiner significantly. It turns out that in Shrink(), we were doing a (vacuously true) check on:
if b.reserved >= b.mon.poolAllocationSize {
b.mon.releaseBytes(ctx, b.reserved-b.mon.poolAllocationSize)
b.reserved = b.mon.poolAllocationSize
}
And, a very significant amount of overhead was due to the >=. If we change it to > then the block size of 1 becomes irrelevant. I think it makes sense for minAllocated to be related to row size in for the merge joiner though, will work on that.
pkg/util/mon/bytes_usage.go, line 452 at r2 (raw file):
Previously, solongordon wrote…
Could use a comment here.
Done.
pkg/util/mon/bytes_usage.go, line 481 at r2 (raw file):
Previously, solongordon wrote…
SetMinAllocatedmight be clearer.
Done.
pkg/util/mon/bytes_usage.go, line 484 at r2 (raw file):
Previously, solongordon wrote…
I wonder if
blocksis the best interface here. It's hard to know what the right blocks value is without knowing what the pool allocation size is. Maybe better to just specify it in bytes?
I think it might be important to enforce that the minAllocated is some multiple of poolAllocationSize because of the way Shrink() is related to poolAllocationSize. For example, if the client chooses 20479 (poolAllocationSize * 2 - 1) bytes, then this is logically equivalent to using 10240 (poolAllocationSize) which might be confusing. What do you think?
pkg/util/mon/bytes_usage.go, line 486 at r2 (raw file):
Previously, solongordon wrote…
Should this be
b.mon.poolAllocationSize?
Done. I'm not too sure what the difference is :P
pkg/util/mon/bytes_usage.go, line 501 at r2 (raw file):
Previously, solongordon wrote…
I'm not sure if it makes sense to zero this out in
Clearsince it's more of a configuration setting than a state variable. I guess it depends on howClearis generally used.
Done. Yeah that makes sense.
solongordon
reviewed
Oct 4, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 120 at r1 (raw file):
Previously, changangela (Angela Chang) wrote…
Hmm, the reason I thought it was best to put it into the
BoundAccountinstead of(*BytesMonitor).releaseByteswas because function calling is quite expensive in this part of the code (ie. one level deeper function call decreases the benchmark speed by ~5%), and so it might take little longer before theShrinkdecides to stop shrinking. Is this what you had in mind?
Sorry, I meant to say BoundAccount not BytesMonitor. I was just thinking that for your suggestion above, you could implement that in BoundAccount rather than the MergeJoiner itself and keep things simpler. Though perhaps you are trying to avoid the function calls to Grow?
pkg/util/mon/bytes_usage.go, line 484 at r2 (raw file):
Previously, changangela (Angela Chang) wrote…
I think it might be important to enforce that the
minAllocatedis some multiple ofpoolAllocationSizebecause of the wayShrink()is related topoolAllocationSize. For example, if the client chooses20479(poolAllocationSize * 2 - 1) bytes, then this is logically equivalent to using10240(poolAllocationSize) which might be confusing. What do you think?
Yeah, I see your point, though of the two I think it's more confusing to specify blocks without knowing what the block size is. And the concept of blocks is an implementation detail of the monitor so it feels weird to expose it. I'm not sure there's much harm in reserving more than specified, since that's already the case in calls to Grow.
pkg/util/mon/bytes_usage.go, line 486 at r2 (raw file):
Previously, changangela (Angela Chang) wrote…
Done. I'm not too sure what the difference is :P
Looks like the default value can be overridden via the increment parameter in MakeMonitor.
solongordon
reviewed
Oct 4, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 120 at r1 (raw file):
Previously, solongordon wrote…
Sorry, I meant to say
BoundAccountnotBytesMonitor. I was just thinking that for your suggestion above, you could implement that inBoundAccountrather than theMergeJoineritself and keep things simpler. Though perhaps you are trying to avoid the function calls toGrow?
Oops, didn't realize the original comment referred to what was already implemented. Disregard.
changangela
reviewed
Oct 8, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 63 at r2 (raw file):
Previously, changangela (Angela Chang) wrote…
Yeah, I was actually just investigating that because using
1optimized the joiner significantly. It turns out that inShrink(), we were doing a (vacuously true) check on:if b.reserved >= b.mon.poolAllocationSize { b.mon.releaseBytes(ctx, b.reserved-b.mon.poolAllocationSize) b.reserved = b.mon.poolAllocationSize }And, a very significant amount of overhead was due to the
>=. If we change it to>then the block size of1becomes irrelevant. I think it makes sense forminAllocatedto be related to row size in for the merge joiner though, will work on that.
Done.
pkg/util/mon/bytes_usage.go, line 484 at r2 (raw file):
Previously, solongordon (Solon) wrote…
Yeah, I see your point, though of the two I think it's more confusing to specify blocks without knowing what the block size is. And the concept of blocks is an implementation detail of the monitor so it feels weird to expose it. I'm not sure there's much harm in reserving more than specified, since that's already the case in calls to
Grow.
Done.
changangela
reviewed
Oct 8, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/util/mon/bytes_usage.go, line 501 at r2 (raw file):
Previously, changangela (Angela Chang) wrote…
Done. Yeah that makes sense.
Since we're now reserving the minAllocated memory right on call to SetMinAllocated, I think it makes sense to reset minAllocated when used + reserved = 0
solongordon
requested changes
Oct 8, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 126 at r3 (raw file):
s.curGroup = s.curGroup[:0] if totalSize > s.memAcc.Allocated() {
Hm, I'm worried this approach is too greedy now. If I understand this right, you now have a BoundAccount which only ever reserves more bytes without releasing any until it is closed. That means if there are many small groups with one large one, we'll reserve the large group size until the merge join is complete.
Maybe there is a middle ground, like setting minAllocated to the size of one row?
changangela
reviewed
Oct 8, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 126 at r3 (raw file):
Previously, solongordon (Solon) wrote…
Hm, I'm worried this approach is too greedy now. If I understand this right, you now have a BoundAccount which only ever reserves more bytes without releasing any until it is closed. That means if there are many small groups with one large one, we'll reserve the large group size until the merge join is complete.
Maybe there is a middle ground, like setting minAllocated to the size of one row?
Done.
changangela
reviewed
Oct 8, 2018
Fixed SetMinAllocated to release bytes if the new minAllocated shrinks below the current used. Also added some tests for the bytes_usage functions.
Reviewable status:
complete! 0 of 0 LGTMs obtained
solongordon
requested changes
Oct 9, 2018
Nice, a few last comments but this is looking good.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/sql/distsqlrun/stream_group_accumulator.go, line 59 at r4 (raw file):
ordering: ordering, memAcc: memMonitor.MakeBoundAccount(), minAllocatedSet: false,
Nit: This line isn't really necessary since false is the zero value for bools. No harm having it though.
pkg/util/mon/bytes_usage.go, line 491 at r4 (raw file):
if size >= 0 { b.minAllocated = b.mon.roundSize(size) }
You probably want to set minAllocated = 0 if size == 0, right?
pkg/util/mon/bytes_usage.go, line 508 at r4 (raw file):
b.mon.releaseBytes(ctx, released) b.reserved -= released }
The logic above looks good to me after some staring, though it would be nice to add some basic unit tests to cover the different cases here, explicitly checking that the b.reserved value is as expected after SetMinAllocated is called.
changangela
reviewed
Oct 9, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/util/mon/bytes_usage.go, line 491 at r4 (raw file):
Previously, solongordon (Solon) wrote…
You probably want to set
minAllocated = 0ifsize == 0, right?
Yeah, mon.roundSize should take care of that properly.
pkg/util/mon/bytes_usage.go, line 508 at r4 (raw file):
Previously, solongordon (Solon) wrote…
The logic above looks good to me after some staring, though it would be nice to add some basic unit tests to cover the different cases here, explicitly checking that the
b.reservedvalue is as expected afterSetMinAllocatedis called.
I've added some tests to cover the different logic paths in bytes_usage_test.
changangela
reviewed
Oct 9, 2018
Updated the benchmark results in the comments.
Reviewable status:
complete! 0 of 0 LGTMs obtained
solongordon
approved these changes
Oct 9, 2018
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale)
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
|
bors r+ |
bot
pushed a commit
that referenced
this pull request
Oct 9, 2018
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
craig
bot
commented
Oct 9, 2018
Build succeeded |
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
changangela
Oct 9, 2018
Collaborator
@knz we are thinking of backporting this commit, would you be able to take a look just in case we missed something?
|
@knz we are thinking of backporting this commit, would you be able to take a look just in case we missed something? |
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
|
will look tomorrow |
knz
reviewed
Oct 10, 2018
Friends, there are several things here that needs some additional work before I can even start to evaluate it for a backport.
I appreciate this is still early steps for Angela so please assume kindness in my words, I promise we'll take this easy and make it an opportunity to learn. Cheers!
The changes I am strongly suggesting (let me be honest, it's "requesting" at this point) can be brought in a separate PR. Ping me when that's created, I'll continue the discussion there.
Reviewed 2 of 5 files at r2, 1 of 4 files at r3, 3 of 3 files at r5.
Reviewable status:complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/util/mon/bytes_usage.go, line 453 at r5 (raw file):
reserved int64 // minAllocated is a minimum allocated bytes size that the account reach // before being able to release bytes.
This comment is:
-
unclear about what this is trying to achieve (I don't understand the sentence)
-
is not in the right place: the comment above (for BoundAccount) and the comment above that (at the start of the file) must thoroughly document the "big picture" and the protocol to use these data structures. This PR is making a rather major change in the protocol, and it is therefore absolutely required to update the module doc to reflect this change. The extended commentS will need exampleS to
a) justify this change
b) show how to use the new interface -
the commentS you are going to use must describe more-or-less axiomatically the mathematical relationship between reserved, used, minAllocated, the size requested to the monitor so far. I don't see this relationship stated/specified anywhere, so I can't evaluate whether your code preserves the required invariants.
-
in general - this single one liner here is increasing the size of a pretty common data structure by 25%. I'd like to see alloc benchmarks beyond the single-use test you've added, for example a couple of the standard SQL benchmarks, to properly confirmed this addition is amortized throughout CockroachDB.
pkg/util/mon/bytes_usage.go, line 475 at r5 (raw file):
// Allocated returns the total number of bytes which this account is using or // reserving
period missing.
pkg/util/mon/bytes_usage.go, line 486 at r5 (raw file):
// SetMinAllocated allocates a minimum Allocated size (reserved + used) for the // account. The account would not be able to Shrink() unless it has surpassed
"The account would not be able to ..."
-
what happens if one tries?
-
why is this restriction put in place? (see my comment above)
pkg/util/mon/bytes_usage.go, line 490 at r5 (raw file):
func (b *BoundAccount) SetMinAllocated(ctx context.Context, size int64) error { if size < 0 { panic(fmt.Sprintf("%s: cannot set bound account min allocated to a negative value",
Don't use panic in functions that return error. Instead use pgerror.NewErrorf(pgerror.CodeInternalError, "programming error: ...")
pkg/util/mon/bytes_usage.go, line 493 at r5 (raw file):
b.mon.name)) } b.minAllocated = b.mon.roundSize(size)
You may have noticed that every function larger than ~10 LOCs in this file has comments in the body of the function to explain what is going on.
Aside (not specific to this code, but to give you insight into the culture): the purpose of exhaustive comments (and exhaustive commit messages, and PR descriptions) is not to just be nice and polite. The purpose is to establish a baseline about your own expectations (as a human) about what you (as a human) intend your code to do. Then the reviewer can compare your stated intent with the actual realization, and properly recognize any divergences as bugs.
The problem we're trying to avoid this way is when the code appears correct on its face but is not actually what you intended. Unless you tell us what you intend these bugs would survive review.
So yeah before I can review all this work I need a lot more explanation (in writing! in the code, commit message and PR description) about:
- what you're intending to do here, why you're intending it
- what you are doing and how you expect it to work
- how life is going to be different for users of this module from now on
pkg/util/mon/bytes_usage.go, line 661 at r5 (raw file):
// reserveBytes(). func (mm *BytesMonitor) releaseBytes(ctx context.Context, sz int64) { if sz == 0 {
Please justify this conditional. When this code was designed, the sz = 0 case was sufficiently uncommon.
When you add a conditional on a hot path, you're making every call of that method pay the price of this conditional, even when the condition is rarely taken. The benefit when the condition is true must be so large that it properly outweighs/amortizes the cost paid on every other call. Is that the case here?
This comment has been minimized.
Show comment
Hide comment
This comment has been minimized.
changangela
Oct 10, 2018
Collaborator
Thanks @knz for the review! I am working on the changes and will create another PR.
|
Thanks @knz for the review! I am working on the changes and will create another PR. |
changangela commentedOct 3, 2018
•
edited
Fixes #30687.
Instead of opening and closing the memory account on every single row of the merge join, we can allocate a certain minimum block size on the memory account. This way, we are not constantly requesting and releasing memory.
Benchmark results for
BenchmarkMergeJoiner: