Skip to content

storage: improve batching algorithm when sending keys for GC#20373

Merged
spencerkimball merged 1 commit intocockroachdb:masterfrom
spencerkimball:local-gc
Dec 4, 2017
Merged

storage: improve batching algorithm when sending keys for GC#20373
spencerkimball merged 1 commit intocockroachdb:masterfrom
spencerkimball:local-gc

Conversation

@spencerkimball
Copy link
Copy Markdown
Member

@spencerkimball spencerkimball commented Dec 1, 2017

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 #20052

Release note: improved p99 latencies for garbage collection of
previous versions of a key, when there are many versions.

@spencerkimball spencerkimball requested review from a team and tbg December 1, 2017 05:35
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 1, 2017

Good point about the alternative suffering from stats complexities. Still possible, but certainly less appealing.
I think you could get the high latencies back if you used long keys, though, since you chunk on the count and not on the byte size. I think it's acceptable to hold off on that though.

:lgtm:


Reviewed 4 of 4 files at r1.
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):

	// gcKeyVersionChunkSize is the default size for GCRequest's batch key size.
	gcKeyVersionChunkSize = 1000

Clarify that this counts versions (I'd even rename the const to gcKeyVersionChunkCount to make it obvious). I think the right thing to chunk on is key size though, as the old variable tried to do. For example, if you have 10mb keys (god forbid) you'd pile up 1000*10mb and would really want to send each key alone. OTOH, if you have 10k keys and they're all 1 byte each, no problem. (I think it's acceptable to hold off on that now, but it should be mentioned, say in a TODO).


pkg/storage/gc_queue.go, line 798 at r1 (raw file):

						}
						txnMap[txnID] = txn
						infoMu.IntentTxns++

Only increment IntentTxns when you are first creating the entry in txnMap. The current code is overcounting.


pkg/storage/gc_queue.go, line 806 at r1 (raw file):

				}
				// See if any values may be GC'd.
				if idx, gcTS := gc.Filter(keys[startIdx:], vals[startIdx:]); gcTS != (hlc.Timestamp{}) {

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 len(keys[:startIdx]) to be known). I think that's ok.


pkg/storage/gc_queue.go, line 808 at r1 (raw file):

				if idx, gcTS := gc.Filter(keys[startIdx:], vals[startIdx:]); gcTS != (hlc.Timestamp{}) {
					// Add key to slice of local keys to be deleted directly
					// from engine _after_ the GC threshold timestamp has been

Stale comment? We won't delete straight from the engine.


pkg/storage/gc_queue.go, line 812 at r1 (raw file):

					// GC requests through Raft.
					numVersions := int64(len(keys) - (startIdx + idx))
					if len(localGCKeys) > 0 && (localGCKeysVersionCount+numVersions) > gcKeyVersionChunkSize {

Why the len(.) check?


pkg/storage/gc_queue.go, line 814 at r1 (raw file):

					if len(localGCKeys) > 0 && (localGCKeysVersionCount+numVersions) > gcKeyVersionChunkSize {
						gcKeys = append(gcKeys, localGCKeys)
						localGCKeys = []roachpb.GCRequest_GCKey{}
localGCKeys = nil

This code looks like it might benefit from use of a bufalloc.ByteAllocator, but not without measuring first (so not anytime soon).


pkg/storage/gc_queue_test.go, line 963 at r1 (raw file):

// We wrote two batches worth of keys spread out, and two keys that each have enough old versions to fill a whole batch each, adding up to four batches in total.


Comments from Reviewable

@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 1, 2017

Reminder: add a release note

@spencerkimball
Copy link
Copy Markdown
Member Author

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…

Clarify that this counts versions (I'd even rename the const to gcKeyVersionChunkCount to make it obvious). I think the right thing to chunk on is key size though, as the old variable tried to do. For example, if you have 10mb keys (god forbid) you'd pile up 1000*10mb and would really want to send each key alone. OTOH, if you have 10k keys and they're all 1 byte each, no problem. (I think it's acceptable to hold off on that now, but it should be mentioned, say in a TODO).

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…

Only increment IntentTxns when you are first creating the entry in txnMap. The current code is overcounting.

Done.


pkg/storage/gc_queue.go, line 806 at r1 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

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 len(keys[:startIdx]) to be known). I think that's ok.

I thought about that a lot actually. The idea originally was to allow an open ended policy for the GC.Filter function which could consider all values together. So far we haven't used it for anything, so might be better to punt. Worst case we could pass the iterator to the filter function instead so it could do what it needed to do. Things seem reasonably performant in any case. I'm going to leave as is.


pkg/storage/gc_queue.go, line 808 at r1 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

Stale comment? We won't delete straight from the engine.

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…

Why the len(.) check?

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…
localGCKeys = nil

This code looks like it might benefit from use of a bufalloc.ByteAllocator, but not without measuring first (so not anytime soon).

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…

// We wrote two batches worth of keys spread out, and two keys that each have enough old versions to fill a whole batch each, adding up to four batches in total.

Good add.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the local-gc branch 4 times, most recently from 5ce9fa9 to 2e89d6b Compare December 2, 2017 20:52
@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 3, 2017

Update the commit message (byte count). Also, the release note should have a category. I think in this case

Release note (performance improvement): ...

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.
Review status: all files reviewed at latest revision, 11 unresolved discussions, all commit checks successful.


pkg/storage/gc_queue.go, line 78 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I've switched to using byte counts instead of number of keys.

Rename to gcKeyVersionChunkBytes


pkg/storage/gc_queue.go, line 814 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Not sure what that's for...you mean just to avoid reallocating large slices?

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):
We don't actually want to put any data into the first request. The first one is special in that it sets the GC thresholds, and as such, it can't intermingle with any on-going reads (that is, it's pretty heavy on the command queue). Consequently, we want it as small as possible so that it slips through Raft quickly. This may be less of an issue now that we're properly chunking, but at the very least a table with ~1mb keys would have that problem, so it's worth the one-off.

While you're here, please add that as a comment. For example,

The first request is intentionally kept very small since it updates the Range-wide GCThresholds, and thus must block all reads and writes while it is being applied.


pkg/storage/gc_queue.go, line 800 at r2 (raw file):

							infoMu.IntentTxns++
						}
						txnMap[txnID] = txn

Move this into the !ok along with txn := ... above.


pkg/storage/gc_queue.go, line 816 at r2 (raw file):

// If the current chunk is non-empty and the current key would bring it over the target size, flush and start a new chunk.


pkg/storage/gc_queue.go, line 821 at r2 (raw file):

						batchGCKeysByteCount = 0
					}
					batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: gcTS})

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 1kb and 1mb chunks. The straightforward alternative would be adding to a chunk and resetting after it goes over target size, but then we might go over by quite a bit. I think what you're doing here is probably better. There's of course also the third option, which is keeping the chunks in a min-heap (by size) and filling them up optimally, but pretty sure that's a tangent we don't have to go on right now.

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 ts <= 10, ts <= 20, ... ts <= 1000 because evaluated in that order, each of them will generate ~80kb of deletions.

Your current code has all the ingredients. You just need to change the loop over keys to go in reverse direction (old versions to new) and move the chunk-emitting logic into it. I think that's worth doing and also means that chunks will be more balanced in general, except when folks throw very large keys into the mix (at which point, not much we can do except damage control).


pkg/storage/gc_queue.go, line 822 at r2 (raw file):

					}
					batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{Key: expBaseKey, Timestamp: gcTS})
					batchGCKeysByteCount += byteCount

You can move this line up above the if len(.) and then just use batchGCKeysByteCount in the if.


pkg/storage/gc_queue_test.go, line 917 at r2 (raw file):

	tc.StartWithStoreConfig(t, stopper, tsc)

	keyCount := 1000

nit: const keyCount = 1000


pkg/storage/gc_queue_test.go, line 919 at r2 (raw file):

	keyCount := 1000
	if gcKeyVersionChunkSize%keyCount != 0 {
		panic("expected gcKeyVersionChunkSize to be a multiple of 1000")

`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):

		panic("expected gcKeyVersionChunkSize to be a multiple of 1000")
	}
	keySize := gcKeyVersionChunkSize / keyCount

nit: can be a const too, I think! Always nice to see at first glance the things that can't change.


pkg/storage/gc_queue_test.go, line 923 at r2 (raw file):

// length keySize.


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

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…

Rename to gcKeyVersionChunkBytes

Done.


pkg/storage/gc_queue.go, line 814 at r1 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

Yeah, it could reduce the allocations for getting all these key values. But not for now.

Ack


pkg/storage/gc_queue.go, line 578 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

We don't actually want to put any data into the first request. The first one is special in that it sets the GC thresholds, and as such, it can't intermingle with any on-going reads (that is, it's pretty heavy on the command queue). Consequently, we want it as small as possible so that it slips through Raft quickly. This may be less of an issue now that we're properly chunking, but at the very least a table with ~1mb keys would have that problem, so it's worth the one-off.

While you're here, please add that as a comment. For example,

The first request is intentionally kept very small since it updates the Range-wide GCThresholds, and thus must block all reads and writes while it is being applied.

Done.


pkg/storage/gc_queue.go, line 800 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

Move this into the !ok along with txn := ... above.

Done.


pkg/storage/gc_queue.go, line 816 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

// If the current chunk is non-empty and the current key would bring it over the target size, flush and start a new chunk.

Done.


pkg/storage/gc_queue.go, line 821 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

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 1kb and 1mb chunks. The straightforward alternative would be adding to a chunk and resetting after it goes over target size, but then we might go over by quite a bit. I think what you're doing here is probably better. There's of course also the third option, which is keeping the chunks in a min-heap (by size) and filling them up optimally, but pretty sure that's a tangent we don't have to go on right now.

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 ts <= 10, ts <= 20, ... ts <= 1000 because evaluated in that order, each of them will generate ~80kb of deletions.

Your current code has all the ingredients. You just need to change the loop over keys to go in reverse direction (old versions to new) and move the chunk-emitting logic into it. I think that's worth doing and also means that chunks will be more balanced in general, except when folks throw very large keys into the mix (at which point, not much we can do except damage control).

Swell idea! Implemented.


pkg/storage/gc_queue.go, line 822 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

You can move this line up above the if len(.) and then just use batchGCKeysByteCount in the if.

Then I'd have to reset batchGCKeysByteCount to byteCount if the if clause is true, which I find less readable.


pkg/storage/gc_queue_test.go, line 917 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

nit: const keyCount = 1000

Done.


pkg/storage/gc_queue_test.go, line 919 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

`t.Fatalf("expected GC chunk size to be a multiple of %s", keyCount)

Done.


pkg/storage/gc_queue_test.go, line 921 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

nit: can be a const too, I think! Always nice to see at first glance the things that can't change.

Done.


pkg/storage/gc_queue_test.go, line 923 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

// length keySize.

Done.


Comments from Reviewable

@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 4, 2017

:lgtm_strong: 🎉


Reviewed 2 of 2 files at r3.
Review status: all files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


pkg/storage/gc_queue.go, line 800 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Done.

and the txn := is better in the if _, ok branch too. Actually you can just inline the def: txnMap[txnID] = &roachpb.Transaction { ... }


pkg/storage/gc_queue.go, line 79 at r3 (raw file):

	// gcKeyVersionChunkBytes is the threshold size for splitting
	// GCRequests into multiple batches.
	gcKeyVersionChunkBytes = 1024 * 1000

1mb could still show up significantly in latencies. Might be worth starting smaller here. How about 256 KiB (i.e. 1000*256)?


pkg/storage/gc_queue.go, line 570 at r3 (raw file):

) []roachpb.GCRequest {
	var template roachpb.GCRequest
	var ret []roachpb.GCRequest

nit: ret := make([]roachpb.GCRequest, 0, len(gcKeys)+1)


pkg/storage/gc_queue.go, line 818 at r3 (raw file):

// size, add the current timestamp and start a new chunk.

(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

@spencerkimball
Copy link
Copy Markdown
Member Author

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…

and the txn := is better in the if _, ok branch too. Actually you can just inline the def: txnMap[txnID] = &roachpb.Transaction { ... }

Done.


pkg/storage/gc_queue.go, line 79 at r3 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

1mb could still show up significantly in latencies. Might be worth starting smaller here. How about 256 KiB (i.e. 1000*256)?

Done.


pkg/storage/gc_queue.go, line 570 at r3 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

nit: ret := make([]roachpb.GCRequest, 0, len(gcKeys)+1)

Done.


pkg/storage/gc_queue.go, line 818 at r3 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

// size, add the current timestamp and start a new chunk.

(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).

Done.


Comments from Reviewable

@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 4, 2017

Reviewed 1 of 1 files at r4.
Review status: all files reviewed at latest revision, all discussions resolved, some commit checks failed.


Comments from Reviewable

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.
@spencerkimball spencerkimball requested a review from a team as a code owner December 4, 2017 18:24
@spencerkimball spencerkimball merged commit c58a3fa into cockroachdb:master Dec 4, 2017
@spencerkimball spencerkimball deleted the local-gc branch December 4, 2017 19:12
tbg added a commit to tbg/cockroach that referenced this pull request Dec 24, 2017
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.
tbg added a commit to tbg/cockroach that referenced this pull request Dec 27, 2017
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.
tbg added a commit to tbg/cockroach that referenced this pull request Dec 28, 2017
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.
tbg added a commit to tbg/cockroach that referenced this pull request Dec 28, 2017
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.
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.

3 participants