diff --git a/pkg/kv/kvserver/batcheval/cmd_gc.go b/pkg/kv/kvserver/batcheval/cmd_gc.go index 861eb2cdbc76..c07613882c2f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_gc.go +++ b/pkg/kv/kvserver/batcheval/cmd_gc.go @@ -54,11 +54,30 @@ func declareKeysGC( Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), }) } - // For ClearRangeKey request we still obtain a wide write lock as we don't - // expect any operations running on the range. - if rk := gcr.ClearRangeKey; rk != nil { - latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: rk.StartKey, EndKey: rk.EndKey}, - hlc.MaxTimestamp) + // We have three distinct cases for GCClearRange. + // First is when range has no timestamp which means we will attempt to delete + // all data in range and that requires wide lock of the whole key span. This + // is fine as we are discarding all data in range. + // Second is deleting multiple versions of a single key when most recent data + // remains. This case is not different from non-ranged GC for individual keys. + // See explanation of correctness below why we can avoid obtaining locks. + // Thirs case is deleting multiple keys, and for it we need to obtain a write + // lock to prevent any addition of the keys that would be covered by the + // request range span. We will obtain it on highest timestamp to avoid + // interference with readers that want to read keys. It is safe to read as per + // correctness explanation below because we only delete data below threshold + // and that data is not readable. + // This lock could create contention for writers, but in practice if range + // contain live data, we will not create wide GCClearRange requests. And if + // there's no live data for at least gc.ttl period, then it is not likely + // there would be other writes to the range. + // A corner case could be if writes were stopped for GCTTL and then resumed, + // they could hit contention on the first GC run. + if rk := gcr.ClearRange; rk != nil { + if rk.StartKeyTimestamp.IsEmpty() || !rk.StartKey.Next().Equal(rk.EndKey) { + latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: rk.StartKey, EndKey: rk.EndKey}, + hlc.MaxTimestamp) + } } // The RangeGCThresholdKey is only written to if the // req.(*GCRequest).Threshold is set. However, we always declare an exclusive @@ -143,7 +162,7 @@ func GC( // GC request's effect from the raft log. Latches held on the leaseholder // would have no impact on a follower read. if !args.Threshold.IsEmpty() && - (len(args.Keys) != 0 || len(args.RangeKeys) != 0 || args.ClearRangeKey != nil) && + (len(args.Keys) != 0 || len(args.RangeKeys) != 0 || args.ClearRange != nil) && !cArgs.EvalCtx.EvalKnobs().AllowGCWithNewThresholdAndKeys { return result.Result{}, errors.AssertionFailedf( "GC request can set threshold or it can GC keys, but it is unsafe for it to do both") @@ -151,9 +170,10 @@ func GC( // We do not allow removal of point or range keys combined with clear range // operation as they could cover the same set of keys. - if (len(args.Keys) != 0 || len(args.RangeKeys) != 0) && args.ClearRangeKey != nil { + if (len(args.Keys) != 0 || len(args.RangeKeys) != 0) && + args.ClearRange != nil { return result.Result{}, errors.AssertionFailedf( - "GC request can remove point and range keys or clear entire range, but it is unsafe for it to do both") + "GC request can remove point and range keys or clear range, but it is unsafe for it to do both") } // All keys must be inside the current replica range. Keys outside @@ -184,10 +204,36 @@ func GC( } } + desc := cArgs.EvalCtx.Desc() + + if cr := args.ClearRange; cr != nil { + // Check if we are performing a fast path operation to try to remove all user + // key data from the range. All data must be deleted by a range tombstone for + // the operation to succeed. + if cr.StartKeyTimestamp.IsEmpty() { + if !cr.StartKey.Equal(desc.StartKey.AsRawKey()) || !cr.EndKey.Equal(desc.EndKey.AsRawKey()) { + return result.Result{}, errors.Errorf("gc with clear range operation could only be used on the full range") + } + + if err := storage.MVCCGarbageCollectWholeRange(ctx, readWriter, cArgs.Stats, + cr.StartKey, cr.EndKey, cArgs.EvalCtx.GetGCThreshold(), + cArgs.EvalCtx.GetMVCCStats()); err != nil { + return result.Result{}, err + } + } else { + // Otherwise garbage collect specified keys defined by clear range i.e. parts + // of the whole range containing no live data. + if err := storage.MVCCGarbageCollectPointsWithClearRange(ctx, readWriter, cArgs.Stats, + cr.StartKey, cr.EndKey, cr.StartKeyTimestamp, + cArgs.EvalCtx.GetGCThreshold()); err != nil { + return result.Result{}, err + } + } + } + // Garbage collect range keys. Note that we pass latch range boundaries for // each key as we may need to merge range keys with adjacent ones, but we // are restricted on how far we are allowed to read. - desc := cArgs.EvalCtx.Desc() rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), args.RangeKeys) if err := storage.MVCCGarbageCollectRangeKeys(ctx, readWriter, cArgs.Stats, rangeKeys); err != nil { @@ -196,18 +242,6 @@ func GC( var res result.Result - // Fast path operation to try to remove all user key data from the range. - if rk := args.ClearRangeKey; rk != nil { - if !rk.StartKey.Equal(desc.StartKey.AsRawKey()) || !rk.EndKey.Equal(desc.EndKey.AsRawKey()) { - return result.Result{}, errors.Errorf("gc with clear range operation could only be used on the full range") - } - - if err := storage.MVCCGarbageCollectWholeRange(ctx, readWriter, cArgs.Stats, - rk.StartKey, rk.EndKey, cArgs.EvalCtx.GetGCThreshold(), cArgs.EvalCtx.GetMVCCStats()); err != nil { - return result.Result{}, err - } - } - gcThreshold := cArgs.EvalCtx.GetGCThreshold() // Optionally bump the GC threshold timestamp. if !args.Threshold.IsEmpty() { @@ -234,7 +268,7 @@ func GC( // unnecessarily GC'd with high priority again. // We should only do that when we are doing actual cleanup as we want to have // a hint when request is being handled. - if len(args.Keys) != 0 || len(args.RangeKeys) != 0 || args.ClearRangeKey != nil { + if len(args.Keys) != 0 || len(args.RangeKeys) != 0 || args.ClearRange != nil { sl := MakeStateLoader(cArgs.EvalCtx) hint, err := sl.LoadGCHint(ctx, readWriter) if err != nil { diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index 1d83de4e9791..516e6a4f4d85 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -388,11 +388,11 @@ type uniformDistSpec struct { deleteFrac float64 // keysPerValue parameters determine number of versions for a key. This number // includes tombstones and intents which may be present on top of the history. - keysPerValueMin, keysPerValueMax int + versionsPerKeyMin, versionsPerKeyMax int // Fractions define how likely is that a key will belong to one of categories. // If we only had a single version for each key, then that would be fraction // of total number of objects, but if we have many versions, this value would - // roughly be total objects/avg(keysPerValueMin, keysPerValueMax) * frac. + // roughly be total objects/avg(versionsPerKeyMin, versionsPerKeyMax) * frac. intentFrac, oldIntentFrac float64 rangeKeyFrac float64 } @@ -414,7 +414,7 @@ func (ds uniformDistSpec) dist(maxRows int, rng *rand.Rand) dataDistribution { uniformTableStringKeyDistribution(ds.desc().StartKey.AsRawKey(), ds.keySuffixMin, ds.keySuffixMax, rng), uniformValueStringDistribution(ds.valueLenMin, ds.valueLenMax, ds.deleteFrac, rng), - uniformValuesPerKey(ds.keysPerValueMin, ds.keysPerValueMax, rng), + uniformVersionsPerKey(ds.versionsPerKeyMin, ds.versionsPerKeyMax, rng), ds.intentFrac, ds.oldIntentFrac, ds.rangeKeyFrac, @@ -436,12 +436,12 @@ func (ds uniformDistSpec) String() string { "ts=[%d,%d],"+ "keySuffix=[%d,%d],"+ "valueLen=[%d,%d],"+ - "keysPerValue=[%d,%d],"+ + "versionsPerKey=[%d,%d],"+ "deleteFrac=%f,intentFrac=%f,oldIntentFrac=%f,rangeFrac=%f", ds.tsSecFrom, ds.tsSecTo, ds.keySuffixMin, ds.keySuffixMax, ds.valueLenMin, ds.valueLenMax, - ds.keysPerValueMin, ds.keysPerValueMax, + ds.versionsPerKeyMin, ds.versionsPerKeyMax, ds.deleteFrac, ds.intentFrac, ds.oldIntentFrac, ds.rangeKeyFrac) } @@ -497,7 +497,7 @@ func uniformValueStringDistribution( } } -func uniformValuesPerKey(valuesPerKeyMin, valuesPerKeyMax int, rng *rand.Rand) func() int { +func uniformVersionsPerKey(valuesPerKeyMin, valuesPerKeyMax int, rng *rand.Rand) func() int { if valuesPerKeyMin > valuesPerKeyMax { panic(fmt.Errorf("min (%d) > max (%d)", valuesPerKeyMin, valuesPerKeyMax)) } diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 5194fd3bcea4..40182aad27df 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -19,6 +19,8 @@ import ( "context" "fmt" "math" + "sort" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -46,6 +48,19 @@ const ( // Raft command for each GCRequest does not significantly exceed // this threshold. KeyVersionChunkBytes = base.ChunkRaftCommandThresholdBytes + // defaultClearRangeMinKeys is min number of versions we will try to + // remove using clear range. If we have less than this number, using pebble + // range tombstone is more expensive than using multiple point deletions. + defaultClearRangeMinKeys = 2000 + // defaultMaxPendingKeysSize is a max amount of memory used to store pending + // keys while trying to decide if clear range operation is feasible. If we + // don't have enough keys when this value is reached, points GC operation + // will be flushed to free memory even if we still can potentially issue + // clear range in the future. + defaultMaxPendingKeysSize = 64 << 20 + // hlcTimestampSize is the size of hlc timestamp used for mem usage + // calculations. + hlcTimestampSize = 16 ) // IntentAgeThreshold is the threshold after which an extant intent @@ -116,6 +131,16 @@ var MaxIntentKeyBytesPerCleanupBatch = settings.RegisterIntSetting( settings.NonNegativeInt, ) +// ClearRangeMinKeys is a minimum number of keys that GC will consider +// for executing a ClearRange request. Since ClearRange requests translate into +// pebble tombstones, they should be used on large sequence of keys only. +var ClearRangeMinKeys = settings.RegisterIntSetting(settings.SystemOnly, + "kv.gc.clear_range_min_keys", + "if non zero, gc will issue clear range requests if number of consecutive garbage keys exceeds this threshold", + defaultClearRangeMinKeys, + settings.NonNegativeInt, +) + // AdmissionPriority determines the admission priority level to use for MVCC GC // work. var AdmissionPriority = settings.RegisterEnumSetting( @@ -152,7 +177,7 @@ type Thresholder interface { // PureGCer is part of the GCer interface. type PureGCer interface { GC(context.Context, []roachpb.GCRequest_GCKey, []roachpb.GCRequest_GCRangeKey, - *roachpb.GCRequest_GCClearRangeKey, + *roachpb.GCRequest_GCClearRange, ) error } @@ -175,7 +200,7 @@ func (NoopGCer) GC( context.Context, []roachpb.GCRequest_GCKey, []roachpb.GCRequest_GCRangeKey, - *roachpb.GCRequest_GCClearRangeKey, + *roachpb.GCRequest_GCClearRange, ) error { return nil } @@ -236,12 +261,12 @@ type Info struct { // AffectedVersionsRangeValBytes is the number of (fully encoded) bytes deleted from values that // belong to removed range keys. AffectedVersionsRangeValBytes int64 - // ClearRangeKeyOperations reports 1 if GC succeeded performing collection with - // ClearRange operation. - ClearRangeKeyOperations int - // ClearRangeKeyFailures reports 1 if GC identified a possibility to collect - // with ClearRange operation, but request failed. - ClearRangeKeyFailures int + // ClearRangeSpanOperations number of ClearRange requests performed by GC. This + // number includes full range clear requests as well as requests covering + // multiple keys or versions of the same key. + ClearRangeSpanOperations int + // ClearRangeSpanFailures number of ClearRange requests GC failed to perform. + ClearRangeSpanFailures int } // RunOptions contains collection of limits that GC run applies when performing operations @@ -265,6 +290,22 @@ type RunOptions struct { // considered abandoned and fit for removal, as measured by the maximum of // its last heartbeat and read timestamp. TxnCleanupThreshold time.Duration + // ClearRangeMinKeys is minimum number of keys to delete with a ClearRange + // operation. If there's less than this number of keys to delete, GC would + // fall back to point deletions. + // 0 means usage of clear range requests is disabled. + ClearRangeMinKeys int64 + // MaxKeyVersionChunkBytes is the max size of keys for all versions of objects a single gc + // batch would contain. This includes not only keys within the request, but also all versions + // covered below request keys. This is important because of the resulting raft command size + // generated in response to GC request. + MaxKeyVersionChunkBytes int64 + // MaxPendingKeysSize maximum amount of bytes of pending GC batches kept in + // memory while searching for eligible clear range span. If span of minimum + // configured length is not found before this limit is reached, gc will resort + // to issuing point delete requests for the oldest batch to free up memory + // before resuming further iteration. + MaxPendingKeysSize int64 } // CleanupIntentsFunc synchronously resolves the supplied intents @@ -310,7 +351,9 @@ func Run( Threshold: newThreshold, } - fastPath, err := processReplicatedKeyRange(ctx, desc, snap, now, newThreshold, options.IntentAgeThreshold, gcer, + fastPath, err := processReplicatedKeyRange(ctx, desc, snap, now, newThreshold, options.IntentAgeThreshold, + populateBatcherOptions(options), + gcer, intentBatcherOptions{ maxIntentsPerIntentCleanupBatch: options.MaxIntentsPerIntentCleanupBatch, maxIntentKeyBytesPerIntentCleanupBatch: options.MaxIntentKeyBytesPerIntentCleanupBatch, @@ -328,7 +371,8 @@ func Run( // From now on, all keys processed are range-local and inline (zero timestamp). // Process local range key entries (txn records, queue last processed times). - if err := processLocalKeyRange(ctx, snap, desc, txnExp, &info, cleanupTxnIntentsAsyncFn, gcer); err != nil { + if err := processLocalKeyRange(ctx, snap, desc, txnExp, &info, cleanupTxnIntentsAsyncFn, + gcer); err != nil { if errors.Is(err, ctx.Err()) { return Info{}, err } @@ -349,6 +393,24 @@ func Run( return info, nil } +func populateBatcherOptions(options RunOptions) gcKeyBatcherThresholds { + batcherOptions := gcKeyBatcherThresholds{ + clearRangeMinKeys: int(options.ClearRangeMinKeys), + batchGCKeysBytesThreshold: options.MaxKeyVersionChunkBytes, + maxPendingKeysSize: int(options.MaxPendingKeysSize), + } + if batcherOptions.clearRangeMinKeys > 0 { + batcherOptions.clearRangeEnabled = true + } + if batcherOptions.batchGCKeysBytesThreshold == 0 { + batcherOptions.batchGCKeysBytesThreshold = KeyVersionChunkBytes + } + if batcherOptions.maxPendingKeysSize == 0 { + batcherOptions.maxPendingKeysSize = defaultMaxPendingKeysSize + } + return batcherOptions +} + // processReplicatedKeyRange identifies garbage and sends GC requests to // remove it. // @@ -362,7 +424,8 @@ func processReplicatedKeyRange( now hlc.Timestamp, threshold hlc.Timestamp, intentAgeThreshold time.Duration, - gcer GCer, + batcherThresholds gcKeyBatcherThresholds, + gcer PureGCer, options intentBatcherOptions, cleanupIntentsFn CleanupIntentsFunc, info *Info, @@ -376,154 +439,514 @@ func processReplicatedKeyRange( end := desc.EndKey.AsRawKey() if coveredByRangeTombstone, err := storage.CanGCEntireRange(ctx, snap, start, end, threshold); err == nil && coveredByRangeTombstone { - if err = gcer.GC(ctx, nil, nil, &roachpb.GCRequest_GCClearRangeKey{ + if err = gcer.GC(ctx, nil, nil, &roachpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, }); err == nil { excludeUserKeySpan = true - info.ClearRangeKeyOperations++ + info.ClearRangeSpanOperations++ } else { log.Warningf(ctx, "failed to perform GC clear range operation on range %s: %s", desc.String(), err) - info.ClearRangeKeyFailures++ + info.ClearRangeSpanFailures++ } } } - var alloc bufalloc.ByteAllocator // Compute intent expiration (intent age at which we attempt to resolve). intentExp := now.Add(-intentAgeThreshold.Nanoseconds(), 0) - intentBatcher := newIntentBatcher(cleanupIntentsFn, options, info) - - // handleIntent will deserialize transaction info and if intent is older than - // threshold enqueue it to batcher, otherwise ignore it. - handleIntent := func(keyValue *mvccKeyValue) error { - meta := &enginepb.MVCCMetadata{} - if err := protoutil.Unmarshal(keyValue.metaValue, meta); err != nil { - log.Errorf(ctx, "unable to unmarshal MVCC metadata for key %q: %+v", keyValue.key, err) + return excludeUserKeySpan, rditer.IterateMVCCReplicaKeySpans(desc, snap, rditer.IterateOptions{ + CombineRangesAndPoints: true, + Reverse: true, + ExcludeUserKeySpan: excludeUserKeySpan, + }, func(iterator storage.MVCCIterator, span roachpb.Span, keyType storage.IterKeyType) error { + intentBatcher := newIntentBatcher(cleanupIntentsFn, options, info) + + // handleIntent will deserialize transaction info and if intent is older than + // threshold enqueue it to batcher, otherwise ignore it. + handleIntent := func(keyValue *mvccKeyValue) error { + meta := &enginepb.MVCCMetadata{} + if err := protoutil.Unmarshal(keyValue.metaValue, meta); err != nil { + log.Errorf(ctx, "unable to unmarshal MVCC metadata for key %q: %+v", keyValue.key, err) + return nil + } + if meta.Txn != nil { + // Keep track of intent to resolve if older than the intent + // expiration threshold. + if meta.Timestamp.ToTimestamp().Less(intentExp) { + info.IntentsConsidered++ + if err := intentBatcher.addAndMaybeFlushIntents(ctx, keyValue.key.Key, meta); err != nil { + if errors.Is(err, ctx.Err()) { + return err + } + log.Warningf(ctx, "failed to cleanup intents batch: %v", err) + } + } + } return nil } - if meta.Txn != nil { - // Keep track of intent to resolve if older than the intent - // expiration threshold. - if meta.Timestamp.ToTimestamp().Less(intentExp) { - info.IntentsConsidered++ - if err := intentBatcher.addAndMaybeFlushIntents(ctx, keyValue.key.Key, meta); err != nil { - if errors.Is(err, ctx.Err()) { - return err - } - log.Warningf(ctx, "failed to cleanup intents batch: %v", err) + + // Iterate all versions of all keys from oldest to newest. If a version is an + // intent it will have the highest timestamp of any versions and will be + // followed by a metadata entry. + // The loop determines if next object is garbage, non-garbage or intent and + // notifies batcher with its detail. Batcher is responsible for accumulating + // pending key data and sending sending keys to GCer as needed. + // It could also request the main loop to rewind to a previous point to + // retry (this is needed when attempt to collect a clear range batch fails + // in the middle of key versions). + it := makeGCIterator(iterator, threshold) + + b := gcKeyBatcher{ + gcKeyBatcherThresholds: batcherThresholds, + gcer: gcer, + info: info, + pointsBatches: make([]pointsBatch, 1), + // We must clone here as we reuse key slice to avoid realocating on every + // key. + clearRangeEndKey: span.EndKey.Clone(), + prevWasNewest: true, + } + + for ; ; it.step() { + var err error + + s, ok := it.state() + if !ok { + if it.err != nil { + return it.err } + break + } + + switch { + case s.curIsNotValue(): + // Skip over non mvcc data and intents. + err = b.foundNonGCableData(ctx, s.cur, true /* isNewestPoint */) + case s.curIsIntent(): + err = b.foundNonGCableData(ctx, s.cur, true /* isNewestPoint */) + if err != nil { + return err + } + if err = handleIntent(s.next); err != nil { + return err + } + // For intents, we force step over the intent metadata after provisional + // value is found. + it.step() + default: + if isGarbage(threshold, s.cur, s.next, s.curIsNewest(), s.firstRangeTombstoneTsAtOrBelowGC) { + err = b.foundGarbage(ctx, s.cur, s.curLastKeyVersion()) + } else { + err = b.foundNonGCableData(ctx, s.cur, s.curLastKeyVersion()) + } + } + if err != nil { + return err } } + + err := b.flushLastBatch(ctx) + if err != nil { + return err + } + + // We need to send out last intent cleanup batch. + if err := intentBatcher.maybeFlushPendingIntents(ctx); err != nil { + if errors.Is(err, ctx.Err()) { + return err + } + log.Warningf(ctx, "failed to cleanup intents batch: %v", err) + } return nil + }) +} + +// gcBatchCounters contain statistics about garbage that is collected for the +// range of keys. +type gcBatchCounters struct { + keyBytes, valBytes int64 + // keysAffected includes key when it is first encountered, but would not be + // included if key versions were split between batches. + keysAffected int + // memUsed is amount of bytes batch takes in memory (sum of all keys and + // timestamps excluding any supporting structures). + memUsed int + // versionsAffected is a number of key versions covered by batch. This is + // used for clear range accounting. + versionsAffected int +} + +func (c gcBatchCounters) updateGcInfo(info *Info) { + info.AffectedVersionsKeyBytes += c.keyBytes + info.AffectedVersionsValBytes += c.valBytes + info.NumKeysAffected += c.keysAffected +} + +func (c gcBatchCounters) String() string { + return fmt.Sprintf("Counters{keyBytes=%d, valBytes=%d, keys=%d, versions=%d, memUsed=%d}", + c.keyBytes, c.valBytes, c.keysAffected, c.versionsAffected, c.memUsed) +} + +// gcKeyBatcherThresholds collection configuration options. +type gcKeyBatcherThresholds struct { + batchGCKeysBytesThreshold int64 + clearRangeMinKeys int + clearRangeEnabled bool + maxPendingKeysSize int +} + +type pointsBatch struct { + gcBatchCounters + batchGCKeys []roachpb.GCRequest_GCKey + alloc bufalloc.ByteAllocator +} + +func (b pointsBatch) String() string { + return fmt.Sprintf("PointsBatch{keyCount=%d, counters=%s}", + len(b.batchGCKeys), b.gcBatchCounters.String()) +} + +// gcKeyBatcher is responsible for collecting MVCCKeys and feeding them to GCer +// for removal. +// It is receiving notifications if next key is garbage or not and makes decision +// how to batch the data and weather to use point deletion requests or clear +// range requests. +// Batcher will create GC requests containing either collection of individual +// mvcc keys (batch) or clear range span. It will look on three thresholds when +// building a request: +// - clearRangeMinKeys - minimum number of sequential keys eligible for clear +// range +// - batchGCKeysBytesThreshold - maximum byte size of all deleted key versions +// per request +// - maxPendingKeysSize - maximum number of bytes taken by pending requests in +// memory +// +// Batcher tries to find the longest sequence of keys to delete and if the +// sequence is not found (because of non collectable data inbetween) it will +// resort to sending request with point deletions. +// To achieve this behaviour batcher will collect point batches in parallel to +// tracking the sequence length. Once minimum sequence length is reached, it +// stops collecting point batches since we are guaranteed to send a clear range. +// If however it reaches maxPendingKeysSize limit before clearRangeMinKeys is +// met (that would happen if keys are particularly large) then the oldest batch +// is sent, and clear range end is moved to the lowest key if sent batch. +// Once non-garbage key is found, if sequence length is at or above +// clearRangeMinKeys threshold, GC will first send relevant portion of first +// points batch (up to the first key covered by clear range) and then clear +// range request for the remainder of keys. After that it drops all pending +// batches. +// If however the sequence is too short then batcher will send all previous +// batches up to the current one. Record current key as an end of clear range +// span (remember that we are iterating from back to front), record current +// batch counters in partialPointBatchCounters and restart sequence +// search. partialPointBatchCounters is required to make accurate counter +// updates when clear range starts mid points batch. It covers part of the +// first batch from its beginning to the point where clear range could start, +// and remainder of the batch is covered by counters for clear range span. +// Described behaviour support the following pending batches invariant: +// - only the oldest batch in the history could be split by non-garbage key in +// the middle, other batches always cover contiguous sets of keys. +type gcKeyBatcher struct { + gcKeyBatcherThresholds + + // prevWasNewest indicates that we are at oldest version of the key. + // its value is a saved isNewest from previous key. + prevWasNewest bool + // Points batches are accumulated until we cross clear range threshold. + // There's always at least one points batch initialized. + pointsBatches []pointsBatch + // totalMemUsed sum of memUsed of all points batches cached to avoid + // recomputing on every check. + totalMemUsed int + + // Tracking of clear range requests, only used if clearRangeEnabled is true. + + // clearRangeEndKey contains a key following first garbage key found. It would + // be greater or equal to the last key (lowest, collected last) of the first + // (collected first, containing highest keys) points batch. It is guaranteed + // that there are no non-garbage keys between this key and the start of first + // batch. + // Updated whenever batcher is provided with latest version of non garbage + // key or when oldest points batch is flushed. + clearRangeEndKey roachpb.Key + // Last garbage key found. Its Key component is updated whenever new key is + // found, timestamp component updated on each garbage key. Its slice is shared + // with points batch allocator if batcher didn't reach min key size, or shared + // with clearRangeStartKeyAlloc which is used to eliminate unnecessary + // allocations for the start key. + clearRangeStartKey storage.MVCCKey + // We use this key slice to avoid allocating for every new key. We can't + // blindly reuse slice in clearRangeStartKey because we avoid allocations + // for clearRangeStartKey if they could be shared with points batch. + // This is saving us any extra key copying if we can't find enough consecutive + // keys. + clearRangeStartKeyAlloc roachpb.Key + // GC Counters for data covered by currently tracked clear range span. + clearRangeCounters gcBatchCounters + // Stats counters saved from points batch when clear range tracking started. + // Those stats combined with clearRangeCounters cover all unprocessed keys + // since last sent request. + partialPointBatchCounters gcBatchCounters + + gcer PureGCer + info *Info +} + +func (b *gcKeyBatcher) String() string { + pbs := make([]string, len(b.pointsBatches)) + for i, pb := range b.pointsBatches { + pbs[i] = pb.String() } + return fmt.Sprintf("gcKeyBatcher{batches=[%s], clearRangeCnt=%s, partialBatchCnt=%s, totalMem=%d}", + strings.Join(pbs, ", "), + b.clearRangeCounters.String(), + b.partialPointBatchCounters.String(), + b.totalMemUsed) +} - // Iterate all versions of all keys from oldest to newest. If a version is an - // intent it will have the highest timestamp of any versions and will be - // followed by a metadata entry. The loop will determine whether a given key - // has garbage and, if so, will determine the timestamp of the latest version - // which is garbage to be added to the current batch. If the current version - // pushes the size of keys to be removed above the limit, the current key will - // be added with that version and the batch will be sent. When the newest - // 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 - ) - it := makeGCIterator(desc, snap, threshold, excludeUserKeySpan) - defer it.close() - for ; ; it.step() { - s, ok := it.state() - if !ok { - if it.err != nil { - return false, it.err +func (b *gcKeyBatcher) foundNonGCableData( + ctx context.Context, cur *mvccKeyValue, isNewestPoint bool, +) (err error) { + b.prevWasNewest = isNewestPoint + if !b.clearRangeEnabled { + return nil + } + + // Check if there are any complete clear range or point batches collected + // and flush them as we reached end of current consecutive key span. + err = b.maybeFlushPendingBatches(ctx) + if err != nil { + return err + } + + if isNewestPoint { + b.clearRangeEndKey = append(b.clearRangeEndKey[:0], cur.key.Key...) + } + return nil +} + +func (b *gcKeyBatcher) foundGarbage( + ctx context.Context, cur *mvccKeyValue, isNewestPoint bool, +) (err error) { + // If we are restarting clear range collection, then last points batch might + // be reverted to partial at the time of flush. We will save its + // pre-clear-range state and use for GC stats update. + if b.clearRangeEnabled && b.prevWasNewest && b.clearRangeCounters.versionsAffected == 0 { + b.partialPointBatchCounters = b.pointsBatches[len(b.pointsBatches)-1].gcBatchCounters + } + + var key roachpb.Key + var keySize = int64(cur.key.EncodedSize()) + // If still collecting points, add to points first. + if !b.clearRangeEnabled || b.clearRangeCounters.versionsAffected < b.clearRangeMinKeys { + // First check if current batch is full and add a new points batch. + i := len(b.pointsBatches) - 1 + if b.pointsBatches[i].gcBatchCounters.keyBytes >= b.batchGCKeysBytesThreshold { + // If clear range is disabled, flush batches immediately as they are + // formed. + if !b.clearRangeEnabled { + if err = b.flushPointsBatch(ctx, &b.pointsBatches[i]); err != nil { + return err + } + } else { + b.pointsBatches = append(b.pointsBatches, pointsBatch{}) + i++ } - break } - if s.curIsNotValue() { // Step over metadata or other system keys - continue + if b.prevWasNewest { + b.pointsBatches[i].gcBatchCounters.keysAffected++ } - if s.curIsIntent() { - if err := handleIntent(s.next); err != nil { - return false, err + // Whenever new key is started or new batch is started with the same key in + // it, record key value using batches' allocator. + if b.prevWasNewest || len(b.pointsBatches[i].batchGCKeys) == 0 { + b.pointsBatches[i].alloc, key = b.pointsBatches[i].alloc.Copy(cur.key.Key, 0) + b.pointsBatches[i].batchGCKeys = append(b.pointsBatches[i].batchGCKeys, + roachpb.GCRequest_GCKey{Key: key, Timestamp: cur.key.Timestamp}) + keyMemUsed := len(key) + hlcTimestampSize + b.pointsBatches[i].memUsed += keyMemUsed + b.totalMemUsed += keyMemUsed + } else { + // For already registered key just bump the timestamp to the newer one. + b.pointsBatches[i].batchGCKeys[len(b.pointsBatches[i].batchGCKeys)-1].Timestamp = cur.key.Timestamp + } + b.pointsBatches[i].gcBatchCounters.keyBytes += keySize + b.pointsBatches[i].gcBatchCounters.valBytes += int64(cur.mvccValueLen) + b.pointsBatches[i].gcBatchCounters.versionsAffected++ + + // Check if we exceeded in memory bytes limit allowed for points collection. + if i > 0 && b.totalMemUsed >= b.maxPendingKeysSize { + lastKey := b.pointsBatches[0].batchGCKeys[len(b.pointsBatches[0].batchGCKeys)-1].Key + // If oldest batch intersected with currently tracked clear range request + // then bump clear range end key to the one that follows lowest key of the + // cleared batch. We know that all keys for current key with lower + // timestamps are eligible for collection and end range is exclusive. + // The check is needed because clearRangeEndKey cloud be bumped already + // to current key by non gc data on the previous key, but only now we + // exceeded the size threshold and trying to remove oldest batch. + if b.clearRangeEndKey.Compare(lastKey) > 0 { + flushedCounters := b.pointsBatches[0].gcBatchCounters + // If flushed batch was split by start of the range, then we must remove + // preceding part from clear range total to avoid. + b.clearRangeCounters.keyBytes += b.partialPointBatchCounters.keyBytes - flushedCounters.keyBytes + b.clearRangeCounters.valBytes += b.partialPointBatchCounters.valBytes - flushedCounters.valBytes + b.clearRangeCounters.keysAffected += b.partialPointBatchCounters.keysAffected - flushedCounters.keysAffected + b.clearRangeCounters.versionsAffected += b.partialPointBatchCounters.versionsAffected - flushedCounters.versionsAffected + b.clearRangeCounters.memUsed += b.partialPointBatchCounters.memUsed - flushedCounters.memUsed + b.clearRangeEndKey = lastKey[:len(lastKey):len(lastKey)].Next() } - continue - } - // No more values in buffer or next value has different key. - isNewestPoint := s.curIsNewest() - if garbage, err := - isGarbage(threshold, s.cur, s.next, isNewestPoint, s.firstRangeTombstoneTsAtOrBelowGC); garbage && err == nil { - keyBytes := int64(s.cur.key.EncodedSize()) - batchGCKeysBytes += keyBytes - haveGarbageForThisKey = true - gcTimestampForThisKey = s.cur.key.Timestamp - info.AffectedVersionsKeyBytes += keyBytes - info.AffectedVersionsValBytes += int64(s.cur.mvccValueLen) - } else if err != nil { - return false, err - } - // We bump how many keys were processed when we reach newest key and looking - // if key has garbage or if garbage for this key was included in previous - // batch. - if affected := isNewestPoint && (sentBatchForThisKey || haveGarbageForThisKey); affected { - info.NumKeysAffected++ - // If we reached newest timestamp for the key then we should reset sent - // batch to ensure subsequent keys are not included in affected keys if - // they don't have garbage. - sentBatchForThisKey = false - } - shouldSendBatch := batchGCKeysBytes >= KeyVersionChunkBytes - if shouldSendBatch || isNewestPoint && haveGarbageForThisKey { - 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, - }) - haveGarbageForThisKey = false - gcTimestampForThisKey = hlc.Timestamp{} - - // 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 && !isNewestPoint - } - // If limit was reached, delegate to GC'r to remove collected batch. - if shouldSendBatch { - if err := gcer.GC(ctx, batchGCKeys, nil, nil); err != nil { - if errors.Is(err, ctx.Err()) { - return false, err - } - // Even though we are batching the GC process, it's - // safe to continue because we bumped the GC - // thresholds. We may leave some inconsistent history - // behind, but nobody can read it. - log.Warningf(ctx, "failed to GC a batch of keys: %v", err) + b.partialPointBatchCounters = gcBatchCounters{} + + // We accumulated more keys in memory than allowed by thresholds, we need + // to flush oldest batch to protect node from exhausting memory. + err := b.flushOldestPointBatches(ctx, 1) + if err != nil { + return err } - batchGCKeys = nil - batchGCKeysBytes = 0 - alloc = bufalloc.ByteAllocator{} } } - // We need to send out last intent cleanup batch. - if err := intentBatcher.maybeFlushPendingIntents(ctx); err != nil { - if errors.Is(err, ctx.Err()) { - return false, err + + if b.clearRangeEnabled { + if b.prevWasNewest { + if key == nil { + // Reuse key slice for start range to avoid allocating on every key + // change while tracking clear range. + b.clearRangeStartKeyAlloc = append(b.clearRangeStartKeyAlloc[:0], cur.key.Key...) + key = b.clearRangeStartKeyAlloc + } + b.clearRangeStartKey = storage.MVCCKey{Key: key} + b.clearRangeCounters.keysAffected++ } - log.Warningf(ctx, "failed to cleanup intents batch: %v", err) + b.clearRangeStartKey.Timestamp = cur.key.Timestamp + b.clearRangeCounters.keyBytes += keySize + b.clearRangeCounters.valBytes += int64(cur.mvccValueLen) + b.clearRangeCounters.versionsAffected++ } - if len(batchGCKeys) > 0 { - if err := gcer.GC(ctx, batchGCKeys, nil, nil); err != nil { - return false, err + + b.prevWasNewest = isNewestPoint + return nil +} + +// maybeFlushPendingBatches flushes accumulated GC requests. If clear range is +// used and current data is eligible for deletion then pending points batch and +// clear range request are flushed. +// If there's not enough point keys to justify clear range, then point point +// batches may be flushed: +// - if there's a single batch and it didn't reach desired size, nothing is +// flushed +// - if there's more than single points batch accumulated, then all batches +// except for last are flushed +func (b *gcKeyBatcher) maybeFlushPendingBatches(ctx context.Context) (err error) { + if b.clearRangeEnabled && b.clearRangeCounters.versionsAffected >= b.clearRangeMinKeys { + // Optionally flush parts of the first batch if it is not fully covered by + // pending clear range span. + batchLen := len(b.pointsBatches[0].batchGCKeys) + // Find a key that is equal or less than end of clear range key. + // Batch is sorted in the reverse order, timestamps are not relevant since + // we can't have the same key more than once within single points batch. + lastIdx := sort.Search(batchLen, func(i int) bool { + return b.pointsBatches[0].batchGCKeys[i].Key.Compare(b.clearRangeEndKey) < 0 + }) + if lastIdx == 0 { + // If lastIdx == 0 we can have a clear range request which goes back to + // previous key completely covering this range. If this is true then we + // don't need to handle this batch and can tighten the clear range span + // to cover only up to the start of first batch. + b.clearRangeEndKey = b.pointsBatches[0].batchGCKeys[0].Key.Next() + } else { + b.pointsBatches[0].batchGCKeys = b.pointsBatches[0].batchGCKeys[:lastIdx] + b.pointsBatches[0].gcBatchCounters = b.partialPointBatchCounters + err := b.flushPointsBatch(ctx, &b.pointsBatches[0]) + if err != nil { + return err + } + } + b.pointsBatches = make([]pointsBatch, 1) + + // Flush clear range. + // To reduce locking contention between keys, if we have a range that only + // covers multiple versions of a single key we could avoid latching range + // and fall back to poing GC behaviour where gc threshold guards against + // key interference. We do it by setting end key to StartKey.Next() and thus + // signalling gc request handler to skip locking. + endRange := b.clearRangeEndKey + if b.clearRangeCounters.keysAffected == 1 { + endRange = b.clearRangeStartKey.Key.Next() + } + if err := b.gcer.GC(ctx, nil, nil, &roachpb.GCRequest_GCClearRange{ + StartKey: b.clearRangeStartKey.Key, + StartKeyTimestamp: b.clearRangeStartKey.Timestamp, + EndKey: endRange, + }); err != nil { + if errors.Is(err, ctx.Err()) { + return err + } + // Even though we are batching the GC process, it's + // safe to continue because we bumped the GC + // thresholds. We may leave some inconsistent history + // behind, but nobody can read it. + log.Warningf(ctx, "failed to GC keys with clear range: %v", err) + b.info.ClearRangeSpanFailures++ + } + b.clearRangeCounters.updateGcInfo(b.info) + b.info.ClearRangeSpanOperations++ + b.totalMemUsed = 0 + } else if flushTo := len(b.pointsBatches) - 1; flushTo > 0 { + err := b.flushOldestPointBatches(ctx, flushTo) + if err != nil { + return err } } - return excludeUserKeySpan, nil + b.clearRangeCounters = gcBatchCounters{} + b.clearRangeEndKey = nil + return nil +} + +// flushOldestPointBatches flushes oldest points batch and returns its counters. +// unsafeLastKey must not be retained as it will prevent batch allocator to be +// released. +func (b *gcKeyBatcher) flushOldestPointBatches(ctx context.Context, count int) (err error) { + var i int + for i = 0; i < count; i++ { + if err = b.flushPointsBatch(ctx, &b.pointsBatches[i]); err != nil { + return err + } + } + remaining := len(b.pointsBatches) - i + copy(b.pointsBatches[0:remaining], b.pointsBatches[i:]) + // Zero out remaining batches to free app allocators. + for i = remaining; i < len(b.pointsBatches); i++ { + b.pointsBatches[i] = pointsBatch{} + } + b.pointsBatches = b.pointsBatches[:remaining] + return nil +} + +// flushPointsBatch flushes points batch and zeroes out its content. +func (b *gcKeyBatcher) flushPointsBatch(ctx context.Context, batch *pointsBatch) (err error) { + if err := b.gcer.GC(ctx, batch.batchGCKeys, nil, nil); err != nil { + if errors.Is(err, ctx.Err()) { + return err + } + // Even though we are batching the GC process, it's + // safe to continue because we bumped the GC + // thresholds. We may leave some inconsistent history + // behind, but nobody can read it. + log.Warningf(ctx, "failed to GC a batch of keys: %v", err) + } + batch.gcBatchCounters.updateGcInfo(b.info) + b.totalMemUsed -= batch.gcBatchCounters.memUsed + *batch = pointsBatch{} + return nil +} + +func (b *gcKeyBatcher) flushLastBatch(ctx context.Context) (err error) { + if len(b.pointsBatches[0].batchGCKeys) == 0 { + return nil + } + b.pointsBatches = append(b.pointsBatches, pointsBatch{}) + return b.maybeFlushPendingBatches(ctx) } type intentBatcher struct { @@ -668,10 +1091,10 @@ func isGarbage( cur, next *mvccKeyValue, isNewestPoint bool, firstRangeTombstoneTsAtOrBelowGC hlc.Timestamp, -) (bool, error) { +) bool { // If the value is not at or below the threshold then it's not garbage. if belowThreshold := cur.key.Timestamp.LessEq(threshold); !belowThreshold { - return false, nil + return false } if cur.key.Timestamp.Less(firstRangeTombstoneTsAtOrBelowGC) { if util.RaceEnabled { @@ -680,11 +1103,11 @@ func isGarbage( firstRangeTombstoneTsAtOrBelowGC.String(), threshold.String())) } } - return true, nil + return true } isDelete := cur.mvccValueIsTombstone if isNewestPoint && !isDelete { - return false, nil + return false } // INVARIANT: !isNewestPoint || isDelete // Therefore !isDelete => !isNewestPoint, which can be restated as @@ -696,7 +1119,7 @@ func isGarbage( if !isDelete && next == nil { panic("huh") } - return isDelete || next.key.Timestamp.LessEq(threshold), nil + return isDelete || next.key.Timestamp.LessEq(threshold) } // processLocalKeyRange scans the local range key entries, consisting of diff --git a/pkg/kv/kvserver/gc/gc_int_test.go b/pkg/kv/kvserver/gc/gc_int_test.go index faf1d93ad093..28cab4fe9579 100644 --- a/pkg/kv/kvserver/gc/gc_int_test.go +++ b/pkg/kv/kvserver/gc/gc_int_test.go @@ -55,182 +55,218 @@ func TestEndToEndGC(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ctx := context.Background() - - manualClock := hlc.NewHybridManualClock() - tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - SmallEngineBlocks: smallEngineBlocks, - }, - Server: &server.TestingKnobs{ - WallClock: manualClock, - }, - }, - DisableDefaultTestTenant: true, + for _, d := range []struct { + // Using range tombstones to remove data will promote full range deletions + // as a fast path GC operation. + rangeTombstones bool + // Clear range enables usage of clear range operation to remove multiple + // point keys by GC. + clearRange bool + }{ + { + // With range tombstones, fast path will always take precedence so we + // don't care about clearRange being enabled or not. + rangeTombstones: true, }, - }) - tc.Start(t) - defer tc.Stopper().Stop(context.Background()) - require.NoError(t, tc.WaitForFullReplication()) - - sqlDb := tc.ServerConn(0) - kvDb := tc.Server(0).DB() - statusServer := tc.Server(0).StatusServer().(serverpb.StatusServer) - - execOrFatal := func(t *testing.T, db *gosql.DB, stmt string, args ...interface{}) { - t.Helper() - _, err := db.Exec(stmt, args...) - require.NoError(t, err, "failed to execute %s", stmt) - } + { + rangeTombstones: false, + clearRange: true, + }, + { + rangeTombstones: false, + clearRange: false, + }, + } { + t.Run(fmt.Sprintf("rangeTombstones=%t/clearRange=%t", d.rangeTombstones, d.clearRange), + func(t *testing.T) { + ctx := context.Background() - getTableRangeIDs := func(t *testing.T, db *gosql.DB) ids { - t.Helper() - rows, err := db.Query("with r as (show ranges from table kv) select range_id from r order by start_key") - require.NoError(t, err, "failed to query ranges") - var rangeIDs []int64 - for rows.Next() { - var rangeID int64 - require.NoError(t, rows.Scan(&rangeID), "failed to read row with range id") - rangeIDs = append(rangeIDs, rangeID) - } - return rangeIDs - } + manualClock := hlc.NewHybridManualClock() + tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + SmallEngineBlocks: smallEngineBlocks, + }, + Server: &server.TestingKnobs{ + WallClock: manualClock, + }, + }, + DisableDefaultTestTenant: true, + }, + }) + tc.Start(t) + defer tc.Stopper().Stop(context.Background()) + require.NoError(t, tc.WaitForFullReplication()) - readSomeKeys := func(t *testing.T, db *gosql.DB) []int64 { - t.Helper() - var ids []int64 - rows, err := db.Query("select k from kv limit 5") - require.NoError(t, err, "failed to query kv data") - for rows.Next() { - var id int64 - require.NoError(t, rows.Scan(&id), "failed to scan value") - ids = append(ids, id) - } - return ids - } + sqlDb := tc.ServerConn(0) + kvDb := tc.Server(0).DB() + statusServer := tc.Server(0).StatusServer().(serverpb.StatusServer) - getRangeInfo := func(t *testing.T, rangeID int64, db *gosql.DB) (startKey, endKey []byte) { - t.Helper() - row := db.QueryRow("select start_key, end_key from crdb_internal.ranges_no_leases where range_id=$1", - rangeID) - require.NoError(t, row.Err(), "failed to query range info") - require.NoError(t, row.Scan(&startKey, &endKey), "failed to scan range info") - return startKey, endKey - } + execOrFatal := func(t *testing.T, db *gosql.DB, stmt string, args ...interface{}) { + t.Helper() + _, err := db.Exec(stmt, args...) + require.NoError(t, err, "failed to execute %s", stmt) + } - deleteRangeDataWithRangeTombstone := func(t *testing.T, rangeIDs ids, kvDb *kv.DB, db *gosql.DB) { - t.Helper() - for _, id := range rangeIDs { - start, end := getRangeInfo(t, id, db) - require.NoError(t, kvDb.DelRangeUsingTombstone(context.Background(), start, end), - "failed to delete range with tombstone") - } - } + getTableRangeIDs := func(t *testing.T, db *gosql.DB) ids { + t.Helper() + rows, err := db.Query("with r as (show ranges from table kv) select range_id from r order by start_key") + require.NoError(t, err, "failed to query ranges") + var rangeIDs []int64 + for rows.Next() { + var rangeID int64 + require.NoError(t, rows.Scan(&rangeID), "failed to read row with range id") + rangeIDs = append(rangeIDs, rangeID) + } + return rangeIDs + } - getRangeStats := func(t *testing.T, rangeID int64) enginepb.MVCCStats { - t.Helper() - rr := &serverpb.RangesRequest{ - NodeId: "1", - RangeIDs: []roachpb.RangeID{roachpb.RangeID(rangeID)}, - } - infos, err := statusServer.Ranges(ctx, rr) - require.NoError(t, err, "failed to query range info") - return *infos.Ranges[0].State.Stats - } + readSomeKeys := func(t *testing.T, db *gosql.DB) []int64 { + t.Helper() + var ids []int64 + rows, err := db.Query("select k from kv limit 5") + require.NoError(t, err, "failed to query kv data") + for rows.Next() { + var id int64 + require.NoError(t, rows.Scan(&id), "failed to scan value") + ids = append(ids, id) + } + return ids + } - findNonEmptyRanges := func(t *testing.T, rangeIDs ids) (nonEmptyRangeIDs ids) { - t.Helper() - for _, id := range rangeIDs { - stats := getRangeStats(t, id) - t.Logf("range %d stats: %s", id, &stats) - // Test can't give meaningful results if stats contain estimates. - // Test also doesn't perform any operations that result in estimated stats - // being created, so it is a failure in the environment if that happens. - require.Zerof(t, stats.ContainsEstimates, "we must not have estimates") - if stats.RangeKeyCount > 0 || stats.KeyCount > 0 { - nonEmptyRangeIDs = append(nonEmptyRangeIDs, id) - } - } - return nonEmptyRangeIDs - } + getRangeInfo := func(t *testing.T, rangeID int64, db *gosql.DB) (startKey, endKey []byte) { + t.Helper() + row := db.QueryRow("select start_key, end_key from crdb_internal.ranges_no_leases where range_id=$1", + rangeID) + require.NoError(t, row.Err(), "failed to query range info") + require.NoError(t, row.Scan(&startKey, &endKey), "failed to scan range info") + return startKey, endKey + } - rng, _ := randutil.NewTestRand() + deleteRangeDataWithRangeTombstone := func(t *testing.T, rangeIDs ids, kvDb *kv.DB, db *gosql.DB) { + t.Helper() + for _, id := range rangeIDs { + start, end := getRangeInfo(t, id, db) + require.NoError(t, kvDb.DelRangeUsingTombstone(ctx, start, end), + "failed to delete range with tombstone") + } + } - // Set closed timestamp duration, this is needed to avoid waiting for default - // 2 min interval for protected timestamp to get bumped and letting GC collect - // old values. - execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.protectedts.poll_interval = '5s'`) + getRangeStats := func(t *testing.T, rangeID int64) enginepb.MVCCStats { + t.Helper() + rr := &serverpb.RangesRequest{ + NodeId: "1", + RangeIDs: []roachpb.RangeID{roachpb.RangeID(rangeID)}, + } + infos, err := statusServer.Ranges(ctx, rr) + require.NoError(t, err, "failed to query range info") + return *infos.Ranges[0].State.Stats + } - execOrFatal(t, sqlDb, `create table kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`) + findNonEmptyRanges := func(t *testing.T, rangeIDs ids) (nonEmptyRangeIDs ids) { + t.Helper() + for _, id := range rangeIDs { + stats := getRangeStats(t, id) + t.Logf("range %d stats: %s", id, &stats) + // Test can't give meaningful results if stats contain estimates. + // Test also doesn't perform any operations that result in estimated stats + // being created, so it is a failure in the environment if that happens. + require.Zerof(t, stats.ContainsEstimates, "we must not have estimates") + if stats.RangeKeyCount > 0 || stats.KeyCount > 0 { + nonEmptyRangeIDs = append(nonEmptyRangeIDs, id) + } + } + return nonEmptyRangeIDs + } - for i := 0; i < 1000; i++ { - execOrFatal(t, sqlDb, "upsert into kv values ($1, $2)", rng.Int63(), "hello") - } + rng, _ := randutil.NewTestRand() + + // Set closed timestamp duration, this is needed to avoid waiting for default + // 2 min interval for protected timestamp to get bumped and letting GC collect + // old values. + execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.protectedts.poll_interval = '5s'`) + + if d.clearRange { + execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 5`) + } else { + execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 0`) + } + + execOrFatal(t, sqlDb, `create table kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`) + + for i := 0; i < 1000; i++ { + execOrFatal(t, sqlDb, "upsert into kv values ($1, $2)", rng.Int63(), "hello") + } + + require.NotEmptyf(t, readSomeKeys(t, sqlDb), "found no keys in table") + + // Since ranges query and checking range stats are non atomic there could be + // a range split/merge operation caught in between. That could produce empty + // or incomplete results. Moreover, range info produced by ranges doesn't + // provide start/end keys for the range in binary form, so it is hard to make + // consistency check. We rely on retrying several times for simplicity. + const tableRangesRetry = 3 + var tableRangeIDs, nonEmptyRangeIDs ids + for i := 0; i < tableRangesRetry; i++ { + tableRangeIDs = getTableRangeIDs(t, sqlDb) + if len(tableRangeIDs) == 0 { + continue + } + nonEmptyRangeIDs = findNonEmptyRanges(t, tableRangeIDs) + if len(nonEmptyRangeIDs) > 0 { + break + } + } + require.NotEmpty(t, tableRangeIDs, "failed to query ranges belonging to table") + require.NotEmptyf(t, nonEmptyRangeIDs, "all table ranges are empty according to MVCCStats") + + if d.rangeTombstones { + deleteRangeDataWithRangeTombstone(t, tableRangeIDs, kvDb, sqlDb) + } else { + execOrFatal(t, sqlDb, "delete from kv where k is not null") + } + + require.Empty(t, readSomeKeys(t, sqlDb), "table still contains data after range deletion") + + // Push clock forward to make all data eligible for GC. Mind that this is not + // enough just to push the clock, we need to wait for protected timestamp to + // be pushed by periodic task. + manualClock.Increment((time.Hour * 50).Nanoseconds()) + + // Keep pushing replicas through the queue and checking that ranges were + // cleared up. We do both operations in the retry loop because we are dealing + // with two async processes: 1 - protected timestamp update, 2 - queue + // processing as we could only enqueue, but not force GC op. + enqueueSucceeded := false + testutils.SucceedsSoon(t, func() error { + tableRangeIDs := getTableRangeIDs(t, sqlDb) + t.Logf("pushing kv table ranges through mvcc gc queue: %s", tableRangeIDs) + + for _, id := range tableRangeIDs { + _, err := sqlDb.Exec(`SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, id) + if err != nil { + t.Logf("failed to enqueue range to mvcc gc queue: %s", err) + } + enqueueSucceeded = enqueueSucceeded || err == nil + } + + // Enqueue operations could fail if ranges change underneath, test will + // report different error if we didn't enqueue replicas at least once. + // This is giving us a better visibility if failure is because of GC queue + // misbehaving and not actual GC behaviour test is checking. + if !enqueueSucceeded { + return errors.New("failed to enqueue replicas to GC queue") + } - require.NotEmptyf(t, readSomeKeys(t, sqlDb), "found no keys in table") - - // Since ranges query and checking range stats are non atomic there could be - // a range split/merge operation caught in between. That could produce empty - // or incomplete results. Moreover, range info produced by ranges doesn't - // provide start/end keys for the range in binary form, so it is hard to make - // consistency check. We rely on retrying several times for simplicity. - const tableRangesRetry = 3 - var tableRangeIDs, nonEmptyRangeIDs ids - for i := 0; i < tableRangesRetry; i++ { - tableRangeIDs = getTableRangeIDs(t, sqlDb) - if len(tableRangeIDs) == 0 { - continue - } - nonEmptyRangeIDs = findNonEmptyRanges(t, tableRangeIDs) - if len(nonEmptyRangeIDs) > 0 { - break - } + nonEmptyRangeIDs := findNonEmptyRanges(t, tableRangeIDs) + if len(nonEmptyRangeIDs) > 0 { + return errors.New("not all ranges were cleared") + } + return nil + }) + }) } - require.NotEmpty(t, tableRangeIDs, "failed to query ranges belonging to table") - require.NotEmptyf(t, nonEmptyRangeIDs, "all table ranges are empty according to MVCCStats") - - deleteRangeDataWithRangeTombstone(t, tableRangeIDs, kvDb, sqlDb) - - require.Empty(t, readSomeKeys(t, sqlDb), "table still contains data after range deletion") - - // Push clock forward to make all data eligible for GC. Mind that this is not - // enough just to push the clock, we need to wait for protected timestamp to - // be pushed by periodic task. - manualClock.Increment((time.Hour * 50).Nanoseconds()) - - // Keep pushing replicas through the queue and checking that ranges were - // cleared up. We do both operations in the retry loop because we are dealing - // with two async processes: 1 - protected timestamp update, 2 - queue - // processing as we could only enqueue, but not force GC op. - enqueueSucceeded := false - testutils.SucceedsSoon(t, func() error { - tableRangeIDs := getTableRangeIDs(t, sqlDb) - t.Logf("pushing kv table ranges through mvcc gc queue: %s", tableRangeIDs) - - for _, id := range tableRangeIDs { - _, err := sqlDb.Exec(`SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, id) - if err != nil { - t.Logf("failed to enqueue range to mvcc gc queue: %s", err) - } - enqueueSucceeded = enqueueSucceeded || err == nil - } - - // Enqueue operations could fail if ranges change underneath, test will - // report different error if we didn't enqueue replicas at least once. - // This is giving us a better visibility if failure is because of GC queue - // misbehaving and not actual GC behaviour test is checking. - if !enqueueSucceeded { - return errors.New("failed to enqueue replicas to GC queue") - } - - nonEmptyRangeIDs := findNonEmptyRanges(t, tableRangeIDs) - if len(nonEmptyRangeIDs) > 0 { - return errors.New("not all ranges were cleared") - } - return nil - }) } type ids []int64 diff --git a/pkg/kv/kvserver/gc/gc_iterator.go b/pkg/kv/kvserver/gc/gc_iterator.go index a57e1e12347e..b8e33221367c 100644 --- a/pkg/kv/kvserver/gc/gc_iterator.go +++ b/pkg/kv/kvserver/gc/gc_iterator.go @@ -11,7 +11,6 @@ package gc import ( - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" @@ -21,7 +20,7 @@ import ( // gcIterator wraps an rditer.ReplicaMVCCDataIterator which it reverse iterates for // the purpose of discovering gc-able replicated data. type gcIterator struct { - it *rditer.ReplicaMVCCDataIterator + it storage.MVCCIterator threshold hlc.Timestamp err error buf gcIteratorRingBuf @@ -35,19 +34,9 @@ type gcIterator struct { // TODO(sumeer): change gcIterator to use MVCCValueLenAndIsTombstone(). It // needs to get the value only for intents. -func makeGCIterator( - desc *roachpb.RangeDescriptor, - snap storage.Reader, - threshold hlc.Timestamp, - excludeUserKeySpan bool, -) gcIterator { +func makeGCIterator(iter storage.MVCCIterator, threshold hlc.Timestamp) gcIterator { return gcIterator{ - it: rditer.NewReplicaMVCCDataIterator(desc, snap, rditer.ReplicaDataIteratorOptions{ - Reverse: true, - IterKind: storage.MVCCKeyAndIntentsIterKind, - KeyTypes: storage.IterKeyTypePointsAndRanges, - ExcludeUserKeySpan: excludeUserKeySpan, - }), + it: iter, threshold: threshold, } } @@ -69,6 +58,12 @@ func (s *gcIteratorState) curIsNewest() bool { (s.next == nil || (s.afterNext != nil && !s.afterNext.key.IsValue())) } +// True if we are positioned on newest version when there's no intent or on +// intent. +func (s *gcIteratorState) curLastKeyVersion() bool { + return s.next == nil +} + // curIsNotValue returns true if the current MVCCKeyValue in the gcIteratorState // is not a value, i.e. does not have a timestamp. func (s *gcIteratorState) curIsNotValue() bool { @@ -188,11 +183,6 @@ func (it *gcIterator) currentRangeTS() hlc.Timestamp { return it.cachedRangeTombstoneTS } -func (it *gcIterator) close() { - it.it.Close() - it.it = nil -} - // gcIteratorRingBufSize is 3 because the gcIterator.state method at most needs // to look forward two keys ahead of the current key. const gcIteratorRingBufSize = 3 diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index c998c6754f8b..6bb97267b3e5 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -163,8 +163,14 @@ func TestGCIterator(t *testing.T) { ds.setupTest(t, eng, desc) snap := eng.NewSnapshot() defer snap.Close() - it := makeGCIterator(&desc, snap, tc.gcThreshold, false) - defer it.close() + mvccIt := snap.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + LowerBound: desc.StartKey.AsRawKey(), + UpperBound: desc.EndKey.AsRawKey(), + KeyTypes: storage.IterKeyTypePointsAndRanges, + }) + mvccIt.SeekLT(storage.MVCCKey{Key: desc.EndKey.AsRawKey()}) + defer mvccIt.Close() + it := makeGCIterator(mvccIt, tc.gcThreshold) expectations := tc.expectations for i, ex := range expectations { s, ok := it.state() diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 4647360530db..24bbab213339 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -46,47 +46,47 @@ var ( tsSecFrom: 1, tsSecTo: 100, keySuffixMin: 2, keySuffixMax: 6, valueLenMin: 1, valueLenMax: 1, - deleteFrac: 0, - keysPerValueMin: 1, keysPerValueMax: 2, + deleteFrac: 0, + versionsPerKeyMin: 1, versionsPerKeyMax: 2, intentFrac: .1, } someVersionsMidSizeRows = uniformDistSpec{ tsSecFrom: 1, tsSecTo: 100, keySuffixMin: 8, keySuffixMax: 8, valueLenMin: 8, valueLenMax: 16, - deleteFrac: .1, - keysPerValueMin: 1, keysPerValueMax: 100, + deleteFrac: .1, + versionsPerKeyMin: 1, versionsPerKeyMax: 100, intentFrac: .1, } lotsOfVersionsMidSizeRows = uniformDistSpec{ tsSecFrom: 1, tsSecTo: 100, - keySuffixMin: 8, keySuffixMax: 8, + keySuffixMin: 7, keySuffixMax: 9, valueLenMin: 8, valueLenMax: 16, - deleteFrac: .1, - keysPerValueMin: 1000, keysPerValueMax: 1000000, + deleteFrac: .1, + versionsPerKeyMin: 1000, versionsPerKeyMax: 1000000, intentFrac: .1, } // This spec is identical to someVersionsMidSizeRows except for number of // intents. someVersionsMidSizeRowsLotsOfIntents = uniformDistSpec{ tsSecFrom: 1, tsSecTo: 100, - keySuffixMin: 8, keySuffixMax: 8, + keySuffixMin: 7, keySuffixMax: 9, valueLenMin: 8, valueLenMax: 16, - deleteFrac: .1, - keysPerValueMin: 1, keysPerValueMax: 100, + deleteFrac: .1, + versionsPerKeyMin: 1, versionsPerKeyMax: 100, intentFrac: 1, } someVersionsWithSomeRangeKeys = uniformDistSpec{ tsSecFrom: 1, tsSecTo: 100, tsSecMinIntent: 70, tsSecOldIntentTo: 85, - keySuffixMin: 8, keySuffixMax: 8, + keySuffixMin: 7, keySuffixMax: 9, valueLenMin: 8, valueLenMax: 16, - deleteFrac: .1, - keysPerValueMin: 1, - keysPerValueMax: 100, - intentFrac: .1, - oldIntentFrac: .1, - rangeKeyFrac: .1, + deleteFrac: .1, + versionsPerKeyMin: 1, + versionsPerKeyMax: 100, + intentFrac: .1, + oldIntentFrac: .1, + rangeKeyFrac: .1, } // smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke @@ -297,6 +297,7 @@ func TestNewVsInvariants(t *testing.T) { gcThreshold, RunOptions{ IntentAgeThreshold: intentAgeThreshold, TxnCleanupThreshold: txnCleanupThreshold, + ClearRangeMinKeys: 100, }, ttl, &gcer, gcer.resolveIntents, @@ -316,6 +317,15 @@ func TestNewVsInvariants(t *testing.T) { _, err := storage.MVCCResolveWriteIntent(ctx, eng, &stats, l) require.NoError(t, err, "failed to resolve intent") } + for _, cr := range gcer.clearRanges() { + require.False(t, cr.StartKeyTimestamp.IsEmpty(), "unexpected full range delete") + if cr.EndKey == nil { + cr.EndKey = cr.StartKey.Next() + } + require.NoError(t, + storage.MVCCGarbageCollectPointsWithClearRange(ctx, eng, &stats, cr.StartKey, cr.EndKey, + cr.StartKeyTimestamp, gcThreshold)) + } for _, batch := range gcer.rangeKeyBatches() { rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), batch) @@ -323,6 +333,11 @@ func TestNewVsInvariants(t *testing.T) { storage.MVCCGarbageCollectRangeKeys(ctx, eng, &stats, rangeKeys)) } + // For the sake of assertion we need to reset this counter as it signals + // counter for specific feature rather than processed data. Data and number + // of cleared keys and versions should be the same regardless of operations + // used to clear it. + gcInfoNew.ClearRangeSpanOperations = 0 assertLiveData(t, eng, beforeGC, *desc, tc.now, gcThreshold, intentThreshold, ttl, gcInfoNew) }) @@ -738,12 +753,13 @@ func mergeRanges(fragments [][]storage.MVCCRangeKeyValue) []storage.MVCCRangeKey } type fakeGCer struct { - gcKeys map[string]roachpb.GCRequest_GCKey + gcKeys map[string]roachpb.GCRequest_GCKey + gcPointsBatches [][]roachpb.GCRequest_GCKey // fake GCer stores range key batches as it since we need to be able to // feed them into MVCCGarbageCollectRangeKeys and ranges argument should be // non-overlapping. gcRangeKeyBatches [][]roachpb.GCRequest_GCRangeKey - gcClearRangeKeys []roachpb.GCRequest_GCClearRangeKey + gcClearRanges []roachpb.GCRequest_GCClearRange threshold Threshold intents []roachpb.Intent batches [][]roachpb.Intent @@ -767,14 +783,23 @@ func (f *fakeGCer) GC( ctx context.Context, keys []roachpb.GCRequest_GCKey, rangeKeys []roachpb.GCRequest_GCRangeKey, - clearRangeKey *roachpb.GCRequest_GCClearRangeKey, + clearRange *roachpb.GCRequest_GCClearRange, ) error { for _, k := range keys { f.gcKeys[k.Key.String()] = k } - f.gcRangeKeyBatches = append(f.gcRangeKeyBatches, rangeKeys) - if clearRangeKey != nil { - f.gcClearRangeKeys = append(f.gcClearRangeKeys, *clearRangeKey) + if keys != nil { + f.gcPointsBatches = append(f.gcPointsBatches, keys) + } + if rangeKeys != nil { + f.gcRangeKeyBatches = append(f.gcRangeKeyBatches, rangeKeys) + } + if clearRange != nil { + f.gcClearRanges = append(f.gcClearRanges, roachpb.GCRequest_GCClearRange{ + StartKey: clearRange.StartKey.Clone(), + StartKeyTimestamp: clearRange.StartKeyTimestamp, + EndKey: clearRange.EndKey.Clone(), + }) } return nil } @@ -790,6 +815,10 @@ func (f *fakeGCer) resolveIntents(_ context.Context, intents []roachpb.Intent) e return nil } +// normalize will converge GC request history between old and new +// implementations and drop info that is not produced by old GC. +// It will however preserve info like clear range which covers functionality +// not relevant for old gc as it shouldn't be compared between such invocations. func (f *fakeGCer) normalize() { sortIntents := func(i, j int) bool { return intentLess(&f.intents[i], &f.intents[j]) @@ -802,6 +831,7 @@ func (f *fakeGCer) normalize() { return f.txnIntents[i].txn.ID.String() < f.txnIntents[j].txn.ID.String() }) f.batches = nil + f.gcPointsBatches = nil } func (f *fakeGCer) pointKeys() []roachpb.GCRequest_GCKey { @@ -824,6 +854,10 @@ func (f *fakeGCer) rangeKeys() []roachpb.GCRequest_GCRangeKey { return reqs } +func (f *fakeGCer) clearRanges() []roachpb.GCRequest_GCClearRange { + return f.gcClearRanges +} + func intentLess(a, b *roachpb.Intent) bool { cmp := a.Key.Compare(b.Key) switch { @@ -848,25 +882,36 @@ func makeCollectableGCRangesFromGCRequests( ) []storage.CollectableGCRangeKey { collectableKeys := make([]storage.CollectableGCRangeKey, len(rangeKeys)) for i, rk := range rangeKeys { - leftPeekBound := rk.StartKey.Prevish(roachpb.PrevishKeyLength) - if len(rangeStart) > 0 && leftPeekBound.Compare(rangeStart) <= 0 { - leftPeekBound = rangeStart - } - rightPeekBound := rk.EndKey.Next() - if len(rangeEnd) > 0 && rightPeekBound.Compare(rangeEnd) >= 0 { - rightPeekBound = rangeEnd - } + peekBounds := expandRangeSpan(roachpb.Span{ + Key: rk.StartKey, + EndKey: rk.EndKey, + }, roachpb.Span{ + Key: rangeStart, + EndKey: rangeEnd, + }) collectableKeys[i] = storage.CollectableGCRangeKey{ MVCCRangeKey: storage.MVCCRangeKey{ StartKey: rk.StartKey, EndKey: rk.EndKey, Timestamp: rk.Timestamp, }, - LatchSpan: roachpb.Span{ - Key: leftPeekBound, - EndKey: rightPeekBound, - }, + LatchSpan: peekBounds, } } return collectableKeys } + +func expandRangeSpan(rangeKey, limits roachpb.Span) roachpb.Span { + leftPeekBound := rangeKey.Key.Prevish(roachpb.PrevishKeyLength) + if len(limits.Key) > 0 && leftPeekBound.Compare(limits.Key) <= 0 { + leftPeekBound = limits.Key + } + rightPeekBound := rangeKey.EndKey.Next() + if len(limits.EndKey) > 0 && rightPeekBound.Compare(limits.EndKey) >= 0 { + rightPeekBound = limits.EndKey + } + return roachpb.Span{ + Key: leftPeekBound, + EndKey: rightPeekBound, + } +} diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 14005a118064..f8aeebdfed50 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -27,6 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -59,7 +61,7 @@ func (c *collectingGCer) GC( _ context.Context, keys []roachpb.GCRequest_GCKey, _ []roachpb.GCRequest_GCRangeKey, - _ *roachpb.GCRequest_GCClearRangeKey, + _ *roachpb.GCRequest_GCClearRange, ) error { c.keys = append(c.keys, keys) return nil @@ -146,15 +148,15 @@ func TestIntentAgeThresholdSetting(t *testing.T) { nowTs := hlc.Timestamp{ WallTime: now.Nanoseconds(), } - fakeGCer := makeFakeGCer() + gcer := makeFakeGCer() // Test GC desired behavior. info, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{ IntentAgeThreshold: intentLongThreshold, TxnCleanupThreshold: txnCleanupThreshold, - }, gcTTL, &fakeGCer, fakeGCer.resolveIntents, - fakeGCer.resolveIntentsAsync) + }, gcTTL, &gcer, gcer.resolveIntents, + gcer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") assert.Zero(t, info.IntentsConsidered, "Expected no intents considered by GC with default threshold") @@ -163,8 +165,8 @@ func TestIntentAgeThresholdSetting(t *testing.T) { RunOptions{ IntentAgeThreshold: intentShortThreshold, TxnCleanupThreshold: txnCleanupThreshold, - }, gcTTL, &fakeGCer, fakeGCer.resolveIntents, - fakeGCer.resolveIntentsAsync) + }, gcTTL, &gcer, gcer.resolveIntents, + gcer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") assert.Equal(t, 1, info.IntentsConsidered, "Expected 1 intents considered by GC with short threshold") @@ -223,7 +225,7 @@ func TestIntentCleanupBatching(t *testing.T) { baseGCer.normalize() var batchSize int64 = 7 - fakeGCer := makeFakeGCer() + gcer := makeFakeGCer() info, err := Run(ctx, &desc, snap, nowTs, nowTs, RunOptions{ IntentAgeThreshold: intentAgeThreshold, @@ -231,18 +233,18 @@ func TestIntentCleanupBatching(t *testing.T) { TxnCleanupThreshold: txnCleanupThreshold, }, gcTTL, - &fakeGCer, fakeGCer.resolveIntents, fakeGCer.resolveIntentsAsync) + &gcer, gcer.resolveIntents, gcer.resolveIntentsAsync) require.NoError(t, err, "GC Run shouldn't fail") maxIntents := 0 - for _, batch := range fakeGCer.batches { + for _, batch := range gcer.batches { if intents := len(batch); intents > maxIntents { maxIntents = intents } } require.Equal(t, int64(maxIntents), batchSize, "Batch size") require.Equal(t, 15, info.ResolveTotal) - fakeGCer.normalize() - require.EqualValues(t, baseGCer, fakeGCer, "GC result with batching") + gcer.normalize() + require.EqualValues(t, baseGCer, gcer, "GC result with batching") } type testResolver [][]roachpb.Intent @@ -698,6 +700,14 @@ var avoidMergingDifferentTs = ` 1 | ` +type testRunData struct { + data string + deleteRangeThreshold int64 + keyBytesThreshold int64 + disableClearRange bool + maxPendingKeySize int64 +} + func TestGC(t *testing.T) { defer leaktest.AfterTest(t)() for _, d := range []struct { @@ -723,12 +733,322 @@ func TestGC(t *testing.T) { {name: "avoid_merging_different_ts", data: avoidMergingDifferentTs}, } { t.Run(d.name, func(t *testing.T) { - runTest(t, d.data) + testutils.RunTrueAndFalse(t, "clearRange", func(t *testing.T, clearRange bool) { + runTest(t, testRunData{ + data: d.data, + deleteRangeThreshold: 2, + disableClearRange: !clearRange, + }, nil) + }) + }) + } +} + +type gCR roachpb.GCRequest_GCClearRange + +// Format implements the fmt.Formatter interface. +func (k gCR) Format(f fmt.State, r rune) { + k.StartKey.Format(f, r) + fmt.Fprintf(f, "@%d-", k.StartKeyTimestamp.WallTime/1e9) + k.EndKey.Format(f, r) +} + +type gCRS []roachpb.GCRequest_GCClearRange + +func (k gCRS) toTestData() (spans []gCR) { + if len(k) == 0 { + return nil + } + spans = make([]gCR, len(k)) + for i, c := range k { + spans[i] = gCR{ + StartKey: c.StartKey, + StartKeyTimestamp: c.StartKeyTimestamp, + EndKey: c.EndKey, + } + } + return spans +} + +type clearPointsKey roachpb.GCRequest_GCKey + +// Format implements the fmt.Formatter interface. +func (k clearPointsKey) Format(f fmt.State, c rune) { + storage.MVCCKey{ + Key: k.Key, + Timestamp: k.Timestamp, + }.Format(f, c) +} + +type gcPointsBatches [][]roachpb.GCRequest_GCKey + +// toTestData converts batches to test data structs that are easier to create +// and assert. This method also ignores nil convention that allows tests to +// use nil to ignore asserts when not interesting and use empty slice when we +// want to ensure that there are no requests. +func (b gcPointsBatches) toTestData() (keys [][]clearPointsKey) { + keys = make([][]clearPointsKey, len(b)) + for i, b := range b { + keys[i] = make([]clearPointsKey, len(b)) + for j, k := range b { + keys[i][j] = clearPointsKey{ + Key: k.Key, + Timestamp: k.Timestamp, + } + } + } + return keys +} + +// For testing clear range we perform normal checks, but also explicitly verify +// generated clear range spans as normal point key deletions would do the same +// job but less efficiently. +type clearRangeTestData struct { + name string + data testRunData + clearSpans []gCR + clearPoints [][]clearPointsKey +} + +// TestGCUseClearRange tests garbage collection correctness from scanning engine +// data to applying MVCC operations based on GC run results. +// Test doesn't cover GC all internals extensively (batching limits, selecting +// between point and range deletions etc.), but it provides a test that executes +// collection as a whole ensuring GC components work well together. +// There are separate tests for GC iterator, batcher, intent batcher that verify +// corner cases in each component. +func TestGCUseClearRange(t *testing.T) { + defer leaktest.AfterTest(t)() + + first := keys.SystemSQLCodec.TablePrefix(42) + // mkKey creates a key in a table from a string value. If key ends with a + + // then Next() key is returned. + mkKey := func(key string) roachpb.Key { + if l := len(key); l > 1 && key[l-1] == '+' { + return append(first[:len(first):len(first)], key[:l-1]...).Next() + } + return append(first[:len(first):len(first)], key...) + } + last := first.PrefixEnd() + + mkKeyPair := func(start string, startTs int64, end string) gCR { + return gCR{ + StartKey: mkKey(start), + StartKeyTimestamp: hlc.Timestamp{WallTime: startTs * time.Second.Nanoseconds()}, + EndKey: mkKey(end), + } + } + mkKeyPairToLast := func(start string, startTs int64) gCR { + return gCR{ + StartKey: mkKey(start), + StartKeyTimestamp: hlc.Timestamp{WallTime: startTs * time.Second.Nanoseconds()}, + EndKey: last, + } + } + mkRawKeyPair := func(start, end roachpb.Key) gCR { + return gCR{ + StartKey: start, + EndKey: end, + } + } + _, _, _ = mkKeyPair, mkKeyPairToLast, mkRawKeyPair + clearRangeTestDefaults := func(d testRunData) testRunData { + if d.deleteRangeThreshold == 0 { + d.deleteRangeThreshold = 1 + } + if d.keyBytesThreshold == 0 { + d.keyBytesThreshold = 1 + } + return d + } + + for _, d := range []clearRangeTestData{ + { + name: "clear multiple keys", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 6 | +>5 | + 4 | C . F + 3 | A ee + 2 | dd + 1 | +`, + }, + clearSpans: []gCR{ + mkKeyPair("bb", 2, "ccc+"), + }, + }, + { + name: "clear range limits", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 6 | +>5 | X + 4 | C . y + 3 | A ee + 2 | dd z + 1 | +`, + deleteRangeThreshold: 3, + }, + clearSpans: []gCR{ + mkKeyPair("bb", 2, "d"), + }, + }, + { + name: "clear multiple keys till end range", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 6 | +>5 | . + 4 | C . f + 3 | A ee + 2 | dd + 1 | +`, + }, + clearSpans: []gCR{ + mkKeyPair("bb", 2, "d+"), + }, + }, + { + name: "clear multiple keys from start range", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 6 | +>5 | + 4 | . . . F + 3 | a ee + 2 | dd + 1 | +`, + }, + clearSpans: []gCR{ + mkKeyPair("a", 4, "ccc+"), + }, + }, + { + name: "clear range restarts on live data", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 7 | + 6 | X +>5 | . + 4 | . e + 3 | A . + 2 | dd . + 1 | ff y z +`, + deleteRangeThreshold: 2, + }, + clearSpans: []gCR{ + mkKeyPair("d", 3, "e+"), + mkKeyPair("bb", 4, "ccc+"), + }, + }, + { + name: "clear range restart on intents", + data: testRunData{ + data: ` + | a bb ccc d e +---+--------------- + 7 | + 6 | !X +>5 | . + 4 | . e . + 3 | A . + 2 | dd x + 1 | ff y +`, + }, + clearSpans: []gCR{ + mkKeyPair("d", 4, "e+"), + mkKeyPair("bb", 4, "ccc+"), + }, + }, + { + name: "clear range when covered by range tombstone", + data: testRunData{ + data: ` + | a bb ccc d e f +---+------------------ + 6 | Z +>5 | .---------- + 4 | c f . . + 3 | A h i + 2 | b dd ggg + 1 | +`, + }, + clearSpans: []gCR{ + mkKeyPair("a", 2, "e+"), + }, + }, + { + name: "clear range with live data above range tombstone", + data: testRunData{ + data: ` + | a bb ccc d e f +---+------------------ + 6 | E +>5 | .----------- + 4 | c f . . + 3 | A h i + 2 | b dd ggg + 1 | +`, + }, + clearSpans: []gCR{ + mkKeyPair("ccc", 4, "e+"), + mkKeyPair("a", 2, "bb+"), + }, + }, + { + name: "clear full range optimization", + data: testRunData{ + data: ` + | a bb ccc d e f +---+------------------ + 6 | +>5 | .-------------- + 4 | c f . . + 3 | a h i + 2 | b dd ggg + 1 | +`, + }, + clearSpans: []gCR{ + {StartKey: first, EndKey: last}, + }, + clearPoints: [][]clearPointsKey{}, + }, + } { + t.Run(d.name, func(t *testing.T) { + runTest(t, clearRangeTestDefaults(d.data), func(t *testing.T, gcer *fakeGCer) { + require.EqualValues(t, d.clearSpans, gCRS(gcer.gcClearRanges).toTestData(), + "clear range requests") + if d.clearPoints != nil { + require.EqualValues(t, d.clearPoints, gcPointsBatches(gcer.gcPointsBatches).toTestData()) + } + }) }) } } -func runTest(t *testing.T, data string) { +type gcVerifier func(t *testing.T, gcer *fakeGCer) + +func runTest(t *testing.T, data testRunData, verify gcVerifier) { ctx := context.Background() tablePrefix := keys.SystemSQLCodec.TablePrefix(42) desc := roachpb.RangeDescriptor{ @@ -739,25 +1059,50 @@ func runTest(t *testing.T, data string) { eng := storage.NewDefaultInMemForTesting() defer eng.Close() - dataItems, gcTS, now := readTableData(t, desc.StartKey.AsRawKey(), data) + dataItems, gcTS, now := readTableData(t, desc.StartKey.AsRawKey(), data.data) ds := dataItems.fullDistribution() stats := ds.setupTest(t, eng, desc) snap := eng.NewSnapshot() defer snap.Close() + if data.disableClearRange { + data.deleteRangeThreshold = 0 + } + if data.maxPendingKeySize == 0 { + data.maxPendingKeySize = math.MaxInt64 + } + gcer := makeFakeGCer() _, err := Run(ctx, &desc, snap, now, gcTS, RunOptions{ - IntentAgeThreshold: time.Nanosecond * time.Duration(now.WallTime), - TxnCleanupThreshold: txnCleanupThreshold, + IntentAgeThreshold: time.Nanosecond * time.Duration(now.WallTime), + TxnCleanupThreshold: txnCleanupThreshold, + MaxKeyVersionChunkBytes: data.keyBytesThreshold, + ClearRangeMinKeys: data.deleteRangeThreshold, + MaxPendingKeysSize: data.maxPendingKeySize, }, time.Second, &gcer, gcer.resolveIntents, gcer.resolveIntentsAsync) + + if verify != nil { + verify(t, &gcer) + } + require.NoError(t, err) require.Empty(t, gcer.intents, "expecting no intents") require.NoError(t, storage.MVCCGarbageCollect(ctx, eng, &stats, gcer.pointKeys(), gcTS)) + for _, r := range gcer.clearRanges() { + if r.StartKeyTimestamp.IsEmpty() { + require.NoError(t, + storage.MVCCGarbageCollectWholeRange(ctx, eng, &stats, r.StartKey, r.EndKey, gcTS, stats)) + } + require.NoError(t, + storage.MVCCGarbageCollectPointsWithClearRange(ctx, eng, &stats, r.StartKey, r.EndKey, + r.StartKeyTimestamp, gcTS)) + } + for _, batch := range gcer.rangeKeyBatches() { rangeKeys := makeCollectableGCRangesFromGCRequests(desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), batch) @@ -822,7 +1167,12 @@ func requireEqualReaders( break } - require.Equal(t, okExp, okAct, "iterators have different number of elements") + if okExp && !okAct { + t.Errorf("expected data not found in actual: %s", itExp.UnsafeKey().String()) + } + if !okExp && okAct { + t.Errorf("unexpected data found in actual: %s", itActual.UnsafeKey().String()) + } require.True(t, itExp.UnsafeKey().Equal(itActual.UnsafeKey()), "expected key not equal to actual (expected %s, found %s)", itExp.UnsafeKey(), itActual.UnsafeKey()) @@ -1458,3 +1808,503 @@ func TestRangeKeyBatching(t *testing.T) { }) } } + +// Union of GC args. +type gcReq struct { + // Note that we just reuse the type, but semantic is different because EndKey + // is not used, we only use that to have consistent request formatting in + // test results. + gcKeys []gCR + gcClearRangeKey gCR +} + +func (o gcReq) Format(f fmt.State, r rune) { + if len(o.gcKeys) > 0 { + fmt.Fprintf(f, "%s", o.gcKeys) + } else { + o.gcClearRangeKey.Format(f, r) + } +} + +type capturingGCer struct { + t *testing.T + ops []gcReq +} + +func (c *capturingGCer) GC( + _ context.Context, + k []roachpb.GCRequest_GCKey, + _ []roachpb.GCRequest_GCRangeKey, + cr *roachpb.GCRequest_GCClearRange, +) error { + if len(k) > 0 { + kk := make([]gCR, len(k)) + for i, k := range k { + kk[i] = gCR{StartKey: k.Key.Clone(), StartKeyTimestamp: k.Timestamp} + } + c.ops = append(c.ops, gcReq{gcKeys: kk}) + return nil + } + if cr != nil { + c.ops = append(c.ops, gcReq{gcClearRangeKey: gCR{ + StartKey: cr.StartKey.Clone(), + StartKeyTimestamp: cr.StartKeyTimestamp, + EndKey: cr.EndKey.Clone(), + }}) + return nil + } + c.t.Fatal("unexpected or empty GC request") + return nil +} + +type gcData struct { + kv mvccKeyValue + garbage bool +} + +func TestGcKeyBatcher(t *testing.T) { + ctx := context.Background() + + // Size is a GC size including 1 extra byte and MVCCVersionTimestampSize + keyOfSize := func(seq int, size int) roachpb.Key { + t.Helper() + var k roachpb.Key + k = append(k, keys.SystemSQLCodec.IndexPrefix(42, 1)...) + kt := encoding.EncodeStringAscending(k, fmt.Sprintf("%06d", seq)) + baseKeySize := storage.MVCCKey{Key: kt, Timestamp: hlc.Timestamp{WallTime: 1}}.EncodedSize() + if padding := size - baseKeySize; padding < 0 { + t.Fatalf("invalid test data: test key size is too small. must be >= %d", baseKeySize) + } else if padding > 0 { + k = encoding.EncodeStringAscending(k, fmt.Sprintf("%06d-%s", seq, strings.Repeat("a", padding-1))) + } else { + k = kt + } + require.Equal(t, size, + storage.MVCCKey{Key: k, Timestamp: hlc.Timestamp{WallTime: 1}}.EncodedSize(), + "test infra bug: generated key size mismatched requested") + return k + } + + hist := func(key roachpb.Key, versions, garbage int) []gcData { + require.LessOrEqual(t, garbage, versions, "invalid test data: can not have more garbage than versions") + var res []gcData + for i := 0; i < versions; i++ { + res = append(res, gcData{ + kv: mvccKeyValue{ + key: storage.MVCCKey{ + Key: key, + Timestamp: hlc.Timestamp{ + WallTime: int64(i+1) * time.Second.Nanoseconds(), + }, + }, + mvccValueLen: 10, + }, + garbage: i < garbage, + }) + } + return res + } + + point := func(key roachpb.Key, ts int) gCR { + return gCR{ + StartKey: key, + StartKeyTimestamp: hlc.Timestamp{WallTime: int64(ts) * time.Second.Nanoseconds()}, + } + } + + points := func(keys ...gCR) gcReq { + return gcReq{gcKeys: keys} + } + + clearRange := func(key roachpb.Key, ts int, endKey roachpb.Key) gcReq { + return gcReq{ + gcClearRangeKey: gCR{ + StartKey: key, + StartKeyTimestamp: hlc.Timestamp{WallTime: int64(ts) * time.Second.Nanoseconds()}, + EndKey: endKey, + }, + } + } + + const minKeySize = 24 + const largeKeySize = 100 + + for _, d := range []struct { + name string + data []gcData + reqs []gcReq + batchMax int64 + memMax int + rangeMin int + }{ + { + // Test verifies that min key limit is not reached with 4 versions of the + // first key, but reached on the second and third. We have two consecutive + // keys specifically to avoid single key range optimizations. + name: "minimum range limit", + data: keySeq( + hist(keyOfSize(4, minKeySize), 6 /* versions */, 5 /* garbage */), + hist(keyOfSize(3, minKeySize), 1 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 9 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(4, minKeySize), 5)), + clearRange(keyOfSize(2, minKeySize), 9, keyOfSize(4, minKeySize)), + points(point(keyOfSize(1, minKeySize), 1)), + }, + rangeMin: 6, + batchMax: minKeySize * 5, + }, + { + // Test verifies that a batch could be split correctly by the start of + // range key. We have two consecutive keys specifically to avoid single + // key range optimizations. + name: "clear range mid batch", + data: keySeq( + hist(keyOfSize(4, minKeySize), 5 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 1 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 9 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(4, minKeySize), 2)), + clearRange(keyOfSize(2, minKeySize), 9, keyOfSize(4, minKeySize)), + points(point(keyOfSize(1, minKeySize), 1)), + }, + rangeMin: 6, + batchMax: minKeySize * 4, + }, + { + // Test verifies that a batch could be split correctly by the start of + // range key. Batch spans multiple keys with non gc data. + name: "clear range mid batch 2", + data: keySeq( + hist(keyOfSize(5, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(4, minKeySize), 3 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 1 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 8 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(5, minKeySize), 1), + point(keyOfSize(4, minKeySize), 2), + ), + clearRange(keyOfSize(2, minKeySize), 8, keyOfSize(4, minKeySize)), + points(point(keyOfSize(1, minKeySize), 1)), + }, + rangeMin: 6, + batchMax: minKeySize * 4, + }, + { + name: "clear range at batch boundary", + data: keySeq( + hist(keyOfSize(4, minKeySize), 3 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 1 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 9 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(4, minKeySize), 2)), + clearRange(keyOfSize(2, minKeySize), 9, keyOfSize(4, minKeySize)), + points(point(keyOfSize(1, minKeySize), 1)), + }, + rangeMin: 6, + batchMax: minKeySize * 2, + }, + { + // Test verifies that point batch history is not lost when non-garbage + // data is found. Batch spans multiple versions of a key. + name: "clear multiple points batches", + data: keySeq( + hist(keyOfSize(3, minKeySize), 10 /* versions */, 8 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 10 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(3, minKeySize), 5)), + points(point(keyOfSize(3, minKeySize), 8)), + clearRange(keyOfSize(1, minKeySize), 1, keyOfSize(3, minKeySize)), + }, + rangeMin: 10, + batchMax: minKeySize*4 + 1, + }, + { + // Test verifies that point batch history is not lost when non-garbage + // data is found. Batch spans multiple keys. + name: "clear multiple points batches 2", + data: keySeq( + hist(keyOfSize(6, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(5, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(4, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 10 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(6, minKeySize), 2), + point(keyOfSize(5, minKeySize), 2), + point(keyOfSize(4, minKeySize), 1), + ), + points( + point(keyOfSize(4, minKeySize), 2), + point(keyOfSize(3, minKeySize), 1), + ), + clearRange(keyOfSize(1, minKeySize), 1, keyOfSize(3, minKeySize)), + }, + rangeMin: 10, + batchMax: 100, + }, + { + // Test clear range reaches start of the range. + name: "clear range to first key", + data: keySeq( + hist(keyOfSize(3, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 10 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 2 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(3, minKeySize), 1)), + clearRange(keyOfSize(1, minKeySize), 2, keyOfSize(3, minKeySize)), + }, + rangeMin: 10, + batchMax: 100, + }, + { + // Test clear range reaches start of the range. + name: "clear points to first key", + data: keySeq( + hist(keyOfSize(3, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 8 /* versions */, 8 /* garbage */), + hist(keyOfSize(1, minKeySize), 3 /* versions */, 3 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(3, minKeySize), 1), + point(keyOfSize(2, minKeySize), 3), + ), + points( + point(keyOfSize(2, minKeySize), 7), + ), + points( + point(keyOfSize(2, minKeySize), 8), + point(keyOfSize(1, minKeySize), 3), + ), + }, + rangeMin: 100, + batchMax: 96, + }, + { + // Test clear range ends at the end of the range. + name: "clear range to last key", + data: keySeq( + hist(keyOfSize(3, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 10 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 1 /* garbage */), + ), + reqs: []gcReq{ + clearRange(keyOfSize(1, minKeySize), 1, keyOfSize(3, minKeySize).Next()), + }, + rangeMin: 10, + }, + { + // Test memory limit is reached before min range limit. + // For memory limits we need multiple keys (not versions) since we only + // keep latest version of the key per batch. + name: "prevent clear range when reaching memory limit", + data: keySeq( + hist(keyOfSize(6, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(5, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(4, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(2, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 2 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(6, minKeySize), 2), + point(keyOfSize(5, minKeySize), 2), + ), + points( + point(keyOfSize(4, minKeySize), 2), + point(keyOfSize(3, minKeySize), 2), + ), + points( + point(keyOfSize(2, minKeySize), 2), + point(keyOfSize(1, minKeySize), 2), + ), + }, + rangeMin: 6, + batchMax: 96, + memMax: 1, + }, + { + // Test memory limit is reached before min range limit. + // For memory limits we need multiple keys (not versions) since we only + // keep latest version of the key per batch. + name: "delay clear range when reaching memory limit", + data: keySeq( + hist(keyOfSize(6, largeKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(5, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(4, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(2, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 2 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(6, largeKeySize), 1), + ), + points( + point(keyOfSize(6, largeKeySize), 2), + ), + // Next() is iffy, but we don't have a better option when we flush the + // batch as we don't always know if there are more versions of the same + // key or not that should be covered by clear range. + clearRange(keyOfSize(1, minKeySize), 2, keyOfSize(5, minKeySize).Next()), + }, + rangeMin: 5, + batchMax: 100, + // When calculating mem size we don't use encoded length, but raw key and + // timestamp length which adds 3 extra bytes. + memMax: 100, + }, + { + // Test memory limit is reached before min range limit. Range starts in + // the middle of the points batch. Is it even possible? + name: "delay clear range to mid batch", + data: keySeq( + hist(keyOfSize(6, largeKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(5, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(4, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(3, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(2, minKeySize), 2 /* versions */, 2 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 2 /* garbage */), + ), + reqs: []gcReq{ + points( + point(keyOfSize(6, largeKeySize), 1), + point(keyOfSize(5, minKeySize), 1), + ), + clearRange(keyOfSize(1, minKeySize), 2, keyOfSize(5, minKeySize).Next()), + }, + rangeMin: 6, + batchMax: 101, + // When calculating mem size we don't use encoded length, but raw key and + // timestamp length which adds 3 extra bytes. + memMax: 120, + }, + { + // Verify that when we delete multiple versions of the same key, then we + // don't send the end range key to signal that we can avoid locking. + name: "clear range optimize single key deletions for several versions", + data: keySeq( + hist(keyOfSize(3, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 10 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 0 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(3, minKeySize), 1)), + clearRange(keyOfSize(2, minKeySize), 10, keyOfSize(2, minKeySize).Next()), + }, + rangeMin: 6, + }, + { + // Verify that when we delete multiple versions of the same key, then we + // don't send the end range key to signal that we can avoid locking. + name: "clear range optimize single key deletions for all versions", + data: keySeq( + hist(keyOfSize(3, minKeySize), 2 /* versions */, 1 /* garbage */), + hist(keyOfSize(2, minKeySize), 10 /* versions */, 9 /* garbage */), + hist(keyOfSize(1, minKeySize), 2 /* versions */, 0 /* garbage */), + ), + reqs: []gcReq{ + points(point(keyOfSize(3, minKeySize), 1)), + clearRange(keyOfSize(2, minKeySize), 9, keyOfSize(2, minKeySize).Next()), + }, + rangeMin: 6, + }, + } { + t.Run(d.name, func(t *testing.T) { + g := capturingGCer{} + enabled := true + if d.rangeMin == 0 { + enabled = false + } + memMax := d.memMax + if memMax == 0 { + memMax = math.MaxInt + } + batchMax := d.batchMax + if batchMax == 0 { + batchMax = math.MaxInt64 + } + b := gcKeyBatcher{ + gcKeyBatcherThresholds: gcKeyBatcherThresholds{ + batchGCKeysBytesThreshold: batchMax, + clearRangeMinKeys: d.rangeMin, + clearRangeEnabled: enabled, + maxPendingKeysSize: memMax, + }, + gcer: &g, + info: &Info{}, + pointsBatches: make([]pointsBatch, 1), + clearRangeEndKey: keys.MaxKey, + prevWasNewest: true, + } + + assertBatcher := func() { + // Assert used memory invariants. + totalMem := 0 + for _, b := range b.pointsBatches { + batchMem := 0 + for _, k := range b.batchGCKeys { + batchMem += len(k.Key) + hlcTimestampSize + } + require.Equal(t, batchMem, b.memUsed, "batch memory usage") + totalMem += batchMem + } + require.Equal(t, totalMem, b.totalMemUsed, "batcher memory usage") + } + + process := func(k gcData, newest bool) { + var ( + err error + ) + if k.garbage { + err = b.foundGarbage(ctx, &k.kv, newest) + } else { + err = b.foundNonGCableData(ctx, &k.kv, newest) + } + require.NoError(t, err, "failed to update batcher") + assertBatcher() + } + + // Run simulated GC + var prev gcData + for _, k := range d.data { + if prev.kv.key.Key != nil { + newest := prev.kv.key.Key.Compare(k.kv.key.Key) != 0 + process(prev, newest) + } + prev = k + } + process(prev, true) + err := b.flushLastBatch(ctx) + require.NoError(t, err, "failed last batch") + assertBatcher() + + // Assert batch history + require.EqualValues(t, d.reqs, g.ops, "Produced batches") + }) + } +} + +func keySeq(keyHistory ...[]gcData) (res []gcData) { + for _, h := range keyHistory { + res = append(res, h...) + } + return res +} diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index eabb5ccd811b..af9fe5f7d06a 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -1416,13 +1416,13 @@ The messages are dropped to help these replicas to recover from I/O overload.`, } metaGCUsedClearRange = metric.Metadata{ Name: "queue.gc.info.clearrangesuccess", - Help: "Number of successful ClearRange operation during GC", + Help: "Number of successful ClearRange operations during GC", Measurement: "Requests", Unit: metric.Unit_COUNT, } metaGCFailedClearRange = metric.Metadata{ Name: "queue.gc.info.clearrangefailed", - Help: "Number of failed ClearRange operation during GC", + Help: "Number of failed ClearRange operations during GC", Measurement: "Requests", Unit: metric.Unit_COUNT, } diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 12849e75930a..522d3ec6aa52 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -18,6 +18,7 @@ import ( "sync/atomic" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" @@ -626,15 +627,15 @@ func (r *replicaGCer) GC( ctx context.Context, keys []roachpb.GCRequest_GCKey, rangeKeys []roachpb.GCRequest_GCRangeKey, - clearRangeKey *roachpb.GCRequest_GCClearRangeKey, + clearRange *roachpb.GCRequest_GCClearRange, ) error { - if len(keys) == 0 && len(rangeKeys) == 0 && clearRangeKey == nil { + if len(keys) == 0 && len(rangeKeys) == 0 && clearRange == nil { return nil } req := r.template() req.Keys = keys req.RangeKeys = rangeKeys - req.ClearRangeKey = clearRangeKey + req.ClearRange = clearRange return r.send(ctx, req) } @@ -715,6 +716,10 @@ func (mgcq *mvccGCQueue) process( maxIntentsPerCleanupBatch := gc.MaxIntentsPerCleanupBatch.Get(&repl.store.ClusterSettings().SV) maxIntentKeyBytesPerCleanupBatch := gc.MaxIntentKeyBytesPerCleanupBatch.Get(&repl.store.ClusterSettings().SV) txnCleanupThreshold := gc.TxnCleanupThreshold.Get(&repl.store.ClusterSettings().SV) + var clearRangeMinKeys int64 = 0 + if repl.store.ClusterSettings().Version.IsActive(ctx, clusterversion.V23_1) { + clearRangeMinKeys = gc.ClearRangeMinKeys.Get(&repl.store.ClusterSettings().SV) + } info, err := gc.Run(ctx, desc, snap, gcTimestamp, newThreshold, gc.RunOptions{ @@ -724,6 +729,7 @@ func (mgcq *mvccGCQueue) process( TxnCleanupThreshold: txnCleanupThreshold, MaxTxnsPerIntentCleanupBatch: intentresolver.MaxTxnsPerIntentCleanupBatch, IntentCleanupBatchTimeout: mvccGCQueueIntentBatchTimeout, + ClearRangeMinKeys: clearRangeMinKeys, }, conf.TTL(), &replicaGCer{ @@ -815,8 +821,8 @@ func updateStoreMetricsWithGCInfo(metrics *StoreMetrics, info gc.Info) { metrics.GCAbortSpanGCNum.Inc(int64(info.AbortSpanGCNum)) metrics.GCPushTxn.Inc(int64(info.PushTxn)) metrics.GCResolveTotal.Inc(int64(info.ResolveTotal)) - metrics.GCUsedClearRange.Inc(int64(info.ClearRangeKeyOperations)) - metrics.GCFailedClearRange.Inc(int64(info.ClearRangeKeyFailures)) + metrics.GCUsedClearRange.Inc(int64(info.ClearRangeSpanOperations)) + metrics.GCFailedClearRange.Inc(int64(info.ClearRangeSpanFailures)) } func (mgcq *mvccGCQueue) postProcessScheduled( diff --git a/pkg/kv/kvserver/rditer/replica_data_iter.go b/pkg/kv/kvserver/rditer/replica_data_iter.go index a8c6e5fe1914..bc7e2fc21c52 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter.go @@ -406,16 +406,83 @@ func IterateReplicaKeySpans( keyTypes := []storage.IterKeyType{storage.IterKeyTypePointsOnly, storage.IterKeyTypeRangesOnly} for _, span := range spans { for _, keyType := range keyTypes { - iter := reader.NewEngineIterator(storage.IterOptions{ - KeyTypes: keyType, - LowerBound: span.Key, - UpperBound: span.EndKey, - }) - ok, err := iter.SeekEngineKeyGE(storage.EngineKey{Key: span.Key}) - if err == nil && ok { - err = visitor(iter, span, keyType) + err := func() error { + iter := reader.NewEngineIterator(storage.IterOptions{ + KeyTypes: keyType, + LowerBound: span.Key, + UpperBound: span.EndKey, + }) + defer iter.Close() + ok, err := iter.SeekEngineKeyGE(storage.EngineKey{Key: span.Key}) + if err == nil && ok { + err = visitor(iter, span, keyType) + } + return err + }() + if err != nil { + return iterutil.Map(err) } - iter.Close() + } + } + return nil +} + +// IterateOptions instructs how points and ranges should be presented to visitor +// and if iterators should be visited in forward or reverse order. +// Reverse iterator are also positioned at the end of the range prior to being +// passed to visitor. +type IterateOptions struct { + CombineRangesAndPoints bool + Reverse bool + ExcludeUserKeySpan bool +} + +// IterateMVCCReplicaKeySpans iterates over replica's key spans in the similar +// way to IterateReplicaKeySpans, but uses MVCCIterator and gives additional +// options to create reverse iterators and to combine keys are ranges. +func IterateMVCCReplicaKeySpans( + desc *roachpb.RangeDescriptor, + reader storage.Reader, + options IterateOptions, + visitor func(storage.MVCCIterator, roachpb.Span, storage.IterKeyType) error, +) error { + if !reader.ConsistentIterators() { + panic("reader must provide consistent iterators") + } + spans := MakeReplicatedKeySpansExceptLockTable(desc) + if options.ExcludeUserKeySpan { + spans = MakeReplicatedKeySpansExcludingUserAndLockTable(desc) + } + if options.Reverse { + spanMax := len(spans) - 1 + for i := 0; i < len(spans)/2; i++ { + spans[spanMax-i], spans[i] = spans[i], spans[spanMax-i] + } + } + keyTypes := []storage.IterKeyType{storage.IterKeyTypePointsOnly, storage.IterKeyTypeRangesOnly} + if options.CombineRangesAndPoints { + keyTypes = []storage.IterKeyType{storage.IterKeyTypePointsAndRanges} + } + for _, span := range spans { + for _, keyType := range keyTypes { + err := func() error { + iter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + LowerBound: span.Key, + UpperBound: span.EndKey, + KeyTypes: keyType, + }) + defer iter.Close() + if options.Reverse { + iter.SeekLT(storage.MakeMVCCMetadataKey(span.EndKey)) + } else { + iter.SeekGE(storage.MakeMVCCMetadataKey(span.Key)) + } + ok, err := iter.Valid() + if err == nil && ok { + err = visitor(iter, span, keyType) + } + return err + }() if err != nil { return iterutil.Map(err) } diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index cc73567cf99e..cc679ea91328 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -148,52 +148,53 @@ func verifyRDReplicatedOnlyMVCCIter( }, hlc.Timestamp{WallTime: 42}) readWriter = spanset.NewReadWriterAt(readWriter, &spans, hlc.Timestamp{WallTime: 42}) } - iter := NewReplicaMVCCDataIterator(desc, readWriter, ReplicaDataIteratorOptions{ - Reverse: reverse, - IterKind: storage.MVCCKeyAndIntentsIterKind, - KeyTypes: storage.IterKeyTypePointsAndRanges, - }) - defer iter.Close() - next := iter.Next - if reverse { - next = iter.Prev - } var rangeStart roachpb.Key actualKeys := []storage.MVCCKey{} actualRanges := []storage.MVCCRangeKey{} - for { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - p, r := iter.HasPointAndRange() - if p { - if !reverse { - actualKeys = append(actualKeys, iter.Key()) - } else { - actualKeys = append([]storage.MVCCKey{iter.Key()}, actualKeys...) + err := IterateMVCCReplicaKeySpans(desc, readWriter, IterateOptions{ + CombineRangesAndPoints: false, + Reverse: reverse, + }, func(iter storage.MVCCIterator, span roachpb.Span, keyType storage.IterKeyType) error { + for { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break } - } - if r { - rangeKeys := iter.RangeKeys().Clone() - if !rangeKeys.Bounds.Key.Equal(rangeStart) { - rangeStart = rangeKeys.Bounds.Key.Clone() + p, r := iter.HasPointAndRange() + if p { if !reverse { - for _, v := range rangeKeys.Versions { - actualRanges = append(actualRanges, rangeKeys.AsRangeKey(v)) - } + actualKeys = append(actualKeys, iter.Key()) } else { - for i := rangeKeys.Len() - 1; i >= 0; i-- { - actualRanges = append([]storage.MVCCRangeKey{ - rangeKeys.AsRangeKey(rangeKeys.Versions[i])}, - actualRanges...) + actualKeys = append([]storage.MVCCKey{iter.Key()}, actualKeys...) + } + } + if r { + rangeKeys := iter.RangeKeys().Clone() + if !rangeKeys.Bounds.Key.Equal(rangeStart) { + rangeStart = rangeKeys.Bounds.Key + if !reverse { + for _, v := range rangeKeys.Versions { + actualRanges = append(actualRanges, rangeKeys.AsRangeKey(v)) + } + } else { + for i := rangeKeys.Len() - 1; i >= 0; i-- { + actualRanges = append([]storage.MVCCRangeKey{ + rangeKeys.AsRangeKey(rangeKeys.Versions[i]), + }, actualRanges...) + } } } } + if reverse { + iter.Prev() + } else { + iter.Next() + } } - next() - } + return nil + }) + require.NoError(t, err, "visitor failed") require.Equal(t, expectedKeys, actualKeys) require.Equal(t, expectedRangeKeys, actualRanges) } @@ -523,3 +524,174 @@ func benchReplicaEngineDataIterator(b *testing.B, numRanges, numKeysPerRange, va } } } + +func TestIterateMVCCReplicaKeySpans(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Set up a new engine and write a single range key across the entire span. + eng := storage.NewDefaultInMemForTesting() + defer eng.Close() + + require.NoError(t, eng.PutMVCCRangeKey(storage.MVCCRangeKey{ + StartKey: keys.MinKey.Next(), + EndKey: keys.MaxKey, + Timestamp: hlc.Timestamp{WallTime: 10}, + }, storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("a1"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("a2"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("b1"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("b2"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("c1"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + require.NoError(t, + eng.PutMVCC(storage.MVCCKey{Key: roachpb.Key("c2"), Timestamp: hlc.Timestamp{WallTime: 1}}, + storage.MVCCValue{})) + + // Use a snapshot for the iteration, because we need consistent + // iterators. + snapshot := eng.NewSnapshot() + defer snapshot.Close() + + // Iterate over three range descriptors, both replicated and unreplicated. + for _, d := range []struct { + desc roachpb.RangeDescriptor + pts []roachpb.Key + }{ + { + desc: roachpb.RangeDescriptor{ + RangeID: 1, + StartKey: roachpb.RKey("a"), + EndKey: roachpb.RKey("b"), + }, + pts: []roachpb.Key{ + roachpb.Key("a1"), + roachpb.Key("a2"), + }, + }, + { + desc: roachpb.RangeDescriptor{ + RangeID: 2, + StartKey: roachpb.RKey("b"), + EndKey: roachpb.RKey("c"), + }, + pts: []roachpb.Key{ + roachpb.Key("b1"), + roachpb.Key("b2"), + }, + }, + { + desc: roachpb.RangeDescriptor{ + RangeID: 3, + StartKey: roachpb.RKey("c"), + EndKey: roachpb.RKey("d"), + }, + pts: []roachpb.Key{ + roachpb.Key("c1"), + roachpb.Key("c2"), + }, + }, + } { + t.Run(d.desc.KeySpan().String(), func(t *testing.T) { + testutils.RunTrueAndFalse(t, "reverse", func(t *testing.T, reverse bool) { + // Each iteration would go through all spans because we usa a single + // range key covering all spans of all ranges. + expectedSpans := MakeReplicatedKeySpansExceptLockTable(&d.desc) + if reverse { + for i, j := 0, len(expectedSpans)-1; i < j; i, j = i+1, j-1 { + expectedSpans[i], expectedSpans[j] = expectedSpans[j], expectedSpans[i] + } + } + points := append([]roachpb.Key(nil), d.pts...) + if reverse { + for i, j := 0, len(points)-1; i < j; i, j = i+1, j-1 { + points[i], points[j] = points[j], points[i] + } + } + advance := func(iter storage.MVCCIterator) { + if reverse { + iter.Prev() + } else { + iter.Next() + } + } + t.Run("sequential", func(t *testing.T) { + var actualSpans []roachpb.Span + var actualPoints []roachpb.Key + require.NoError(t, IterateMVCCReplicaKeySpans(&d.desc, snapshot, + IterateOptions{CombineRangesAndPoints: false, Reverse: reverse}, + func(iter storage.MVCCIterator, span roachpb.Span, keyType storage.IterKeyType) error { + if keyType == storage.IterKeyTypePointsOnly { + for { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + p, r := iter.HasPointAndRange() + require.False(t, r, "unexpected ranges found") + require.True(t, p, "no points found") + actualPoints = append(actualPoints, iter.UnsafeKey().Key.Clone()) + advance(iter) + } + } + if keyType == storage.IterKeyTypeRangesOnly { + // We only count spans for range keys for simplicity. They should + // register since we have a single range key spanning all key + // space. + actualSpans = append(actualSpans, span.Clone()) + + p, r := iter.HasPointAndRange() + require.True(t, r, "no ranges found") + require.False(t, p, "unexpected points found") + rk := iter.RangeBounds() + require.True(t, span.Contains(rk), "found range key is not contained to iterator span") + advance(iter) + ok, err := iter.Valid() + require.NoError(t, err) + require.False(t, ok) + } + return nil + })) + require.Equal(t, points, actualPoints) + require.Equal(t, expectedSpans, actualSpans) + }) + + t.Run("combined", func(t *testing.T) { + var actualSpans []roachpb.Span + var actualPoints []roachpb.Key + require.NoError(t, IterateMVCCReplicaKeySpans(&d.desc, snapshot, IterateOptions{CombineRangesAndPoints: true, Reverse: reverse}, + func(iter storage.MVCCIterator, span roachpb.Span, keyType storage.IterKeyType) error { + actualSpans = append(actualSpans, span.Clone()) + _, r := iter.HasPointAndRange() + require.True(t, r, "must have range") + rk := iter.RangeBounds() + require.True(t, span.Contains(rk), "found range key is not contained to iterator span") + for { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + if p, _ := iter.HasPointAndRange(); p { + actualPoints = append(actualPoints, iter.UnsafeKey().Key.Clone()) + } + advance(iter) + } + return nil + })) + require.Equal(t, points, actualPoints) + require.Equal(t, expectedSpans, actualSpans) + }) + }) + }) + } +} diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 3556d1adcd99..f26c74176186 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1361,7 +1361,7 @@ func (gcr *GCRequest) flags() flag { flags := isWrite | isRange | bypassesReplicaCircuitBreaker // For clear range requests that GC entire range we don't want to batch with // anything else. - if gcr.ClearRangeKey != nil { + if gcr.ClearRange != nil { flags |= isAlone } return flags diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 2dbf6e25a496..4e66368bee02 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1076,17 +1076,36 @@ message GCRequest { // Threshold is the expiration timestamp. util.hlc.Timestamp threshold = 4 [(gogoproto.nullable) = false]; - // GCClearRangeKey contains a range for clear range operation. - message GCClearRangeKey { + // GCClearRange contains a range for GC requests that would use + // pebble range tombstones to delete data. + message GCClearRange { bytes start_key = 1 [(gogoproto.casttype) = "Key"]; bytes end_key = 2 [(gogoproto.casttype) = "Key"]; + util.hlc.Timestamp start_key_timestamp = 3 [(gogoproto.nullable) = false]; } - // clear_range_key contains zero or one range that would be deleted using - // storage range clear operation. Note that this range key must cover - // cleared range in its entirety, range must not have any data newer than - // GC threshold and all data must be covered by a range tombstone otherwise - // the request will fail. - GCClearRangeKey clear_range_key = 7; + // GCClearRange specifies a span that will be GCed using a Pebble range + // tombstone. This is not a generalized operation, it currently only has two + // very specialized modes: + // + // 1. Empty StartKeyTimestamp: used to delete an entire Raft range, both point + // keys and range keys. StartKey and EndKey must exactly match the Raft range + // bounds, and all data (including point tombstones) must be covered by + // an MVCC range tombstone, otherwise an error is returned. This is intended + // for use with SQL schema GC, which deletes e.g. an entire table using a + // single MVCC range tombstone. + // + // 2. Non-empty StartKeyTimestamp: deletes only point key versions in the span + // [StartKey@StartKeyTimestamp, EndKey), leaving newer versions of StartKey + // and all MVCC range keys intact. This is intended for bulk GC of long runs + // of point key garbage, with range keys GCed separately. + // + // In both cases, the request will error if any data is found above the GC + // threshold. + // TODO: generalize this to clear both point keys and range keys in an + // arbitrary key span (with or without a start key timestamp). The GC code + // currently isn't structured in a way that can easily process point keys and + // range keys simultaneously. + GCClearRange clear_range = 7; reserved 5; } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 10d8d99a9b04..f032d32dea90 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -758,8 +758,8 @@ func updateStatsOnClear( // updateStatsOnGC updates stat counters after garbage collection // by subtracting key and value byte counts, updating key and -// value counts, and updating the GC'able bytes age. If meta is -// not nil, then the value being GC'd is the mvcc metadata and we +// value counts, and updating the GC'able bytes age. If metaKey is +// true, then the value being GC'd is the mvcc metadata and we // decrement the key count. // // nonLiveMS is the timestamp at which the value became non-live. @@ -767,13 +767,13 @@ func updateStatsOnClear( // in updateStatsOnPut) and for a regular version it will be the closest // newer version's (rule one). func updateStatsOnGC( - key roachpb.Key, keySize, valSize int64, meta *enginepb.MVCCMetadata, nonLiveMS int64, + key roachpb.Key, keySize, valSize int64, metaKey bool, nonLiveMS int64, ) enginepb.MVCCStats { var ms enginepb.MVCCStats if isSysLocal(key) { - ms.SysBytes -= (keySize + valSize) - if meta != nil { + ms.SysBytes -= keySize + valSize + if metaKey { ms.SysCount-- } return ms @@ -782,7 +782,7 @@ func updateStatsOnGC( ms.AgeTo(nonLiveMS) ms.KeyBytes -= keySize ms.ValBytes -= valSize - if meta != nil { + if metaKey { ms.KeyCount-- } else { ms.ValCount-- @@ -4961,7 +4961,7 @@ func MVCCGarbageCollect( // If first object in history is at or below gcKey timestamp then we // have no explicit meta and all objects are subject to deletion. if ms != nil { - ms.Add(updateStatsOnGC(gcKey.Key, metaKeySize, metaValSize, meta, + ms.Add(updateStatsOnGC(gcKey.Key, metaKeySize, metaValSize, true, /* metaKey */ realKeyChanged.WallTime)) } } @@ -4984,7 +4984,7 @@ func MVCCGarbageCollect( updateStatsForInline(ms, gcKey.Key, metaKeySize, metaValSize, 0, 0) ms.AgeTo(timestamp.WallTime) } else { - ms.Add(updateStatsOnGC(gcKey.Key, metaKeySize, metaValSize, meta, meta.Timestamp.WallTime)) + ms.Add(updateStatsOnGC(gcKey.Key, metaKeySize, metaValSize, true /* metaKey */, meta.Timestamp.WallTime)) } } if !implicitMeta { @@ -5140,7 +5140,7 @@ func MVCCGarbageCollect( } } - ms.Add(updateStatsOnGC(gcKey.Key, keySize, valSize, nil, fromNS)) + ms.Add(updateStatsOnGC(gcKey.Key, keySize, valSize, false /* metaKey */, fromNS)) } count++ if err := rw.ClearMVCC(unsafeIterKey); err != nil { @@ -5423,6 +5423,143 @@ func CanGCEntireRange( return coveredByRangeTombstones, nil } +// MVCCGarbageCollectPointsWithClearRange removes garbage collected points data +// within range [start@startTimestamp, endTimestamp). This function performs a +// check to ensure that no non-garbage data (most recent or history with +// timestamp greater that threshold) is being deleted. Range tombstones are kept +// intact and need to be removed separately. +func MVCCGarbageCollectPointsWithClearRange( + ctx context.Context, + rw ReadWriter, + ms *enginepb.MVCCStats, + start, end roachpb.Key, + startTimestamp hlc.Timestamp, + gcThreshold hlc.Timestamp, +) error { + var countKeys int64 + var removedEntries int64 + defer func(begin time.Time) { + // TODO(oleg): this could be misleading if GC fails, but this function still + // reports how many keys were GC'd. The approach is identical to what point + // key GC does for consistency, but both places could be improved. + log.Eventf(ctx, + "done with GC evaluation for clear range of %d keys at %.2f keys/sec. Deleted %d entries", + countKeys, float64(countKeys)*1e9/float64(timeutil.Since(begin)), removedEntries) + }(timeutil.Now()) + + iter := rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + LowerBound: start, + UpperBound: end, + KeyTypes: IterKeyTypePointsAndRanges, + }) + defer iter.Close() + + iter.SeekGE(MVCCKey{Key: start}) + + var ( + // prevPointKey is a newer version (with higher timestamp) of current key. + // Its key component is updated when key is first seen at the beginning of + // loop, and timestamp is reset to empty. + // Its timestamp component is updated at the end of loop (including + // continue). + // It is used to check that current key is covered and eligible for GC as + // well as mvcc stats calculations. + prevPointKey MVCCKey + rangeTombstones MVCCRangeKeyStack + firstKey = true + ) + + for ; ; iter.Next() { + if ok, err := iter.Valid(); err != nil { + return err + } else if !ok { + break + } + + if iter.RangeKeyChanged() { + iter.RangeKeys().CloneInto(&rangeTombstones) + if hasPoint, _ := iter.HasPointAndRange(); !hasPoint { + continue + } + } + + // Invariant: we're now positioned on a point key. The iterator can only + // be positioned on a bare range key when `RangeKeyChanged()` returns `true`. + countKeys++ + unsafeKey := iter.UnsafeKey() + newKey := !prevPointKey.Key.Equal(unsafeKey.Key) + if newKey { + unsafeKey.CloneInto(&prevPointKey) + prevPointKey.Timestamp = hlc.Timestamp{} + } + + // Skip keys that fall outside of range (only until we reach the first + // eligible key). + if firstKey && unsafeKey.Compare(MVCCKey{Key: start, Timestamp: startTimestamp}) < 0 { + prevPointKey.Timestamp = unsafeKey.Timestamp + continue + } + firstKey = false + + if unsafeKey.Timestamp.IsEmpty() { + // Found unresolved intent. We use .String() explicitly as it is not + // including timestamps if they are zero, but Format() does. + return errors.Errorf("attempt to GC intent %s using clear range", + unsafeKey.String()) + } + if gcThreshold.Less(unsafeKey.Timestamp) { + // Current version is above GC threshold so it is not safe to clear. + return errors.Errorf("attempt to GC data %s above threshold %s with clear range", + unsafeKey, gcThreshold) + } + + valueLen, isTombstone, err := iter.MVCCValueLenAndIsTombstone() + if err != nil { + return err + } + + // Find timestamp covering current key. + coveredBy := prevPointKey.Timestamp + if rangeKeyCover, ok := rangeTombstones.FirstAtOrAbove(unsafeKey.Timestamp); ok { + // If there's a range between current value and value above + // use that timestamp. + if coveredBy.IsEmpty() || rangeKeyCover.Timestamp.Less(coveredBy) { + coveredBy = rangeKeyCover.Timestamp + } + } + + if isGarbage := !coveredBy.IsEmpty() && coveredBy.LessEq(gcThreshold) || isTombstone; !isGarbage { + // Current version is below threshold and is not a tombstone, but + // preceding one is above so it is visible and can't be cleared. + return errors.Errorf("attempt to GC data %s still visible at GC threshold %s with clear range", + unsafeKey, gcThreshold) + } + + validTill := coveredBy + if isTombstone { + validTill = unsafeKey.Timestamp + } + + if ms != nil { + if newKey { + ms.Add(updateStatsOnGC(unsafeKey.Key, int64(EncodedMVCCKeyPrefixLength(unsafeKey.Key)), 0, + true /* metaKey */, validTill.WallTime)) + } + ms.Add(updateStatsOnGC(unsafeKey.Key, MVCCVersionTimestampSize, int64(valueLen), false, /* metaKey */ + validTill.WallTime)) + } + prevPointKey.Timestamp = unsafeKey.Timestamp + removedEntries++ + } + + // If timestamp is not empty we delete subset of versions (this may be first + // key of requested range or full extent). + if err := rw.ClearMVCCVersions(MVCCKey{Key: start, Timestamp: startTimestamp}, MVCCKey{Key: end}); err != nil { + return err + } + return nil +} + // MVCCFindSplitKey finds a key from the given span such that the left side of // the split is roughly targetSize bytes. It only considers MVCC point keys, not // range keys. The returned key will never be chosen from the key ranges listed diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index c8d55299c74c..fa3d50a5f59e 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -119,6 +119,7 @@ var ( // clear_time_range k= end= ts=[,] targetTs=[,] [clearRangeThreshold=] [maxBatchSize=] [maxBatchByteSize=] // // gc_clear_range k= end= startTs=[,] ts=[,] +// gc_points_clear_range k= end= startTs=[,] ts=[,] // replace_point_tombstones_with_range_tombstones k= [end=] // // sst_put [ts=[,]] [localTs=[,]] k= [v=] @@ -693,25 +694,26 @@ var commands = map[string]cmd{ "check_intent": {typReadOnly, cmdCheckIntent}, "add_lock": {typLocksUpdate, cmdAddLock}, - "clear": {typDataUpdate, cmdClear}, - "clear_range": {typDataUpdate, cmdClearRange}, - "clear_rangekey": {typDataUpdate, cmdClearRangeKey}, - "clear_time_range": {typDataUpdate, cmdClearTimeRange}, - "cput": {typDataUpdate, cmdCPut}, - "del": {typDataUpdate, cmdDelete}, - "del_range": {typDataUpdate, cmdDeleteRange}, - "del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone}, - "del_range_pred": {typDataUpdate, cmdDeleteRangePredicate}, - "export": {typReadOnly, cmdExport}, - "get": {typReadOnly, cmdGet}, - "gc_clear_range": {typDataUpdate, cmdGCClearRange}, - "increment": {typDataUpdate, cmdIncrement}, - "initput": {typDataUpdate, cmdInitPut}, - "merge": {typDataUpdate, cmdMerge}, - "put": {typDataUpdate, cmdPut}, - "put_rangekey": {typDataUpdate, cmdPutRangeKey}, - "scan": {typReadOnly, cmdScan}, - "is_span_empty": {typReadOnly, cmdIsSpanEmpty}, + "clear": {typDataUpdate, cmdClear}, + "clear_range": {typDataUpdate, cmdClearRange}, + "clear_rangekey": {typDataUpdate, cmdClearRangeKey}, + "clear_time_range": {typDataUpdate, cmdClearTimeRange}, + "cput": {typDataUpdate, cmdCPut}, + "del": {typDataUpdate, cmdDelete}, + "del_range": {typDataUpdate, cmdDeleteRange}, + "del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone}, + "del_range_pred": {typDataUpdate, cmdDeleteRangePredicate}, + "export": {typReadOnly, cmdExport}, + "get": {typReadOnly, cmdGet}, + "gc_clear_range": {typDataUpdate, cmdGCClearRange}, + "gc_points_clear_range": {typDataUpdate, cmdGCPointsClearRange}, + "increment": {typDataUpdate, cmdIncrement}, + "initput": {typDataUpdate, cmdInitPut}, + "merge": {typDataUpdate, cmdMerge}, + "put": {typDataUpdate, cmdPut}, + "put_rangekey": {typDataUpdate, cmdPutRangeKey}, + "scan": {typReadOnly, cmdScan}, + "is_span_empty": {typReadOnly, cmdIsSpanEmpty}, "iter_new": {typReadOnly, cmdIterNew}, "iter_new_incremental": {typReadOnly, cmdIterNewIncremental}, // MVCCIncrementalIterator @@ -1021,6 +1023,15 @@ func cmdGCClearRange(e *evalCtx) error { }) } +func cmdGCPointsClearRange(e *evalCtx) error { + key, endKey := e.getKeyRange() + gcTs := e.getTs(nil) + startTs := e.getTsWithName("startTs") + return e.withWriter("gc_clear_range", func(rw storage.ReadWriter) error { + return storage.MVCCGarbageCollectPointsWithClearRange(e.ctx, rw, e.ms, key, endKey, startTs, gcTs) + }) +} + func cmdCPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 570c1cc52d13..17a771229eb6 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -4986,6 +4986,11 @@ func pt(key roachpb.Key, ts hlc.Timestamp) rangeTestDataItem { // values. var inlineValue hlc.Timestamp +// tb creates a point tombstone. +func tb(key roachpb.Key, ts hlc.Timestamp) rangeTestDataItem { + return rangeTestDataItem{point: MVCCKeyValue{Key: mvccVersionKey(key, ts)}} +} + // txn wraps point update and adds transaction to it for intent creation. func txn(d rangeTestDataItem) rangeTestDataItem { ts := d.point.Key.Timestamp @@ -5648,8 +5653,8 @@ func TestMVCCGarbageCollectClearRange(t *testing.T) { tsGC := mkTs(5) tsMax := mkTs(9) - mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRangeKey { - return roachpb.GCRequest_GCClearRangeKey{ + mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRange { + return roachpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, } @@ -5711,8 +5716,8 @@ func TestMVCCGarbageCollectClearRangeInlinedValue(t *testing.T) { tsGC := mkTs(5) - mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRangeKey { - return roachpb.GCRequest_GCClearRangeKey{ + mkGCReq := func(start roachpb.Key, end roachpb.Key) roachpb.GCRequest_GCClearRange { + return roachpb.GCRequest_GCClearRange{ StartKey: start, EndKey: end, } @@ -5739,6 +5744,106 @@ func TestMVCCGarbageCollectClearRangeInlinedValue(t *testing.T) { "expected error '%s' found '%s'", expectedError, err) } +func TestMVCCGarbageCollectClearPointsInRange(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + mkKey := func(k string) roachpb.Key { + return append(keys.SystemSQLCodec.TablePrefix(42), k...) + } + rangeStart := mkKey("") + rangeEnd := rangeStart.PrefixEnd() + + // Note we use keys of different lengths so that stats accounting errors + // would not obviously cancel out if right and left bounds are used + // incorrectly. + keyA := mkKey("a") + keyB := mkKey("bb") + keyC := mkKey("ccc") + keyD := mkKey("dddd") + + mkTs := func(wallTimeSec int64) hlc.Timestamp { + return hlc.Timestamp{WallTime: time.Second.Nanoseconds() * wallTimeSec} + } + + ts1 := mkTs(1) + ts2 := mkTs(2) + ts3 := mkTs(3) + ts4 := mkTs(4) + tsMax := mkTs(9) + + engine := NewDefaultInMemForTesting() + defer engine.Close() + + var ms enginepb.MVCCStats + rangeTestData{ + pt(keyB, ts1), + pt(keyB, ts2), + tb(keyB, ts3), + tb(keyB, ts4), + pt(keyC, ts1), + tb(keyC, ts2), + pt(keyC, ts3), + tb(keyC, ts4), + }.populateEngine(t, engine, &ms) + + require.NoError(t, + MVCCGarbageCollectPointsWithClearRange(ctx, engine, &ms, keyA, keyD, hlc.Timestamp{}, tsMax), + "failed to run mvcc range tombstone garbage collect") + + expected := NewDefaultInMemForTesting() + defer expected.Close() + var expMs enginepb.MVCCStats + + rks := scanRangeKeys(t, engine) + expRks := scanRangeKeys(t, expected) + require.EqualValues(t, expRks, rks) + + ks := scanPointKeys(t, engine) + expKs := scanPointKeys(t, expected) + require.EqualValues(t, expKs, ks) + + ms.AgeTo(tsMax.WallTime) + expMs, err := ComputeStats(engine, rangeStart, rangeEnd, tsMax.WallTime) + require.NoError(t, err, "failed to compute stats for range") + require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") +} + +func TestMVCCGarbageCollectClearRangeFailure(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + mkKey := func(k string) roachpb.Key { + return append(keys.SystemSQLCodec.TablePrefix(42), k...) + } + + // Note we use keys of different lengths so that stats accounting errors + // would not obviously cancel out if right and left bounds are used + // incorrectly. + keyA := mkKey("a") + keyD := mkKey("dddd") + + mkTs := func(wallTimeSec int64) hlc.Timestamp { + return hlc.Timestamp{WallTime: time.Second.Nanoseconds() * wallTimeSec} + } + + engine := NewDefaultInMemForTesting() + defer engine.Close() + + var ms enginepb.MVCCStats + rangeTestData{ + pt(keyA, mkTs(1)), + }.populateEngine(t, engine, &ms) + + err := MVCCGarbageCollectPointsWithClearRange(ctx, engine, &ms, keyA, keyD, mkTs(1), mkTs(5)) + errMsg := `attempt to GC data /Table/42/"a"/1.000000000,0 still visible at GC threshold 5.000000000,0 with clear range` + require.Errorf(t, err, "expected error '%s' but found none", errMsg) + require.True(t, testutils.IsError(err, errMsg), + "expected error '%s' found '%s'", errMsg, err) +} + // TestResolveIntentWithLowerEpoch verifies that trying to resolve // an intent at an epoch that is lower than the epoch of the intent // leaves the intent untouched. diff --git a/pkg/storage/testdata/mvcc_histories/gc_clear_full_range b/pkg/storage/testdata/mvcc_histories/gc_clear_full_range new file mode 100644 index 000000000000..ee8d92332813 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/gc_clear_full_range @@ -0,0 +1,121 @@ +run ok +put k=A v=B ts=10 +---- +>> at end: +data: "A"/10.000000000,0 -> /BYTES/B + +# Check that we can't invoke gc_clear_range if we have live data. +run error +gc_clear_range k=A end=Z ts=30 +---- +>> at end: +data: "A"/10.000000000,0 -> /BYTES/B +error: (*withstack.withStack:) range contains live data, can't use GC clear range + +run ok +del k=A ts=30 +---- +del: "A": found key true +>> at end: +data: "A"/30.000000000,0 -> / +data: "A"/10.000000000,0 -> /BYTES/B + +# Check that we can't invoke gc_clear_range if we are not covered by range tombstones. +run error +gc_clear_range k=A end=Z ts=30 +---- +>> at end: +data: "A"/30.000000000,0 -> / +data: "A"/10.000000000,0 -> /BYTES/B +error: (*withstack.withStack:) found key not covered by range tombstone "A"/30.000000000,0 + +run ok stats +del_range_ts k=A endKey=Z ts=50 +---- +>> del_range_ts k=A endKey=Z ts=50 +stats: range_key_count=+1 range_key_bytes=+14 range_val_count=+1 gc_bytes_age=+700 +>> at end: +rangekey: A{-\x00}/[50.000000000,0=/] +data: "A"/30.000000000,0 -> / +data: "A"/10.000000000,0 -> /BYTES/B +stats: key_count=1 key_bytes=26 val_count=2 val_bytes=6 range_key_count=1 range_key_bytes=14 range_val_count=1 gc_bytes_age=2940 + +# Check that we can't delete if range tombstone covering all range is above gc threshold. +run error +gc_clear_range k=A end=Z ts=40 +---- +>> at end: +rangekey: A{-\x00}/[50.000000000,0=/] +data: "A"/30.000000000,0 -> / +data: "A"/10.000000000,0 -> /BYTES/B +error: (*withstack.withStack:) range tombstones above gc threshold. GC=40.000000000,0, range=50.000000000,0 + +# Check that we can delete if range tombstone covers all range. +run stats ok +gc_clear_range k=A end=Z ts=60 +---- +>> gc_clear_range k=A end=Z ts=60 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-6 range_key_count=-1 range_key_bytes=-14 range_val_count=-1 gc_bytes_age=-2940 +>> at end: + +stats: + +# Check that is we have range tombstone coverage that covers subset but there's no other data we can still clear. +run ok +put k=A v=B ts=10 +del_range_ts k=A endKey=D ts=20 +---- +>> at end: +rangekey: A{-\x00}/[20.000000000,0=/] +data: "A"/10.000000000,0 -> /BYTES/B + +run stats ok +gc_clear_range k=A end=Z ts=60 +---- +>> gc_clear_range k=A end=Z ts=60 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-6 range_key_count=-1 range_key_bytes=-14 range_val_count=-1 gc_bytes_age=-2720 +>> at end: + +stats: + +# Check that we can't delete range if there's some data that is outside of range tombstone +run ok +put k=A v=B ts=5 +del_range_ts k=C endKey=Z ts=6 +---- +>> at end: +rangekey: C{-\x00}/[6.000000000,0=/] +data: "A"/5.000000000,0 -> /BYTES/B + +run error +gc_clear_range k=A end=Z ts=6 +---- +>> at end: +rangekey: C{-\x00}/[6.000000000,0=/] +data: "A"/5.000000000,0 -> /BYTES/B +error: (*withstack.withStack:) range contains live data, can't use GC clear range + +run ok +clear_range k=A end=Z +---- +>> at end: + + +# Check that gc clear range can't be performed over intents +run ok +with t=A + txn_begin ts=10 + put k=B v=O +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 +meta: "B"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "B"/10.000000000,0 -> /BYTES/O + +run error +gc_clear_range k=A end=Z ts=40 +---- +>> at end: +meta: "B"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "B"/10.000000000,0 -> /BYTES/O +error: (*withstack.withStack:) range contains live data, can't use GC clear range diff --git a/pkg/storage/testdata/mvcc_histories/gc_clear_range b/pkg/storage/testdata/mvcc_histories/gc_clear_range index d3f8dc651f6c..1352fca17db2 100644 --- a/pkg/storage/testdata/mvcc_histories/gc_clear_range +++ b/pkg/storage/testdata/mvcc_histories/gc_clear_range @@ -1,98 +1,263 @@ +# 1.Clear empty range +run ok stats +gc_points_clear_range k=a ts=3 end=z +---- +>> gc_points_clear_range k=a ts=3 end=z +stats: no change +>> at end: + +stats: + +# 2.Clear range starting from a version under value +run ok +put k=a v=11 ts=2,0 +put k=a v=12 ts=5,0 +---- +>> at end: +data: "a"/5.000000000,0 -> /BYTES/12 +data: "a"/2.000000000,0 -> /BYTES/11 + +run ok stats +gc_points_clear_range k=a startTs=2 end=z ts=5 +---- +>> gc_points_clear_range k=a startTs=2 end=z ts=5 +stats: key_bytes=-12 val_count=-1 val_bytes=-7 gc_bytes_age=-1805 +>> at end: +data: "a"/5.000000000,0 -> /BYTES/12 +stats: key_count=1 key_bytes=14 val_count=1 val_bytes=7 live_count=1 live_bytes=21 + +run ok +clear_range k=a end=z +---- +>> at end: + + +# 3. Clear range from version under tombstone +run ok +put k=a v=11 ts=2,0 +del k=a ts=5,0 +---- +del: "a": found key true +>> at end: +data: "a"/5.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/11 + +run ok stats +gc_points_clear_range k=a startTs=5 end=z ts=5 +---- +>> gc_points_clear_range k=a startTs=5 end=z ts=5 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-7 gc_bytes_age=-3135 +>> at end: + +stats: + +# 4. Clear range from version under range tombstone run ok -put k=A v=B ts=10 +put k=a v=11 ts=2,0 +del_range_ts k=a end=b ts=5,0 ---- >> at end: -data: "A"/10.000000000,0 -> /BYTES/B +rangekey: {a-b}/[5.000000000,0=/] +data: "a"/2.000000000,0 -> /BYTES/11 -# Check that we can't invoke gc_clear_range if we have live data. -run error -gc_clear_range k=A end=Z ts=30 +run ok stats +gc_points_clear_range k=a startTs=2 end=z ts=5 ---- +>> gc_points_clear_range k=a startTs=2 end=z ts=5 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-7 gc_bytes_age=-1995 >> at end: -data: "A"/10.000000000,0 -> /BYTES/B -error: (*withstack.withStack:) range contains live data, can't use GC clear range +rangekey: {a-b}/[5.000000000,0=/] +stats: range_key_count=1 range_key_bytes=13 range_val_count=1 gc_bytes_age=1235 run ok -del k=A ts=30 +clear_range k=a end=z ---- -del: "A": found key true >> at end: -data: "A"/30.000000000,0 -> / -data: "A"/10.000000000,0 -> /BYTES/B + -# Check that we can't invoke gc_clear_range if we are not covered by range tombstones. -run error -gc_clear_range k=A end=Z ts=30 +# 5. Clear range up to another value +run ok +put k=a v=11 ts=2,0 +put k=a v=12 ts=5,0 +put k=c v=13 ts=5,0 ---- >> at end: -data: "A"/30.000000000,0 -> / -data: "A"/10.000000000,0 -> /BYTES/B -error: (*withstack.withStack:) found key not covered by range tombstone "A"/30.000000000,0 +data: "a"/5.000000000,0 -> /BYTES/12 +data: "a"/2.000000000,0 -> /BYTES/11 +data: "c"/5.000000000,0 -> /BYTES/13 run ok stats -del_range_ts k=A endKey=Z ts=50 +gc_points_clear_range k=a startTs=2 end=c ts=5 +---- +>> gc_points_clear_range k=a startTs=2 end=c ts=5 +stats: key_bytes=-12 val_count=-1 val_bytes=-7 gc_bytes_age=-1805 +>> at end: +data: "a"/5.000000000,0 -> /BYTES/12 +data: "c"/5.000000000,0 -> /BYTES/13 +stats: key_count=2 key_bytes=28 val_count=2 val_bytes=14 live_count=2 live_bytes=42 + +run ok +clear_range k=a end=z ---- ->> del_range_ts k=A endKey=Z ts=50 -stats: range_key_count=+1 range_key_bytes=+14 range_val_count=+1 gc_bytes_age=+700 >> at end: -rangekey: A{-\x00}/[50.000000000,0=/] -data: "A"/30.000000000,0 -> / -data: "A"/10.000000000,0 -> /BYTES/B -stats: key_count=1 key_bytes=26 val_count=2 val_bytes=6 range_key_count=1 range_key_bytes=14 range_val_count=1 gc_bytes_age=2940 + + +# 6. Clear range up to another intent +run ok +put k=a v=11 ts=2,0 +put k=a v=12 ts=5,0 +with t=A k=c + txn_begin ts=4,0 + put v=1 +---- +>> at end: +txn: "A" meta={id=00000000 key="c" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=4.000000000,0 wto=false gul=0,0 +data: "a"/5.000000000,0 -> /BYTES/12 +data: "a"/2.000000000,0 -> /BYTES/11 +meta: "c"/0,0 -> txn={id=00000000 key="c" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} ts=4.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "c"/4.000000000,0 -> /BYTES/1 -# Check that we can't delete if range tombstone covering all range is above gc threshold. -run error -gc_clear_range k=A end=Z ts=40 +run ok stats +gc_points_clear_range k=a startTs=2 end=c ts=5 ---- +>> gc_points_clear_range k=a startTs=2 end=c ts=5 +stats: key_bytes=-12 val_count=-1 val_bytes=-7 gc_bytes_age=-1805 >> at end: -rangekey: A{-\x00}/[50.000000000,0=/] -data: "A"/30.000000000,0 -> / -data: "A"/10.000000000,0 -> /BYTES/B -error: (*withstack.withStack:) range tombstones above gc threshold. GC=40.000000000,0, range=50.000000000,0 +data: "a"/5.000000000,0 -> /BYTES/12 +meta: "c"/0,0 -> txn={id=00000000 key="c" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} ts=4.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "c"/4.000000000,0 -> /BYTES/1 +stats: key_count=2 key_bytes=28 val_count=2 val_bytes=64 live_count=2 live_bytes=92 intent_count=1 intent_bytes=18 separated_intent_count=1 intent_age=96 -# Check that we can delete if range tombstone covers all range. -run stats ok -gc_clear_range k=A end=Z ts=60 +run ok +clear_range k=a end=z ---- ->> gc_clear_range k=A end=Z ts=60 -stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-6 range_key_count=-1 range_key_bytes=-14 range_val_count=-1 gc_bytes_age=-2940 +>> at end: + + +# 7. Clear from before first key +run ok +put k=b v=11 ts=2,0 +del k=b ts=5,0 +---- +del: "b": found key true +>> at end: +data: "b"/5.000000000,0 -> / +data: "b"/2.000000000,0 -> /BYTES/11 + +run ok stats +gc_points_clear_range k=a end=z ts=5 +---- +>> gc_points_clear_range k=a end=z ts=5 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-7 gc_bytes_age=-3135 >> at end: stats: -# Check that is we have range tombstone coverage that covers subset but there's no other data we can still clear. run ok -put k=A v=B ts=10 -del_range_ts k=A endKey=D ts=20 +clear_range k=a end=z +---- +>> at end: + + +# 8. Clear value under range tombstone +run ok +put k=b v=123 ts=2,0 +del_range_ts k=a end=c ts=5,0 +---- +>> at end: +rangekey: {a-c}/[5.000000000,0=/] +data: "b"/2.000000000,0 -> /BYTES/123 + +run ok stats +gc_points_clear_range k=a end=z ts=5 +---- +>> gc_points_clear_range k=a end=z ts=5 +stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-8 gc_bytes_age=-2090 +>> at end: +rangekey: {a-c}/[5.000000000,0=/] +stats: range_key_count=1 range_key_bytes=13 range_val_count=1 gc_bytes_age=1235 + +run ok +clear_range k=a end=z +---- +>> at end: + + +# 9. Clear value under point tombstone +run ok +put k=b v=123 ts=2,0 +del k=b ts=5,0 ---- +del: "b": found key true >> at end: -rangekey: A{-\x00}/[20.000000000,0=/] -data: "A"/10.000000000,0 -> /BYTES/B +data: "b"/5.000000000,0 -> / +data: "b"/2.000000000,0 -> /BYTES/123 -run stats ok -gc_clear_range k=A end=Z ts=60 +run ok stats +gc_points_clear_range k=a end=z ts=5 ---- ->> gc_clear_range k=A end=Z ts=60 -stats: key_count=-1 key_bytes=-14 val_count=-1 val_bytes=-6 range_key_count=-1 range_key_bytes=-14 range_val_count=-1 gc_bytes_age=-2720 +>> gc_points_clear_range k=a end=z ts=5 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-8 gc_bytes_age=-3230 >> at end: stats: -# Check that gc clear range can't be performed over intents run ok -with t=A - txn_begin ts=10 - put k=B v=O +clear_range k=a end=z ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=0,0 -meta: "B"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true -data: "B"/10.000000000,0 -> /BYTES/O + -run error -gc_clear_range k=A end=Z ts=40 +# 10. Clear value under point and range tombstone (checking that garbage age is correctly calculated) +run ok +put k=b v=123 ts=2,0 +del k=b ts=4,0 +del_range_ts k=a end=c ts=5,0 +---- +del: "b": found key true +>> at end: +rangekey: {a-c}/[5.000000000,0=/] +data: "b"/4.000000000,0 -> / +data: "b"/2.000000000,0 -> /BYTES/123 + +run ok stats +gc_points_clear_range k=a end=z ts=5 +---- +>> gc_points_clear_range k=a end=z ts=5 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-8 gc_bytes_age=-3264 +>> at end: +rangekey: {a-c}/[5.000000000,0=/] +stats: range_key_count=1 range_key_bytes=13 range_val_count=1 gc_bytes_age=1235 + +run ok +clear_range k=a end=z ---- >> at end: -meta: "B"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true -data: "B"/10.000000000,0 -> /BYTES/O -error: (*withstack.withStack:) range contains live data, can't use GC clear range + + +# 11. Clear value under range and point tombstone (checking that garbage age is correctly calculated) +run ok +put k=b v=123 ts=2,0 +del_range_ts k=a end=c ts=4,0 +del k=b ts=5,0 +---- +del: "b": found key false +>> at end: +rangekey: {a-c}/[4.000000000,0=/] +data: "b"/5.000000000,0 -> / +data: "b"/2.000000000,0 -> /BYTES/123 + +run ok stats +gc_points_clear_range k=a end=z ts=5 +---- +>> gc_points_clear_range k=a end=z ts=5 +stats: key_count=-1 key_bytes=-26 val_count=-2 val_bytes=-8 gc_bytes_age=-3250 +>> at end: +rangekey: {a-c}/[4.000000000,0=/] +stats: range_key_count=1 range_key_bytes=13 range_val_count=1 gc_bytes_age=1248 + +run ok +clear_range k=a end=z +---- +>> at end: + diff --git a/pkg/storage/testdata/mvcc_histories/gc_clear_range_errors b/pkg/storage/testdata/mvcc_histories/gc_clear_range_errors new file mode 100644 index 000000000000..5e962deeb7e8 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/gc_clear_range_errors @@ -0,0 +1,94 @@ +# 1. Fail to clear non deleted key +run ok +put k=a v=12 ts=2,0 +---- +>> at end: +data: "a"/2.000000000,0 -> /BYTES/12 + +run error +gc_points_clear_range k=a startTs=2 end=z ts=5 +---- +>> at end: +data: "a"/2.000000000,0 -> /BYTES/12 +error: (*withstack.withStack:) attempt to GC data "a"/2.000000000,0 still visible at GC threshold 5.000000000,0 with clear range + +run ok +clear_range k=a end=z +---- +>> at end: + + +# 2. Fail to clear intent +run ok +with t=A k=a + txn_begin ts=4,0 + put v=1 +---- +>> at end: +txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=4.000000000,0 wto=false gul=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} ts=4.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "a"/4.000000000,0 -> /BYTES/1 + +run error +gc_points_clear_range k=a end=z ts=5 +---- +>> at end: +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=4.000000000,0 min=0,0 seq=0} ts=4.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "a"/4.000000000,0 -> /BYTES/1 +error: (*withstack.withStack:) attempt to GC intent "a" using clear range + +run ok +clear_range k=a end=z +---- +>> at end: + + +# 3. Fail to delete key above gc threshold +run ok +put k=a v=11 ts=2,0 +del k=a ts=5,0 +---- +del: "a": found key true +>> at end: +data: "a"/5.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/11 + +run error +gc_points_clear_range k=a startTs=2 end=z ts=4 +---- +>> at end: +data: "a"/5.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/11 +error: (*withstack.withStack:) attempt to GC data "a"/2.000000000,0 still visible at GC threshold 4.000000000,0 with clear range + +run ok +clear_range k=a end=z +---- +>> at end: + + +# 4. Fail to delete key above range tombstone (checking if range tombstones are not breaking timestamp check logic) +run ok +put k=a v=11 ts=2,0 +del_range_ts k=a end=b ts=3,0 +put k=a v=12 ts=5,0 +---- +>> at end: +rangekey: {a-b}/[3.000000000,0=/] +data: "a"/5.000000000,0 -> /BYTES/12 +data: "a"/2.000000000,0 -> /BYTES/11 + +run error +gc_points_clear_range k=a startTs=5 end=z ts=5 +---- +>> at end: +rangekey: {a-b}/[3.000000000,0=/] +data: "a"/5.000000000,0 -> /BYTES/12 +data: "a"/2.000000000,0 -> /BYTES/11 +error: (*withstack.withStack:) attempt to GC data "a"/5.000000000,0 still visible at GC threshold 5.000000000,0 with clear range + +run ok +clear_range k=a end=z +---- +>> at end: +