storage: improve batching algorithm when sending keys for GC#20373
storage: improve batching algorithm when sending keys for GC#20373spencerkimball merged 1 commit intocockroachdb:masterfrom
Conversation
5cadfc8 to
022f85a
Compare
|
Good point about the alternative suffering from stats complexities. Still possible, but certainly less appealing. Reviewed 4 of 4 files at r1. pkg/storage/gc_queue.go, line 78 at r1 (raw file):
Clarify that this counts versions (I'd even rename the const to pkg/storage/gc_queue.go, line 798 at r1 (raw file):
Only increment pkg/storage/gc_queue.go, line 806 at r1 (raw file):
Not really a request to change anything, but just wanted to point out that when there are many versions, we could've considered using an iterator here and jumping to the next key once we know the cutoff. This isn't possible any more after this change (since we need pkg/storage/gc_queue.go, line 808 at r1 (raw file):
Stale comment? We won't delete straight from the engine. pkg/storage/gc_queue.go, line 812 at r1 (raw file):
Why the pkg/storage/gc_queue.go, line 814 at r1 (raw file):
This code looks like it might benefit from use of a pkg/storage/gc_queue_test.go, line 963 at r1 (raw file):
Comments from Reviewable |
|
Reminder: add a release note |
022f85a to
176748e
Compare
|
I've switched to use byte count and added a release note. Review status: all files reviewed at latest revision, 7 unresolved discussions, all commit checks successful. pkg/storage/gc_queue.go, line 78 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
I've switched to using byte counts instead of number of keys. pkg/storage/gc_queue.go, line 798 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 806 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
I thought about that a lot actually. The idea originally was to allow an open ended policy for the pkg/storage/gc_queue.go, line 808 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Yeah, that was left over from original direction. Removed. pkg/storage/gc_queue.go, line 812 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
In case the total count (or now size) of all of the versions on the very first key exceeds the threshold. In that case, we'd append an empty gcKeys slice. pkg/storage/gc_queue.go, line 814 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Not sure what that's for...you mean just to avoid reallocating large slices? pkg/storage/gc_queue_test.go, line 963 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Good add. Comments from Reviewable |
5ce9fa9 to
2e89d6b
Compare
|
Update the commit message (byte count). Also, the release note should have a category. I think in this case
is appropriate. This change is starting to look really good! I had one comment which I think substantially improves on this, namely being able to chunk single keys with lots of versions by creating a sequence of GCRequests for the key with ascending target timestamp. Looks like a big win (for example, would occur as folks tighten the GC TTL on overwrite-heavy workloads) and the code is basically there already in your current work. Reviewed 2 of 2 files at r2. pkg/storage/gc_queue.go, line 78 at r1 (raw file): Previously, spencerkimball (Spencer Kimball) wrote…
Rename to pkg/storage/gc_queue.go, line 814 at r1 (raw file): Previously, spencerkimball (Spencer Kimball) wrote…
Yeah, it could reduce the allocations for getting all these key values. But not for now. pkg/storage/gc_queue.go, line 578 at r2 (raw file): While you're here, please add that as a comment. For example,
pkg/storage/gc_queue.go, line 800 at r2 (raw file):
Move this into the pkg/storage/gc_queue.go, line 816 at r2 (raw file):
pkg/storage/gc_queue.go, line 821 at r2 (raw file):
So if you had alternatingly 1kb and 1mb keys to GC (not unlikely with column families), it would be pretty inefficient because you'd be sending But I'm still worried about single keys with a huge history, which you still won't chunk down here. But, what you could do, is artificially lower the GC timestamp so that it touches less version, and chunk on that. That is, you have a key with versions at 1000, 999, ..., 1 and each version is 8kb (and say your target size is 80kb) you'd emit GC requests for Your current code has all the ingredients. You just need to change the loop over pkg/storage/gc_queue.go, line 822 at r2 (raw file):
You can move this line up above the pkg/storage/gc_queue_test.go, line 917 at r2 (raw file):
nit: pkg/storage/gc_queue_test.go, line 919 at r2 (raw file):
`t.Fatalf("expected GC chunk size to be a multiple of %s", keyCount) pkg/storage/gc_queue_test.go, line 921 at r2 (raw file):
nit: can be a pkg/storage/gc_queue_test.go, line 923 at r2 (raw file):
Comments from Reviewable |
2e89d6b to
af0ca7d
Compare
|
OK, I've incorporated that change (and further expanded the unittest). Review status: 2 of 4 files reviewed at latest revision, 11 unresolved discussions. pkg/storage/gc_queue.go, line 78 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 814 at r1 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Ack pkg/storage/gc_queue.go, line 578 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 800 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 816 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 821 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Swell idea! Implemented. pkg/storage/gc_queue.go, line 822 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Then I'd have to reset pkg/storage/gc_queue_test.go, line 917 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue_test.go, line 919 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue_test.go, line 921 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue_test.go, line 923 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. Comments from Reviewable |
|
Reviewed 2 of 2 files at r3. pkg/storage/gc_queue.go, line 800 at r2 (raw file): Previously, spencerkimball (Spencer Kimball) wrote…
and the pkg/storage/gc_queue.go, line 79 at r3 (raw file):
1mb could still show up significantly in latencies. Might be worth starting smaller here. How about 256 KiB (i.e. pkg/storage/gc_queue.go, line 570 at r3 (raw file):
nit: pkg/storage/gc_queue.go, line 818 at r3 (raw file):
(if my reading of this code is correct, this is how it works now, where before you'd save the current key for the next batch). Comments from Reviewable |
af0ca7d to
870ae14
Compare
|
Review status: 3 of 4 files reviewed at latest revision, 4 unresolved discussions. pkg/storage/gc_queue.go, line 800 at r2 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 79 at r3 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 570 at r3 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. pkg/storage/gc_queue.go, line 818 at r3 (raw file): Previously, tschottdorf (Tobias Schottdorf) wrote…
Done. Comments from Reviewable |
|
Reviewed 1 of 1 files at r4. Comments from Reviewable |
870ae14 to
ffae1fa
Compare
Previously the batching algorithm only considered the size of top-level keys. This used to work well when we were doing replica-evaluated Raft. With leader-evaluated, keys with many versions can blow up into massive write batch values which can end up clogging up Raft, causing performance degradation. This PR augments the `GC.Filter` method to return the index of the first version of a key to be GC'ed, allowing us to keep track of the total number of keys to be GC'd. The GC queue is now able to send a more appropriate number of keys per batch, which results in significant improvements in 99th percentile latencies. This was tested with a modified version of the kv load generator, set up to write large keys (1024 bytes) with a short cycle length, creating lots of key versions for GC. A three node local cluster with the TTL set to 120 seconds could be made to experience up to 6 second 99th %ile latencies after about 10 minutes of running. With this change, p99 latencies appear stable. N.B. an alternative to this change was to let each replica handle GC locally, and change the consistency checker to avoid considering versions earlier than the GC threshold. However, this ran into fairly tricky problems around keeping the MVCC stats values correct. See cockroachdb#20052 Release note (performance improvement): improved p99 latencies for garbage collection of previous versions of a key, when there are **many** versions.
ffae1fa to
a137b05
Compare
Found via `TestMVCCStatsRandomized`. We weren't correctly updating the stats in the case in which the cluster setting limited the number of keys to delete, but the GC Timestamp covered the first entry. In that case, the stats update would assume implicitly that all values were going to be deleted. Since cockroachdb#20373 and others around the same time have made the cluster setting obsolete, the course of action taken here was simply the removal of this limiting functionality. This is congruent with the fact that add the time at which it was added, `kv.gc.batch_size` was a stopgap fix. Release note (general change): removed the obsolete `kv.gc.batch_size` cluster setting.
Found via `TestMVCCStatsRandomized`. We weren't correctly updating the stats in the case in which the cluster setting limited the number of keys to delete, but the GC Timestamp covered the first entry. In that case, the stats update would assume implicitly that all values were going to be deleted. Since cockroachdb#20373 and others around the same time have made the cluster setting obsolete, the course of action taken here was simply the removal of this limiting functionality. This is congruent with the fact that add the time at which it was added, `kv.gc.batch_size` was a stopgap fix. Release note (general change): removed the obsolete `kv.gc.batch_size` cluster setting.
Found via `TestMVCCStatsRandomized`. We weren't correctly updating the stats in the case in which the cluster setting limited the number of keys to delete, but the GC Timestamp covered the first entry. In that case, the stats update would assume implicitly that all values were going to be deleted. Since cockroachdb#20373 and others around the same time have made the cluster setting obsolete, the course of action taken here was simply the removal of this limiting functionality. This is congruent with the fact that at the time at which it was added, `kv.gc.batch_size` was a stopgap fix. Release note (backwards-incompatible change): removed the obsolete `kv.gc.batch_size` cluster setting.
Found via `TestMVCCStatsRandomized`. We weren't correctly updating the stats in the case in which the cluster setting limited the number of keys to delete, but the GC Timestamp covered the first entry. In that case, the stats update would assume implicitly that all values were going to be deleted. Since cockroachdb#20373 and others around the same time have made the cluster setting obsolete, the course of action taken here was simply the removal of this limiting functionality. This is congruent with the fact that at the time at which it was added, `kv.gc.batch_size` was a stopgap fix. Release note (backwards-incompatible change): removed the obsolete `kv.gc.batch_size` cluster setting.
Previously the batching algorithm only considered the size of
top-level keys. This used to work well when we were doing
replica-evaluated Raft. With leader-evaluated, keys with many
versions can blow up into massive write batch values which can
end up clogging up Raft, causing performance degradation.
This PR augments the
GC.Filtermethod to return the index ofthe first version of a key to be GC'ed, allowing us to keep track
of the total number of keys to be GC'd. The GC queue is now able
to send a more appropriate number of keys per batch, which results
in significant improvements in 99th percentile latencies.
This was tested with a modified version of the kv load generator,
set up to write large keys (1024 bytes) with a short cycle length,
creating lots of key versions for GC. A three node local cluster
with the TTL set to 120 seconds could be made to experience up to
6 second 99th %ile latencies after about 10 minutes of running.
With this change, p99 latencies appear stable.
N.B. an alternative to this change was to let each replica handle
GC locally, and change the consistency checker to avoid considering
versions earlier than the GC threshold. However, this ran into
fairly tricky problems around keeping the MVCC stats values correct.
See #20052
Release note: improved p99 latencies for garbage collection of
previous versions of a key, when there are many versions.