Skip to content

Commit

Permalink
storage: clear range keys in Writer.Clear*Range methods
Browse files Browse the repository at this point in the history
This patch clears range keys in the `Writer` methods `ClearRawRange`,
`ClearMVCCRange`, and `ClearMVCCIteratorRange`, as well as in the
`ClearRangeWithHeuristic` helper.

Range keys are not cleared in `ClearMVCCVersions`, since this method is
specifically for clearing MVCC point key versions, and it is not
possible to clear range keys between versions of the same point key.

`Engine.ExperimentalClearAllMVCCRangeKeys()` has been improved to scan
for any range keys in the given span, and only clear the smallest single
span that covers all range keys, to avoid dropping unnecessary Pebble
range tombstones across these range key spans.

Release note: None
  • Loading branch information
erikgrinaker committed Jun 6, 2022
1 parent f0372fd commit bc0e4e5
Show file tree
Hide file tree
Showing 8 changed files with 720 additions and 145 deletions.
80 changes: 44 additions & 36 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
Expand Down Expand Up @@ -630,27 +631,26 @@ type Writer interface {
// returns.
ApplyBatchRepr(repr []byte, sync bool) error

// ClearMVCC removes the item from the db with the given MVCCKey. It
// requires that the timestamp is non-empty (see
// {ClearUnversioned,ClearIntent} if the timestamp is empty). Note that
// clear actually removes entries from the storage engine, rather than
// inserting MVCC tombstones.
// ClearMVCC removes the point key with the given MVCCKey from the db. It does
// not affect range keys. It requires that the timestamp is non-empty (see
// ClearUnversioned or ClearIntent if the timestamp is empty). Note that clear
// actually removes entries from the storage engine, rather than inserting
// MVCC tombstones.
//
// It is safe to modify the contents of the arguments after it returns.
ClearMVCC(key MVCCKey) error
// ClearUnversioned removes an unversioned item from the db. It is for use
// with inline metadata (not intents) and other unversioned keys (like
// Range-ID local keys).
// Range-ID local keys). It does not affect range keys.
//
// It is safe to modify the contents of the arguments after it returns.
ClearUnversioned(key roachpb.Key) error
// ClearIntent removes an intent from the db. Unlike
// {ClearMVCC,ClearUnversioned} this is a higher-level method that may make
// changes in parts of the key space that are not only a function of the
// input, and may choose to use a single-clear under the covers.
// txnDidNotUpdateMeta allows for performance optimization when set to true,
// and has semantics defined in MVCCMetadata.TxnDidNotUpdateMeta (it can
// be conservatively set to false).
// ClearIntent removes an intent from the db. Unlike ClearMVCC and
// ClearUnversioned, this is a higher-level method that may make changes in
// parts of the key space that are not only a function of the input, and may
// choose to use a single-clear under the covers. txnDidNotUpdateMeta allows
// for performance optimization when set to true, and has semantics defined in
// MVCCMetadata.TxnDidNotUpdateMeta (it can be conservatively set to false).
//
// It is safe to modify the contents of the arguments after it returns.
//
Expand All @@ -660,44 +660,46 @@ type Writer interface {
// decrease, we can stop tracking txnDidNotUpdateMeta and still optimize
// ClearIntent by always doing single-clear.
ClearIntent(key roachpb.Key, txnDidNotUpdateMeta bool, txnUUID uuid.UUID) error
// ClearEngineKey removes the item from the db with the given EngineKey.
// Note that clear actually removes entries from the storage engine. This is
// a general-purpose and low-level method that should be used sparingly,
// only when the other Clear* methods are not applicable.
// ClearEngineKey removes the given point key from the engine. It does not
// affect range keys. Note that clear actually removes entries from the
// storage engine. This is a general-purpose and low-level method that should
// be used sparingly, only when the other Clear* methods are not applicable.
//
// It is safe to modify the contents of the arguments after it returns.
ClearEngineKey(key EngineKey) error

// ClearRawRange removes a set of entries from start (inclusive) to end
// (exclusive) using a Pebble range tombstone. It can be applied to a range
// consisting of MVCCKeys or the more general EngineKeys -- it simply uses the
// roachpb.Key parameters as the Key field of an EngineKey.
// ClearRawRange removes both point keys and range keys from start (inclusive)
// to end (exclusive) using a Pebble range tombstone. It can be applied to a
// range consisting of MVCCKeys or the more general EngineKeys -- it simply
// uses the roachpb.Key parameters as the Key field of an EngineKey. This
// implies that it does not clear intents unless the intent lock table is
// targeted explicitly.
//
// Similar to the other Clear* methods, this method actually removes entries
// from the storage engine. It is safe to modify the contents of the arguments
// after it returns.
ClearRawRange(start, end roachpb.Key) error
// ClearMVCCRange removes MVCC keys from start (inclusive) to end (exclusive)
// using a Pebble range tombstone. It will remove everything in the span,
// including intents.
// including intents and range keys.
//
// Similar to the other Clear* methods, this method actually removes entries
// from the storage engine. It is safe to modify the contents of the arguments
// after it returns.
ClearMVCCRange(start, end roachpb.Key) error
// ClearMVCCVersions removes MVCC versions from start (inclusive) to end
// (exclusive) using a Pebble range tombstone. It is meant for efficiently
// ClearMVCCVersions removes MVCC point key versions from start (inclusive) to
// end (exclusive) using a Pebble range tombstone. It is meant for efficiently
// clearing a subset of versions of a key, since the parameters are MVCCKeys
// and not roachpb.Keys, but it can also be used across multiple keys. It will
// ignore intents, leaving them in place.
// ignore intents and range keys, leaving them in place.
//
// Similar to the other Clear* methods, this method actually removes entries
// from the storage engine. It is safe to modify the contents of the arguments
// after it returns.
ClearMVCCVersions(start, end MVCCKey) error
// ClearMVCCIteratorRange removes all keys in the given span using an MVCC
// iterator, by clearing individual keys (including intents) with Pebble point
// tombstones.
// tombstones. It will also clear all range keys in the span.
//
// Similar to the other Clear* methods, this method actually removes entries
// from the storage engine. It is safe to modify the contents of the arguments
Expand All @@ -724,7 +726,10 @@ type Writer interface {
// boundaries will be cleared. Clears are idempotent.
//
// This method is primarily intended for MVCC garbage collection and similar
// internal use.
// internal use. It will do an internal scan across the span first to check
// whether it contains any range keys at all, and clear the smallest single
// span that covers all range keys (if any), to avoid dropping Pebble range
// tombstones across unnecessary spans.
//
// This method is EXPERIMENTAL: range keys are under active development, and
// have severe limitations including being ignored by all KV and MVCC APIs and
Expand Down Expand Up @@ -1210,9 +1215,10 @@ func WriteSyncNoop(eng Engine) error {
}

// ClearRangeWithHeuristic clears the keys from start (inclusive) to end
// (exclusive). Depending on the number of keys, it will either use ClearRawRange
// or clear individual keys. It works with EngineKeys, so don't expect it to
// find and clear separated intents if [start, end) refers to MVCC key space.
// (exclusive), including any range keys. Depending on the number of keys, it
// will either use ClearRawRange or clear individual keys. It works with
// EngineKeys, so don't expect it to find and clear separated intents if
// [start,end) refers to MVCC key space.
func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Key) error {
iter := reader.NewEngineIterator(IterOptions{UpperBound: end})
defer iter.Close()
Expand All @@ -1225,7 +1231,9 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke
// perhaps we should fix Pebble to handle large numbers of tombstones in an
// sstable better. Note that we are referring to storage-level tombstones here,
// and not MVCC tombstones.
const clearRangeMinKeys = 64
clearRangeMinKeys := util.ConstantWithMetamorphicTestRange(
"clear-range-with-heuristic-minkeys", 64, 1, 64)

// Peek into the range to see whether it's large enough to justify
// ClearRawRange. Note that the work done here is bounded by
// clearRangeMinKeys, so it will be fairly cheap even for large
Expand All @@ -1239,16 +1247,13 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke
for valid {
count++
if count > clearRangeMinKeys {
break
return writer.ClearRawRange(start, end)
}
valid, err = iter.NextEngineKey()
}
if err != nil {
return err
}
if count > clearRangeMinKeys {
return writer.ClearRawRange(start, end)
}
valid, err = iter.SeekEngineKeyGE(EngineKey{Key: start})
for valid {
var k EngineKey
Expand All @@ -1260,7 +1265,10 @@ func ClearRangeWithHeuristic(reader Reader, writer Writer, start, end roachpb.Ke
}
valid, err = iter.NextEngineKey()
}
return err
if err != nil {
return err
}
return writer.ExperimentalClearAllMVCCRangeKeys(start, end)
}

var ingestDelayL0Threshold = settings.RegisterIntSetting(
Expand Down
Loading

0 comments on commit bc0e4e5

Please sign in to comment.