Skip to content

kvserver: make MVCC GC less disruptive to foreground traffic#83213

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
aayushshah15:latchlessMVCCGC
Aug 9, 2022
Merged

kvserver: make MVCC GC less disruptive to foreground traffic#83213
craig[bot] merged 2 commits intocockroachdb:masterfrom
aayushshah15:latchlessMVCCGC

Conversation

@aayushshah15
Copy link
Copy Markdown
Contributor

This commit changes GC requests to no longer declare exclusive latches
at their BatchRequest's timestamp. This was already incorrect as
explained in #55293.

The first use is broken because we acquire write latches at the batch
header's timestamp, which is set to time.Now(), so we're only
serializing with reads in the future and all other writes [1]. So we're
disruptive to everyone except who we want to serialize with – reads in
the past!

This commit makes GC requests only declare a non-mvcc exclusive latch
over the RangeGCThresholdKey. This is correct because:


// 1. We define "correctness" to be the property that a reader reading at /
// around the GC threshold will either see the correct results or receive an
// error.
// 2. Readers perform their command evaluation over a stable snapshot of the
// storage engine. This means that the reader will not see the effects of a
// subsequent GC run as long as it created a Pebble iterator before the GC
// request.
// 3. A reader checks the in-memory GC threshold of a Replica after it has
// created this snapshot (i.e. after a Pebble iterator has been created).
// 4. If the in-memory GC threshold is above the timestamp of the read, the
// reader receives an error. Otherwise, the reader is guaranteed to see a
// state of the storage engine that hasn't been affected by the GC request [5].
// 5. GC requests bump the in-memory GC threshold of a Replica as a pre-apply
// side effect. This means that if a reader checks the in-memory GC threshold
// after it has created a Pebble iterator, it is impossible for the iterator
// to point to a storage engine state that has been affected by the GC
// request.

As a result, GC requests should now be much less disruptive to
foreground traffic since they're no longer redundantly declaring
exclusive latches over global keys.

Resolves #55293

Release note(performance improvement): MVCC garbage collection should
now be much less disruptive to foreground traffic than before.

@aayushshah15 aayushshah15 requested a review from a team as a code owner June 22, 2022 18:45
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@aayushshah15
Copy link
Copy Markdown
Contributor Author

Next steps for this PR

  1. Benchmark YCSB-A with a low GC TTL with and without this patch
  2. Confirm that the stats updates for writes and GC requests commute, and write a test for it

@aayushshah15
Copy link
Copy Markdown
Contributor Author

Benchmark YCSB-A with a low GC TTL with and without this patch

I ran ycsb-a with a 1 second GC TTL for 25 iterations (1m ramp and 5m duration) and see the following.

> benchstat ../benchlogs/yscbA_before ../benchlogs/yscbA_after

name  old ops/sec  new ops/sec  delta
.      51.5k ± 6%   51.3k ± 6%   ~     (p=0.658 n=25+25)

name  old p50      new p50      delta
.       1.60 ± 0%    1.60 ± 0%   ~     (all equal)

name  old p95      new p95      delta
.       4.50 ± 0%    4.50 ± 0%   ~     (all equal)

name  old p99      new p99      delta
.       41.0 ±13%    41.7 ±10%   ~     (p=0.326 n=25+25)

On taking a closer look, I think the results stem from the fact that there just weren't a ton of GC requests being issued in this workload (something on the order of once every 10 seconds across the cluster, will take screenshots next time). Perhaps we need a workload that writes bigger datums?

@nvb
Copy link
Copy Markdown
Contributor

nvb commented Jul 11, 2022

Perhaps we need a workload that writes bigger datums?

This sounds like a good idea. It should be easy to make this change.

You could also drop down mvccGCQueueTimerDuration to 0s to run the queue more aggressively.

I also wonder if even with the 1-second GC TTL, the MVCC GC queue is still too unreactive for us to see the effects of this change. We never directly enqueue into it, so for a given range, it will only run at most once every scanner interval.

@aayushshah15 aayushshah15 force-pushed the latchlessMVCCGC branch 2 times, most recently from 5803cdc to bdc900b Compare July 12, 2022 00:58
@aayushshah15 aayushshah15 force-pushed the latchlessMVCCGC branch 3 times, most recently from 6d8e0d8 to 322ec77 Compare July 21, 2022 16:10
@aayushshah15 aayushshah15 force-pushed the latchlessMVCCGC branch 2 times, most recently from e4eafb8 to 28bef39 Compare July 21, 2022 17:00
@aayushshah15
Copy link
Copy Markdown
Contributor Author

@aliher1911 and @erikgrinaker: I'm trying to understand the implications of #84576 over this patch. I'm missing a ton of context here but I'm trying to understand this part from Oleg's issue:

we can optimize range key GC requests to only get read latches to preserve range boundaries for the 
same of MVCC stats calculations, but don't prevent reads.

Is there anything you could point me to that'll help me understand why we need this additional bit of serialization, after #78980 and #76312?

@aayushshah15
Copy link
Copy Markdown
Contributor Author

aayushshah15 commented Jul 21, 2022

@nvanbenschoten: I've added a few microbenchmarks based on our discussions (see the commit message on the second commit for before/after comparisons). Are you interested in any other scenario that you'd suggest we add in this patch?

Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

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

I don't believe there should be a need for point GC to serialize with newer MVCC range key writes, since the point key stats are only affected by the point/range key directly above them, and that point/range key must be below the GC threshold for the point key to be eligible for GC in the first place (assuming a non-tombstone point key).

This last part about tombstone point keys is interesting. If we have a tombstone and a non-tombstone below the GC threshold, we are able to GC both of them, leaving no remaining version on the key. Based on a reading of ComputeStatsForRangeWithVisitors and MVCCGarbageCollect, I think this is ok, as the presence of a range tombstone does not have an effect on point tombstones directly below it, only point non-tombstones.

We could either build one of these, or we could possibly hack around it by taking out a read/write latch at timestamp hlc.MaxTimestamp which would not serialize with any readers (since there cannot be any readers above that timestamp).

Taking out a read/write latch at timestamp hlc.MaxTimestamp seems like a reasonable solution to me.

Reviewed 5 of 6 files at r3, 1 of 1 files at r4, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @arulajmani)


pkg/kv/kvserver/replica_test.go line 8814 at r4 (raw file):

			// Since the GC request does not acquire latches on the keys being GC'ed,
			// they're not guaranteed to wait for these above Puts to get applied on
			// the leaseholder. See AckCommittedEntriesBeforeApplication() the comment

"and the"

aayushshah15 added a commit to aayushshah15/cockroach that referenced this pull request Aug 4, 2022
This commit adds a couple simple microbenchmarks that demonstrate the
improvement in the level of  foreground traffic disruption caused by MVCC GC
after cockroachdb#83213.

Results:
```
> benchdiff --old=beforeLatchlessGC --new=latchlessMVCCGC --post-checkout='make buildshort' ./pkg/kv/kvserver --run=MVCCGCWithForegroundTraffic --count=20;

  pkg=1/1 iter=10/10 cockroachdb/cockroach/pkg/kv/kvserver -

name                                                     old time/op    new time/op    delta
MVCCGCWithForegroundTraffic/gc_with_reads-24               1.67ms ±18%    0.02ms ± 6%  -98.74%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24    1.10ms ±19%    0.57ms ± 4%  -48.45%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24          12.3µs ± 2%     7.3µs ± 1%  -40.33%  (p=0.000 n=10+9)

name                                                     old alloc/op   new alloc/op   delta
MVCCGCWithForegroundTraffic/gc_with_reads-24               17.3kB ±11%     1.7kB ± 4%  -90.44%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24    37.5kB ± 5%    24.4kB ± 3%  -34.92%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24          1.26kB ± 0%    1.25kB ± 1%   -0.43%  (p=0.001 n=10+10)

name                                                     old allocs/op  new allocs/op  delta
MVCCGCWithForegroundTraffic/gc_with_reads-24                  111 ± 1%        32 ± 0%  -71.08%  (p=0.000 n=9+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24       190 ± 7%       128 ± 2%  -32.37%  (p=0.000 n=10+9)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24            15.0 ± 0%      15.0 ± 0%     ~     (all equal)
```

Release note: None
Copy link
Copy Markdown
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

This all seems fine to me, thanks for improving it!

@aliher1911 This mostly addresses #84576. I think we should extend some of these benchmarks to include range tombstone GC, and see if we need to either break up or otherwise limit the work we do per request to make sure we're not holding onto the latches for too long. We also need to look at TBI performance below the GC threshold -- at that point, most data is likely compacted down to the lower LSM levels, and so we'll have to rely on skipping blocks rather than entire SSTs, and I don't know if we get as much of a boost from that.

I think this is ok, as the presence of a range tombstone does not have an effect on point tombstones directly below it, only point non-tombstones.

Yeah, that sounds right.

Taking out a read/write latch at timestamp hlc.MaxTimestamp seems like a reasonable solution to me.

Thanks for confirming -- I think that's the only viable solution with only a week left before stability. Not too thrilled about the wide write latches though, we'll just have to make sure they're short-lived and maybe break them up. Are they essentially going to form a barrier in the concurrency manager queue, or is there some way we can avoid head-of-line blockage? This reminds me about the problems with split latches (#81152).

Reviewed 4 of 6 files at r3, 2 of 2 files at r5, 1 of 1 files at r6, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @arulajmani)


-- commits line 59 at r6:
This is lovely. 🚀


pkg/kv/kvserver/batcheval/cmd_add_sstable.go line 63 at r5 (raw file):

		latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp)
	}
	latchSpans.DisableUndeclaredAccessAssertions()

Any particular reason for this? I'd rather disable the assertions for the one specific access that violates the latches, than the entire request.


pkg/kv/kvserver/batcheval/cmd_recompute_stats.go line 57 at r5 (raw file):

	latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(rdKey, uuid.Nil)})
	// Disable the assertions which check that all reads were previously declared.
	latchSpans.DisableUndeclaredAccessAssertions()

As an apropos, I have been a bit worried about the non-commutativity of MVCC range tombstones in the context of consistency check stats calculations. As far as I can tell, this should still be safe, because the stats computation is only reading (from a stable Pebble snapshot) and the stats delta of the write will be correct regardless of the base stats. The non-commutativity is only a problem with concurrent writers. Anything I'm missing?

@aayushshah15
Copy link
Copy Markdown
Contributor Author

pkg/kv/kvserver/batcheval/cmd_add_sstable.go line 63 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Any particular reason for this? I'd rather disable the assertions for the one specific access that violates the latches, than the entire request.

Are you suggesting that we keep spanset.DisableReaderAssertions around, or something else?

Copy link
Copy Markdown
Contributor

@erikgrinaker erikgrinaker left a comment

Choose a reason for hiding this comment

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

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @arulajmani)


pkg/kv/kvserver/batcheval/cmd_add_sstable.go line 63 at r5 (raw file):

Previously, aayushshah15 (Aayush Shah) wrote…

Are you suggesting that we keep spanset.DisableReaderAssertions around, or something else?

Yeah, for the AddSSTable case there was only a single read access that needed to circumvent the spanset assertion (and it was safe to do so). For the rest of the accesses, I think we'd rather still keep the assertions enabled, if possible.

Copy link
Copy Markdown
Contributor Author

@aayushshah15 aayushshah15 left a comment

Choose a reason for hiding this comment

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

Thanks for the reviews!

bors r+

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, @arulajmani, @erikgrinaker, and @nvanbenschoten)


pkg/kv/kvserver/replica_test.go line 8814 at r4 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

"and the"

Done.


pkg/kv/kvserver/batcheval/cmd_add_sstable.go line 63 at r5 (raw file):

Previously, erikgrinaker (Erik Grinaker) wrote…

Yeah, for the AddSSTable case there was only a single read access that needed to circumvent the spanset assertion (and it was safe to do so). For the rest of the accesses, I think we'd rather still keep the assertions enabled, if possible.

Done.

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 8, 2022

🕐 Waiting for PR status (Github check) to be set, probably by CI. Bors will automatically try to run when all required PR statuses are set.

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 8, 2022

GitHub status checks took too long to complete, so bors is giving up. You can adjust bors configuration to have it wait longer if you like.

This commit changes GC requests to no longer declare exclusive latches
at their BatchRequest's timestamp. This was already incorrect as
explained in cockroachdb#55293.

>The first use is broken because we acquire write latches at the batch
header's timestamp, which is set to time.Now(), so we're only
serializing with reads in the future and all other writes [1]. So we're
disruptive to everyone except who we want to serialize with – reads in
the past!

This commit makes GC requests only declare a non-mvcc exclusive latch
over the `RangeGCThresholdKey`. This is correct because:
```

// 1. We define "correctness" to be the property that a reader reading at /
// around the GC threshold will either see the correct results or receive an
// error.
// 2. Readers perform their command evaluation over a stable snapshot of the
// storage engine. This means that the reader will not see the effects of a
// subsequent GC run as long as it created a Pebble iterator before the GC
// request.
// 3. A reader checks the in-memory GC threshold of a Replica after it has
// created this snapshot (i.e. after a Pebble iterator has been created).
// 4. If the in-memory GC threshold is above the timestamp of the read, the
// reader receives an error. Otherwise, the reader is guaranteed to see a
// state of the storage engine that hasn't been affected by the GC request [5].
// 5. GC requests bump the in-memory GC threshold of a Replica as a pre-apply
// side effect. This means that if a reader checks the in-memory GC threshold
// after it has created a Pebble iterator, it is impossible for the iterator
// to point to a storage engine state that has been affected by the GC
// request.

```

As a result, GC requests should now be much less disruptive to
foreground traffic since they're no longer redundantly declaring
exclusive latches over global keys.

Release note (performance improvement): MVCC garbage collection should
now be much less disruptive to foreground traffic than before.
This commit adds a couple simple microbenchmarks that demonstrate the
improvement in the level of  foreground traffic disruption caused by MVCC GC
after cockroachdb#83213.

Results:
```
> benchdiff --old=beforeLatchlessGC --new=latchlessMVCCGC --post-checkout='make buildshort' ./pkg/kv/kvserver --run=MVCCGCWithForegroundTraffic --count=20;

  pkg=1/1 iter=10/10 cockroachdb/cockroach/pkg/kv/kvserver -

name                                                     old time/op    new time/op    delta
MVCCGCWithForegroundTraffic/gc_with_reads-24               1.67ms ±18%    0.02ms ± 6%  -98.74%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24    1.10ms ±19%    0.57ms ± 4%  -48.45%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24          12.3µs ± 2%     7.3µs ± 1%  -40.33%  (p=0.000 n=10+9)

name                                                     old alloc/op   new alloc/op   delta
MVCCGCWithForegroundTraffic/gc_with_reads-24               17.3kB ±11%     1.7kB ± 4%  -90.44%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24    37.5kB ± 5%    24.4kB ± 3%  -34.92%  (p=0.000 n=10+10)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24          1.26kB ± 0%    1.25kB ± 1%   -0.43%  (p=0.001 n=10+10)

name                                                     old allocs/op  new allocs/op  delta
MVCCGCWithForegroundTraffic/gc_with_reads-24                  111 ± 1%        32 ± 0%  -71.08%  (p=0.000 n=9+10)
MVCCGCWithForegroundTraffic/gc_with_reads_and_writes-24       190 ± 7%       128 ± 2%  -32.37%  (p=0.000 n=10+9)
MVCCGCWithForegroundTraffic/noop_gc_with_reads-24            15.0 ± 0%      15.0 ± 0%     ~     (all equal)
```

Release note: None
@aayushshah15
Copy link
Copy Markdown
Contributor Author

bors r+

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 9, 2022

Build failed (retrying...):

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Aug 9, 2022

Build succeeded:

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.

kv: make MVCC garbage collection latch-less

4 participants