-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
Comments
cc @cockroachdb/replication |
cc @cockroachdb/cdc |
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). cockroach/pkg/kv/kvserver/rangefeed/scheduled_processor.go Lines 772 to 773 in ab00ae9
cockroach/pkg/kv/kvserver/rangefeed/processor.go Lines 953 to 967 in 9640975
|
I don't know if this is a release blocker -- it's not a regression. @erikgrinaker ? |
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. |
cc @cockroachdb/cdc |
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
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
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
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
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
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
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
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
cockroach/pkg/kv/kvserver/rangefeed/budget.go Lines 263 to 264 in 3df5f2c
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
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.
Currently, we are picking solution 4 for its simplicity and will revisit this topic if anything comes up. More details for solution 4
We got these data ^ from test benchmarks documented here #120582. |
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
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
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]>
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
:inuse_objects
:Environment:
V23.2.0-ALPHA.6-DEV
Additional context
More context here (internal).
Debugging artifacts can be found here (internal).
Jira issue: CRDB-33387
The text was updated successfully, but these errors were encountered: