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

kvserver/rangefeed: checkpoint events can cause OOMs #114190

Closed
nicktrav opened this issue Nov 9, 2023 · 7 comments · Fixed by #120347
Closed

kvserver/rangefeed: checkpoint events can cause OOMs #114190

nicktrav opened this issue Nov 9, 2023 · 7 comments · Fixed by #120347
Assignees
Labels
A-cdc Change Data Capture A-kv-rangefeed Rangefeed infrastructure, server+client branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. O-23.2-scale-testing issues found during 23.2 scale testing O-testcluster Issues found or occurred on a test cluster, i.e. a long-running internal cluster T-cdc

Comments

@nicktrav
Copy link
Collaborator

nicktrav commented Nov 9, 2023

Describe the problem

OOMs were observed during a large scale test, specifically when testing changefeed / rangefeed performance.

Heap profiles point the finger at checkpoint events. Retained size and number of objects is far beyond what would be expected if memory budgets were being respected:

inuse_space:

Screenshot 2023-11-09 at 12 50 49 PM

inuse_objects:

Screenshot 2023-11-09 at 1 08 15 PM

Environment:

  • CockroachDB version V23.2.0-ALPHA.6-DEV
  • Server OS: Linux.

Additional context

More context here (internal).

Debugging artifacts can be found here (internal).

Jira issue: CRDB-33387

@nicktrav nicktrav added C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. O-testcluster Issues found or occurred on a test cluster, i.e. a long-running internal cluster A-kv-rangefeed Rangefeed infrastructure, server+client T-kv-replication labels Nov 9, 2023
Copy link

blathers-crl bot commented Nov 9, 2023

cc @cockroachdb/replication

@nicktrav
Copy link
Collaborator Author

nicktrav commented Nov 9, 2023

cc @cockroachdb/cdc

@erikgrinaker erikgrinaker self-assigned this Nov 9, 2023
@erikgrinaker
Copy link
Contributor

This seems legit. The changefeed processors struggled to keep up for whatever reason, and we don't subject checkpoints to memory budgets. The checkpoint events were likely sitting around in the registration channel buffers, which have a 4096-event size per replica (so 614M events with 150k replicas, which can eat up to 40 GB memory if they're all filled up -- which is very unlikely, but gives an upper bound).

event := p.newCheckpointEvent()
p.reg.PublishToOverlapping(ctx, all, event, nil)

// calculateDateEventSize returns estimated size of the event that contain actual
// data. We only account for logical ops and sst's. Those events come from raft
// and are budgeted. Other events come from processor jobs and update timestamps
// we don't take them into account as they are supposed to be small and to avoid
// complexity of having multiple producers getting from budget.
func calculateDateEventSize(e event) int64 {
var size int64
for _, op := range e.ops {
size += int64(op.Size())
}
if e.sst != nil {
size += int64(len(e.sst.data))
}
return size
}

@williamkulju williamkulju added release-blocker Indicates a release-blocker. Use with branch-release-2x.x label to denote which branch is blocked. branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 O-23.2-scale-testing issues found during 23.2 scale testing and removed O-testcluster Issues found or occurred on a test cluster, i.e. a long-running internal cluster labels Nov 10, 2023
@miretskiy
Copy link
Contributor

I don't know if this is a release blocker -- it's not a regression. @erikgrinaker ?

@erikgrinaker
Copy link
Contributor

Yeah, not a blocker. It has always been like this, and it isn't clear that a fix would be safe this late in the cycle.

@erikgrinaker erikgrinaker removed the release-blocker Indicates a release-blocker. Use with branch-release-2x.x label to denote which branch is blocked. label Nov 10, 2023
@williamkulju williamkulju added the O-testcluster Issues found or occurred on a test cluster, i.e. a long-running internal cluster label Dec 6, 2023
Copy link

blathers-crl bot commented Dec 14, 2023

cc @cockroachdb/cdc

@blathers-crl blathers-crl bot added the A-cdc Change Data Capture label Dec 14, 2023
@erikgrinaker erikgrinaker removed their assignment Dec 14, 2023
@exalate-issue-sync exalate-issue-sync bot added A-cdc Change Data Capture and removed A-cdc Change Data Capture labels Dec 14, 2023
@exalate-issue-sync exalate-issue-sync bot assigned wenyihu6 and unassigned miretskiy Jan 29, 2024
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 13, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 13, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 13, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 14, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 15, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 15, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 16, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy in this PR is over-accounting >> under-accounting. GitHub
issue (todo) tracks the remaining work to improve accuracy.

Resolves: cockroachdb#114190
Release note: None
@wenyihu6
Copy link
Contributor

wenyihu6 commented Mar 18, 2024

Each rangefeed processor has an event buffering channel eventC. These events could point to large underlying data structure, and keeping them buffered in the channel prevents the underlying data from being garbage collected. To mitigate OOMs, we previously introduced memory budget to limit event buffering to the eventC channel.

This is how the memory accounting model works currently:

Before sending an event to the eventC channel, it calculatesDataSize based on the event to predict how much memory an event would take. It tries to allocate memory from the budget feed. The event then holds onto the same memory alloc and passed to registrations as part of a shared event when publishing the events. As the registry publishes the event to registrations, it will create a new rangefeed event (e.x.https://github.com/cockroachdb/cockroach/blob/aa58da93a7fa83bbf905c2b096b9f1954d35d402/pkg/kv/kvserver/rangefeed/scheduled_processor.go#L748-L757) based on event c and a shared event holding the memory alloc. It then increases the reference count of the memory alloc. As registration publishes the event to its stream, it decreases the reference count of the memory alloc

nextEvent.alloc.Release(ctx)
. The memory is then returned back to the budget feed when reference count becomes zero
if a != nil && atomic.AddInt32(&a.refCount, -1) == 0 {
a.feed.returnAllocation(ctx, a.size)
.

However, the current memory accounting did not have a good way to predict the memory footprint of events. It uses calculateDateSize which 1. Does not account for memory of the new RangeFeedEvents that could be spawned (such as checkpoint events

func (p *ScheduledProcessor) newCheckpointEvent() *kvpb.RangeFeedEvent {
) 2. inaccurately uses protobuf-encoded wire size for data (
size += int64(op.Size())
) 3. does not account for memory of the base structs.

The challenging part of this issue is we are not sure if new rangefeed events can be spawned before buffering event e to the eventC channel.

  • Solution 1 (most precise, most complex): Ideally, we would want to allocate memory only when we know for sure that we are publishing a checkpoint event so that we are not over- or under-accounting. This is hard since once the processor sends the event to the eventC channel, it is committed to sending it unless we try to disconnect the scheduler. It cannot backoff and decide not to send it if it cannot acquire a memory budget.
  • Solution 2: We can use the processor's resolved timestamp to predict if a checkpoint event will happen. But calling p.rts.ConsumeLogicalOp for memory accounting can change internal field unresolvedIntentQueue. To resolve this, we could make a deep copy of rts, but this is on a hot path and could introduce significant overhead. (Open pr: [wip] rangefeed: improve memory accounting for event queue #120001)
  • Solution 3: We over-accounts at the beginning to account for everything that could have happened. If the memory turns out to be not needed, we de-allocates the memory right away to prevent registrations from holding onto unnecessary memory for too long. (Open pr: rangefeed: improve memory accounting for event queue #120550 ,rangefeed: add memory adjuster before publishing events #120348)
  • Solution 4 (least precise, straightforward): We roughly estimate the memory and sacrifice some precision for simplicity. We are allowing small overhead of shared event structs and checkpoint events caused by logical ops to slip away from memory accounting. (Open pr: rangefeed: improve memory accounting for event queue #120347)

Currently, we are picking solution 4 for its simplicity and will revisit this topic if anything comes up.

More details for solution 4
Instead of accounting memory for both the current event and future rangefeed events, it will always pick the maximum memory between the two. It also allows additional shared event overhead (should be very small) and checkpoint events (caused by logical ops) (should be rare) to slip away.

  • For logical ops, we always take current event memory. Current event memory is always larger if we assume no checkpoint events will be published for any logical ops. (same underlying data)
  • For checkpoint events (ct and initRTS), we always take checkpoint rangefeed event. Future events are larger. (curr: 80, future: 152) (span points to the same underlying data)
  • For sst events, we always take future rangefeed event (future base structs is larger than current base structs by 2) (same underlying data).
  • For sync events, we always take current events. (current memory usage is larger as no future events will be created.

We got these data ^ from test benchmarks documented here #120582.

wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 18, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy is:

Instead of accounting memory for both the current event and future rangefeed
events, it will always pick the maximum memory between the two. It also allows
additional shared event overhead (should be very small) and checkpoint events
(caused by logical ops) (should be rare) to slip away.

- For logical ops, we always take current event memory. Current event memory is
always larger if we assume no checkpoint events will be published for any
logical ops. (same underlying data)
- For checkpoint events (ct and initRTS), we always take checkpoint rangefeed
event. Future events are larger. (curr: 80, future: 152) (span points to the
same underlying data)
- For sst events, we always take future rangefeed event (future base structs is
larger than current base structs by 2) (same underlying data).
- For sync events, we always take current events. (current memory usage is
larger as no future events will be created.

We got these data ^ from test benchmarks documented here
cockroachdb#120582.

Resolves: cockroachdb#114190
Release note: None
wenyihu6 added a commit to wenyihu6/cockroach that referenced this issue Mar 22, 2024
To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy is:

Instead of accounting memory for both the current event and future rangefeed
events, it will always pick the maximum memory between the two. It also allows
additional shared event overhead (should be very small) and checkpoint events
(caused by logical ops) (should be rare) to slip away.

- For logical ops, we always take current event memory. Current event memory is
always larger if we assume no checkpoint events will be published for any
logical ops. (same underlying data)
- For checkpoint events (ct and initRTS), we always take checkpoint rangefeed
event. Future events are larger. (curr: 80, future: 152) (span points to the
same underlying data)
- For sst events, we always take future rangefeed event (future base structs is
larger than current base structs by 2) (same underlying data).
- For sync events, we always take current events. (current memory usage is
larger as no future events will be created.

We got these data ^ from test benchmarks documented here
cockroachdb#120582.

Resolves: cockroachdb#114190
Release note: None
craig bot pushed a commit that referenced this issue Mar 22, 2024
119671: concurrency: implement generalized lock promotion  r=nvanbenschoten a=arulajmani

First two commits are from: #119587

-----

Previously, if a request had acquired a shared lock, it wasn't able to
promote it to an Exclusive or Intent (by writing to the same key) lock.
This was because the lock table could not detect deadlock scenarios
where two transactions that both held shared locks were trying to
promote. Moreover, it also couldn't detect wait queue local deadlocks
that involved non-transactional requests.

These two limitations have now been limited. For the former, we're able
to leverage our existing deadlock detection algorithm by performing the
correct set of pushes. This is done by changing the claimantTxn concept
slightly. Previously, there could only be one claimant for a key. This
is no longer true -- now, the claimant may be different, depending on
who is asking for it.

For the latter, we reorder the wait queue to avoid preventable
deadlocks. This is done by preferring lock promoters over other
requests. The bulk of this was already done in
#118484.

Closes #110435

Release note: shared locks can now be re-acquired with higher strength.

120347: rangefeed: improve memory accounting for event queue r=erikgrinaker a=wenyihu6

To prevent OOMs, we previously introduced memory budget to limit event buffering
to processor.eventC channel. These events could point to large underlying data
structure and keeping them in the channel prevent them from being garbage
collected. However, the current memory accounting did not have a good way to
predict the memory footprint of events. It uses calculateDateSize which 1. does
not account for memory of the new RangeFeedEvents that could be spawned (such as
checkpoint events) 2. inaccurately uses protobuf-encoded wire size for data 3.
does not account for memory of the base structs.

This patch improves the memory estimation by resolving the three points above.
It tries its best effort to account for base struct's memory, predict generation
of new events, and use actual data size rather than compressed protobuf size.
Since it is challenging to predict whether there would be a new checkpoint
event, our strategy is:

Instead of accounting memory for both the current event and future rangefeed
events, it will always pick the maximum memory between the two. It also allows
additional shared event overhead (should be very small) and checkpoint events
(caused by logical ops) (should be rare) to slip away.

- For logical ops, we always take current event memory. Current event memory is
always larger if we assume no checkpoint events will be published for any
logical ops. (same underlying data)
- For checkpoint events (ct and initRTS), we always take checkpoint rangefeed
event. Future events are larger. (curr: 80, future: 152) (span points to the
same underlying data)
- For sst events, we always take future rangefeed event (future base structs is
larger than current base structs by 2) (same underlying data).
- For sync events, we always take current events. (current memory usage is
larger as no future events will be created.

We got these data ^ from test benchmarks documented here
#120582.

Resolves: #114190
Release note: None

120718: sql: fix minor procedure bugs and add UDF/SP mixed tests r=mgartner a=mgartner

#### sql: fix bugs with procedures passing arguments to other procedures

This commit fixes a limitation where a procedure could not pass an
argument through to another procedure, like:

    CREATE PROCEDURE p(n INT) LANGUAGE SQL AS $$
      CALL other_proc(n);
    $$

To lift this restriction, the body of a procedure must be type-checked
with respect to the current scope which contains the procedures named
parameters. This change required a few subsequent changes to prevent
regressions:

  1. Copying some logic that ensures that `CALL` arguments are not
     subqueries to `optbuilder.subquery.TypeCheck`.
  2. Using `FuncExpr.InCall` instead of `CallAncestor` to determine the
     correct verbiage for error messages of procedures. `CallAncestor`
     is no longer used and has been removed.

Epic: None

Release note: None

#### sql/logictest: add tests for UDFs and SPs calling each other

Release note: None


Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: Wenyi Hu <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
@craig craig bot closed this as completed in 13c9b9d Mar 22, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-cdc Change Data Capture A-kv-rangefeed Rangefeed infrastructure, server+client branch-release-23.2 Used to mark GA and release blockers, technical advisories, and bugs for 23.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. O-23.2-scale-testing issues found during 23.2 scale testing O-testcluster Issues found or occurred on a test cluster, i.e. a long-running internal cluster T-cdc
Projects
None yet
5 participants