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

kvevent: implement chunked blocking buffer #86421

Merged
merged 3 commits into from
Aug 19, 2022

Conversation

jayshrivastava
Copy link
Contributor

@jayshrivastava jayshrivastava commented Aug 18, 2022

kvevent: refactor blocking buffer benchmark

This change updates the blocking buffer micro benchmark
in several ways:

  • it uses different types of events
  • it uses more producers than consumers to keep the
    buffer full
  • it makes b.N correspond to the total number of events,
    so the benchmark can analyze allocs per event

Release note: None

Release justification: This change updates a benchmark only.

kvevent: implement chunked buffer event queue

This change implements a simple chunked event queue.
The purpose of this queue is to be used by
kvevent.blockingBuffer in subsequent commits.

Release note: None

Release justification: This change does not affect
any production code. It adds files which are not
called by any packages.

kvevent: refactor memory buffer to chunked linked list

This change refactors kvevent/blocking_buffer.go to use
a chunked linked list instead of a regular linked list to
reduce pointer usage. Note that the underlying sync.Pool,
which is also a linked list, will use less pointers due to
us pooling chunks instead of events.

Release note: None

Release justification: This change significantly
improves performance by significantly reducing
pressure on GC. Consequently, this significantly
improves foreground SQL p99 latency. GC has
been causing severe issues in production changefeeds.
Merging this change in this release is worth it
for its potential to reduce incidents.

Results (micro)

These are the results of running the microbenchmark.
./dev bench pkg/ccl/changefeedccl/kvevent --filter=BenchmarkMemBuffer --count=10 --bench-mem --stream-output --test-args="--test.benchtime=45s" -- --nocache_test_results --test_verbose_timeout_warnings |& tee bench.txt

name          old time/op    new time/op    delta
MemBuffer-10    1.22µs ± 2%    0.85µs ± 3%  -30.04%  (p=0.000 n=8+10)

name          old alloc/op   new alloc/op   delta
MemBuffer-10     0.00B          0.00B          ~     (all equal)

name          old allocs/op  new allocs/op  delta
MemBuffer-10      0.00           0.00          ~     (all equal)
  • Memory usage is 0 due to pooling in both implementations.
  • We can achieve a higher throughput with the chunked implementation - about 50-60M events in 45 seconds as opposed to ~40M with the old implementation.

Results (Macro)

Full results are published here. In summary:

I analyzed performing by running TPC-C for 30 mins on a 15 node cluster with 10k warehouses. Before starting the workload, I started a changefeed on the order_line table (~200GB). I also set the following cluster settings to stress the buffer and pressure GC:
changefeed.backfill.concurrent_scan_requests = 100;
changefeed.memory.per_changefeed_limit = '1073741824'; (~1GB)

Then, I analyzed SQL latency from admin UI and GC performance using the output of GODEBUG=gctrace=1. These are the outcomes:

  • The p99 SQL latency during the workload was reduced from approx. 1.75s -> 0.150s (91%)
  • CPU time spent doing GC was reduced from 37.86 mins -> 20.75 mins (45%)
  • The p99 spike at the beginning of the workload was reduced from approx. 15s -> 12s (20%)

Relevant Issues

Addresses: #84582
(for now...)

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@jayshrivastava jayshrivastava force-pushed the chunked-ll-blocking-buf branch from 9a6052f to ced8985 Compare August 18, 2022 21:22
Yevgeniy Miretskiy and others added 2 commits August 18, 2022 17:24
This change updates the blocking buffer micro benchmark
in several ways:
- it uses different types of events
- it uses more producers than consumers to keep the
  buffer full
- it makes b.N correspond to the total number of events,
  so the benchmark can analyze allocs per event

Release note: None

Release justification: This change updates a benchmark only.
This change implements a simple chunked event queue.
The purpose of this queue is to be used by
kvevent.blockingBuffer in subsequent commits.

Release note: None

Release justification: This change does not affect
any production code. It adds files which are not
called by any packages.
@jayshrivastava jayshrivastava force-pushed the chunked-ll-blocking-buf branch 3 times, most recently from bc95e15 to 5b4b969 Compare August 19, 2022 13:46
@jayshrivastava jayshrivastava marked this pull request as ready for review August 19, 2022 13:46
@jayshrivastava jayshrivastava requested a review from a team as a code owner August 19, 2022 13:46
@jayshrivastava jayshrivastava requested review from gh-casper and miretskiy and removed request for a team and gh-casper August 19, 2022 13:46
Copy link
Contributor

@miretskiy miretskiy left a comment

Choose a reason for hiding this comment

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

Mostly nits; giving LGTM, but please remove unneeded interface, and revert unnecessary refactors from this PR.

pkg/ccl/changefeedccl/kvevent/blocking_buffer.go Outdated Show resolved Hide resolved
pkg/ccl/changefeedccl/kvevent/alloc.go Outdated Show resolved Hide resolved
pkg/ccl/changefeedccl/kvevent/blocking_buffer.go Outdated Show resolved Hide resolved
pkg/ccl/changefeedccl/kvevent/event.go Outdated Show resolved Hide resolved
@miretskiy
Copy link
Contributor

@nvanbenschoten : FYI: thanks to your investigations, @jayshrivastava made these changes that show significant improvements.

This change refactors kvevent/blocking_buffer.go to use
a chunked linked list instead of a regular linked list to
reduce pointer usage. Note that the underlying sync.Pool,
which is also a linked list, will use less pointers due to
us pooling chunks instead of events.

Release note: None

Release justification: This change significantly
improves performance by significantly reducing
pressure on GC. Consequently, this significantly
improves foreground SQL p99 latency. GC has
been causing severe issues in production changefeeds.
Merging this change in this release is worth it
for its potential to reduce incedents.
@jayshrivastava jayshrivastava force-pushed the chunked-ll-blocking-buf branch from 5b4b969 to 5734c3d Compare August 19, 2022 14:40
@jayshrivastava
Copy link
Contributor Author

bors r+

@craig
Copy link
Contributor

craig bot commented Aug 19, 2022

Build failed (retrying...):

@nvanbenschoten
Copy link
Member

This is great! Nice experimentation @jayshrivastava.

Fixes: #84709

I don't think I see where we're addressing #84709 in this PR. Doesn't the Event struct still contain 10 pointers? So doesn't a bufferEventChunk contain 128x10+1=1281 pointers?

@craig
Copy link
Contributor

craig bot commented Aug 19, 2022

Build failed (retrying...):

@shermanCRL shermanCRL changed the title kvevent: implement chunked blocking buff kvevent: implement chunked blocking buffer Aug 19, 2022
@miretskiy
Copy link
Contributor

I don't think I see where we're addressing #84709 in this PR. Doesn't the Event struct still contain 10 pointers? So doesn't a bufferEventChunk contain 128x10+1=1281 pointers?

I think previous changes (e8e664c#diff-a2e21a39cea12e1823c4c6f7ce7e1513214a575ff049f204cf5633284cf8c6c9) replaced resolved events which were allocating a pointer.
Event does contains various slices (key/value/resolved span) but those should be coming from rangefeed; so I don't think we are allocating them.

@nvanbenschoten
Copy link
Member

I think there might be a bit of confusion here. The concern isn't reducing heap allocations, it's about reducing the cost of GC by eliminating pointers that need to be traversed during the GC mark-sweep phase. So e8e664c#diff-a2e21a39cea12e1823c4c6f7ce7e1513214a575ff049f204cf5633284cf8c6c9 might have actually hurt, as the inlining replaced 1 pointer (*ResolvedSpan) for 2 (ResolvedSpan.Span.Key and ResolvedSpan.Span.EndKey).

@craig
Copy link
Contributor

craig bot commented Aug 19, 2022

Build succeeded:

@craig craig bot merged commit 6a51183 into cockroachdb:master Aug 19, 2022
@blathers-crl
Copy link

blathers-crl bot commented Aug 19, 2022

Encountered an error creating backports. Some common things that can go wrong:

  1. The backport branch might have already existed.
  2. There was a merge conflict.
  3. The backport branch contained merge commits.

You might need to create your backport manually using the backport tool.


error creating merge commit from 78b995a to blathers/backport-release-21.2-86421: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict []

you may need to manually resolve merge conflicts with the backport tool.

Backport to branch 21.2.x failed. See errors above.


error creating merge commit from 78b995a to blathers/backport-release-22.1-86421: POST https://api.github.com/repos/cockroachdb/cockroach/merges: 409 Merge conflict []

you may need to manually resolve merge conflicts with the backport tool.

Backport to branch 22.1.x failed. See errors above.


🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan.

@miretskiy
Copy link
Contributor

replaced 1 pointer (*ResolvedSpan) for 2 (ResolvedSpan.Span.Key and ResolvedSpan.Span.EndKey).

This is what's confusing to me too: because those 2 pointers needed to be deleted anyway because those were allocated
by rangefeed RPC. So.. I don't know if we're making situation worse. I think this is an improvement because instead of 3 pointers that need to be traversed, it's 2 (which were allocated anyway).

@nvanbenschoten
Copy link
Member

Does every Event have a non-empty resolved span? My claim that it might make things worse is based on the assumption that the vast majority of Events aren't carrying a resolved span, but if that's not true then I agree that e8e664c#diff-a2e21a39cea12e1823c4c6f7ce7e1513214a575ff049f204cf5633284cf8c6c9 should help.

@miretskiy
Copy link
Contributor

miretskiy commented Aug 19, 2022

No, not every event has non-empty resolved span; Can't really make a claim that majority of events aren't carrying resolved
span (you could have pretty low traffic table where majority of events will be resolved span events).
I don't know if just having 16 bytes in the structure (to hold resolve spans) is necessarily bad -- those arrays are allocated, contiguously.

We could try to basically break up event further:
for "regular" event, we need 3 []byte slices (key, value, prev value). For Resolved, we only need 2.
So, have just 3 slices, plus type of the event (we already have "flush" boolean -- which can be replace
with some sort of uint8). I don't know if it's worth the complexity. But we should chat.

@shermanCRL
Copy link
Contributor

Looks like we need a manual backport? Let’s get that going if we haven’t already.

@jayshrivastava
Copy link
Contributor Author

@miretskiy and I are looking to test out the changes against TPC-E before backporting. It would be nice to see the impact of these changes to the same workload Nathan ran originally.

@jayshrivastava
Copy link
Contributor Author

@shermanCRL Just finished testing with TPC-E, with and without this change. Please see the appendix in this doc more info.

Summary (with both a massive TPC-E load and a massive changefeed backfill running):

  • SQL latency is reduced from 560ms -> 400ms (29%)
  • SQL statement throughput goes up from 5k -> 6k (20%)

@miretskiy and I have more ideas regarding #84709. I think it makes the most sense to carry them out and backport everything at the same time.

@shermanCRL
Copy link
Contributor

Nice! What’s the baseline SQL latency without changefeeds?

@jayshrivastava
Copy link
Contributor Author

40ms with only TPC-E running

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.

5 participants