-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
sql: make aggregate builtins share the same memory account #46700
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What about other aggregate functions that use a memory account ( e.g. array agg and bytes concat), should those be changed too?
Reviewed 3 of 3 files at r1.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis and @yuzefovich)
pkg/sql/rowexec/aggregator.go, line 341 at r1 (raw file):
} ag.EvalCtx.SingleDatumAggMemAccount = &ag.aggFuncsAcc
Do we make a copy of this eval ctx before setting this field?
pkg/sql/sem/builtins/aggregate_builtins.go, line 490 at r1 (raw file):
type anyNotNullAggregate struct { val tree.Datum acc *mon.BoundAccount
Not sure if this makes sense but there seems to be a lot of duplicated logic. Would it be possible to extract this and datum size into a utility struct that takes care of the account resizing/closing details?
pkg/sql/sem/builtins/aggregate_builtins.go, line 546 at r1 (raw file):
func (a *anyNotNullAggregate) Reset(ctx context.Context) { a.val = tree.DNull if a.acc != nil && a.datumSize != 0 {
Why is it ok to have a nil acc here but not in Add
?
pkg/sql/sem/builtins/aggregate_builtins.go, line 1215 at r1 (raw file):
oldDatumSize := a.datumSize a.datumSize = int64(datum.Size()) if err := a.acc.Grow(ctx, a.datumSize-oldDatumSize); err != nil {
I think this is allowed but makes me wonder why we have both Grow
and Shrink
if the former accepts a negative argument.
d63022b
to
a519954
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think arrayAgg
is special because it operates on multiple datums, and it might be benefiting from having its own account. I also initially thought that concat and json aggregates should have their own accounts since they might be using non-trivial amount of memory, but it probably makes sense to unify them with others as well since they do aggregate into a single - possibly very large - datum. Done.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @jordanlewis)
pkg/sql/rowexec/aggregator.go, line 341 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Do we make a copy of this eval ctx before setting this field?
A new eval context is created in aggregatorBase.init
when calling ag.ProcessorBase.Init
. Added a comment.
pkg/sql/sem/builtins/aggregate_builtins.go, line 490 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Not sure if this makes sense but there seems to be a lot of duplicated logic. Would it be possible to extract this and datum size into a utility struct that takes care of the account resizing/closing details?
I was thinking about it myself but initially thought that this would give much benefit, but I like it how it turned out, done.
pkg/sql/sem/builtins/aggregate_builtins.go, line 546 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
Why is it ok to have a nil acc here but not in
Add
?
I don't know, I wanted to be on the safe side, but I agree, this was ill-designed. Refactored.
pkg/sql/sem/builtins/aggregate_builtins.go, line 1215 at r1 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I think this is allowed but makes me wonder why we have both
Grow
andShrink
if the former accepts a negative argument.
The difference is that if we're Grow
ing by negative amount, then the memory account's used
and reserved
fields will be updated accordingly, but we will not release any reservations to the monitor while when Shrink
ing we might release some amount of memory and, thus, reducing reserved
field.
a519954
to
a5c68f9
Compare
return nil | ||
} | ||
|
||
func (b singleDatumAggregateBase) reset(ctx context.Context) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the receivers in these methods need to be pointers otherwise accountedFor
won't be updated correctly.
a5c68f9
to
fdbb7f7
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @asubiotto and @jordanlewis)
pkg/sql/sem/builtins/aggregate_builtins.go, line 543 at r2 (raw file):
Previously, asubiotto (Alfonso Subiotto Marqués) wrote…
I think the receivers in these methods need to be pointers otherwise
accountedFor
won't be updated correctly.
I think you're right, fixed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 2 of 3 files at r2, 2 of 2 files at r3.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @asubiotto and @jordanlewis)
Release justification: fixes for high-priority or high-severity bugs in existing functionality (we could hit memory limit due to accounting long before the available RAM is actually used up). We recently fixed a couple of leaks in memory accounting by aggregate builtins. It was done in the same way that other similar aggregates were doing - by instantiating a separate memory account for each aggregate struct. However, when a single aggregate, say `anyNotNull`, wants to store a single datum, say `DInt` of size 8 bytes, when growing its own memory account will actually make a reservation of `mon.DefaultPoolAllocation = 10240` bytes although we will only be using 8 bytes. This can result in "memory-starvation" for OLAP-y queries because we're likely to hit `max-sql-memory` limit long before we're getting close to it because of such "overestimation" in the accounting. This commit fixes this problem by making all aggregates that aggregate a single datum (these are all aggregate builtins that perform memory accounting except for `arrayAgg` which works with multiple datums) to share the same memory account (when non-nil) which is plumbed via `tree.EvalContext` (this is unfortunate but, as always, seems like necessary evil). That account is instantiated by `rowexec.aggregator` and `rowexec.windower` processors. Also it is acceptable from the concurrency's point of view because the processors run in a single goroutine, so we will never have concurrent calls to grow/shrink this shared memory account. If for some reason the field for shared memory account is nil in the eval context, then we will fallback to old behavior of instantiating a memory account for each aggregate builtin struct. A helper struct was created (that is now embedded by all aggregate builtins in question) that unifies the memory accounting. Release note: None (a follow-up to a recent PR).
fdbb7f7
to
0ac5468
Compare
TFTR! bors r+ |
Build succeeded |
Release justification: fixes for high-priority or high-severity
bugs in existing functionality (we could hit memory limit due to
accounting long before the available RAM is actually used up).
We recently fixed a couple of leaks in memory accounting by aggregate
builtins. It was done in the same way that other similar aggregates were
doing - by instantiating a separate memory account for each aggregate
struct. However, when a single aggregate, say
anyNotNull
, wants tostore a single datum, say
DInt
of size 8 bytes, when growing its ownmemory account will actually make a reservation of
mon.DefaultPoolAllocation = 10240
bytes although we will only be using8 bytes. This can result in "memory-starvation" for OLAP-y queries
because we're likely to hit
max-sql-memory
limit long before we'regetting close to it because of such "overestimation" in the accounting.
This commit fixes this problem by making all aggregates that aggregate
a single datum (these are all aggregate builtins that perform memory
accounting except for
arrayAgg
which works with multiple datums) toshare the same memory account (when non-nil) which is plumbed via
tree.EvalContext
(this is unfortunate but, as always, seems likenecessary evil). That account is instantiated by
rowexec.aggregator
and
rowexec.windower
processors. Also it is acceptable from theconcurrency's point of view because the processors run in a single
goroutine, so we will never have concurrent calls to grow/shrink this
shared memory account.
If for some reason the field for shared memory account is nil in the
eval context, then we will fallback to old behavior of instantiating
a memory account for each aggregate builtin struct. A helper struct was
created (that is now embedded by all aggregate builtins in question)
that unifies the memory accounting.
Fixes: #46664.
Release note: None (a follow-up to a recent PR).