Skip to content

Commit

Permalink
roachpb,gc,mvcc: add UseClearRange option to GCRequest_GCKey
Browse files Browse the repository at this point in the history
This commit adds an optimization to massively reduce the overhead of
garbage collecting large numbers of versions. When running garbage
collection, we currently iterate through the store to send (Key, Timestamp)
pairs in a GCRequest to the store to be evaluated and applied. This can
be rather slow when deleting large numbers of keys, particularly due to the
need to paginate. The primary motivation for pagination is to ensure that
we do not create raft commands with deletions that are too large.

In practice, we find that for the tiny values in a sequence, we find that we
can GC around 1800 versions/s with cockroachdb#51184 and around 900 without it (though
note that in that PR the more versions exist, the worse the throughput will
be). This remains abysmally slow. I imagine that using larger batches could
be one approach to increase the throughput, but, as it stands, 256 KiB is
not a tiny raft command.

This instead turns to the ClearRange operation which can delete all of versions
of a key with the replication overhead of just two copies. This approach is
somewhat controversial because, as @petermattis puts it:

```
We need to be careful about this. Historically, adding many range tombstones
was very bad for performance. I think we resolved most (all?) of those issues,
but I'm still nervous about embracing using range tombstones below the level
of a Range.
```

Nevertheless, the results are enticing. Rather than pinning a core at full
utilization for minutes just to clear the versions written to a sequence over
the course of a bit more than an hour, we can clear that in ~2 seconds.

Release note (performance improvement): Improved performance of garbage
collection in the face of large numbers of versions.
  • Loading branch information
ajwerner committed Jul 9, 2020
1 parent 3a1e899 commit 38441b9
Show file tree
Hide file tree
Showing 6 changed files with 741 additions and 607 deletions.
35 changes: 34 additions & 1 deletion c-deps/libroach/protos/roachpb/api.pb.cc

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

21 changes: 21 additions & 0 deletions c-deps/libroach/protos/roachpb/api.pb.h

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

56 changes: 44 additions & 12 deletions pkg/kv/kvserver/gc/gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,11 +276,13 @@ func processReplicatedKeyRange(
// version for a key has been reached, if haveGarbageForThisKey, we'll add the
// current key to the batch with the gcTimestampForThisKey.
var (
batchGCKeys []roachpb.GCRequest_GCKey
batchGCKeysBytes int64
haveGarbageForThisKey bool
gcTimestampForThisKey hlc.Timestamp
sentBatchForThisKey bool
batchGCKeys []roachpb.GCRequest_GCKey
batchGCKeysBytes int64
haveGarbageForThisKey bool
gcTimestampForThisKey hlc.Timestamp
keyBytesForThisKey int64
sentBatchForThisKey bool
useClearRangeForThisKey bool
)
it := makeGCIterator(desc, snap)
defer it.close()
Expand All @@ -302,7 +304,12 @@ func processReplicatedKeyRange(
isNewest := s.curIsNewest()
if isGarbage(threshold, s.cur, s.next, isNewest) {
keyBytes := int64(s.cur.Key.EncodedSize())
batchGCKeysBytes += keyBytes
// If we have decided that we're going to use clear range for this key,
// we've already accounted for the overhead of those key bytes.
if !useClearRangeForThisKey {
batchGCKeysBytes += keyBytes
keyBytesForThisKey += keyBytes
}
haveGarbageForThisKey = true
gcTimestampForThisKey = s.cur.Key.Timestamp
info.AffectedVersionsKeyBytes += keyBytes
Expand All @@ -311,23 +318,48 @@ func processReplicatedKeyRange(
if affected := isNewest && (sentBatchForThisKey || haveGarbageForThisKey); affected {
info.NumKeysAffected++
}
shouldSendBatch := batchGCKeysBytes >= KeyVersionChunkBytes
if shouldSendBatch || isNewest && haveGarbageForThisKey {

atBatchSizeLimit := batchGCKeysBytes >= KeyVersionChunkBytes
if atBatchSizeLimit && !useClearRangeForThisKey {
// We choose to use clear range for a key if we'd fill up an entire batch
// with just that key.
//
// TODO(ajwerner): Perhaps we should ensure that there are actually a
// large number of versions utilizing all of these bytes and not a small
// number of versions of a very large key. What's the right minimum number
// of keys?
useClearRangeForThisKey = len(batchGCKeys) == 0
if useClearRangeForThisKey {
// Adjust the accounting for the size of this batch given that now
// we're going to deal with this key using clear range.
batchGCKeysBytes -= keyBytesForThisKey
batchGCKeysBytes += 2 * int64(s.cur.Key.EncodedSize())
keyBytesForThisKey = 0
}
}

if addKeyToBatch := (atBatchSizeLimit && !useClearRangeForThisKey) ||
(isNewest && haveGarbageForThisKey); addKeyToBatch {
alloc, s.cur.Key.Key = alloc.Copy(s.cur.Key.Key, 0)
batchGCKeys = append(batchGCKeys, roachpb.GCRequest_GCKey{
Key: s.cur.Key.Key,
Timestamp: gcTimestampForThisKey,
Key: s.cur.Key.Key,
Timestamp: gcTimestampForThisKey,
UseClearRange: useClearRangeForThisKey,
})
haveGarbageForThisKey = false
gcTimestampForThisKey = hlc.Timestamp{}
keyBytesForThisKey = 0
useClearRangeForThisKey = false

// Mark that we sent a batch for this key so we know that we had garbage
// even if it turns out that there's no more garbage for this key.
// We want to count a key as affected once even if we paginate the
// deletion of its versions.
sentBatchForThisKey = shouldSendBatch && !isNewest
sentBatchForThisKey = atBatchSizeLimit && !isNewest
}
if shouldSendBatch {

if shouldSendBatch := (atBatchSizeLimit && !useClearRangeForThisKey) ||
(isNewest && useClearRangeForThisKey); shouldSendBatch {
if err := gcer.GC(ctx, batchGCKeys); err != nil {
// Even though we are batching the GC process, it's
// safe to continue because we bumped the GC
Expand Down
Loading

0 comments on commit 38441b9

Please sign in to comment.