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 automatically clears any range keys in the `Writer` methods
`ClearRawRange`, `ClearMVCCRange`, and `ClearMVCCIteratorRange`.

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.

Release note: None
  • Loading branch information
erikgrinaker committed May 2, 2022
1 parent 54b5232 commit ab2a563
Show file tree
Hide file tree
Showing 6 changed files with 372 additions and 65 deletions.
77 changes: 38 additions & 39 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -566,27 +566,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 from the db with the given MVCCKey. 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 @@ -596,29 +595,30 @@ 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 item from the db with the given EngineKey. 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 does not
//
// 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). 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. Similar to the other Clear* methods,
// this method actually removes entries from the storage engine.
// ClearRawRange removes entries from start (inclusive) to end (exclusive),
// including range keys but excluding any separated intents unless the
// separated lock table is explicitly included in the span. 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.
// 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),
// including intents. Similar to the other Clear* methods, this method
// actually removes entries from the storage engine.
// 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.
//
// TODO(erikgrinaker): This should clear range keys too.
ClearMVCCRange(start, end roachpb.Key) error
// ClearMVCCVersions removes MVCC point key versions from start (inclusive) to
// end (exclusive). It does not affect intents nor range keys. It is meant for
Expand All @@ -629,11 +629,9 @@ type Writer interface {
// 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 and clearing individual point keys (including intents). Similar to
// the other Clear* methods, this method actually removes entries from the
// storage engine.
//
// TODO(erikgrinaker): This should clear range keys too.
// iterator and clearing individual point keys including intents. Any range
// keys in the span are also cleared. Similar to the other Clear* methods,
// this method actually removes entries from the storage engine.
ClearMVCCIteratorRange(start, end roachpb.Key) error

// ExperimentalClearMVCCRangeKey deletes an MVCC range key from start
Expand Down Expand Up @@ -1077,9 +1075,10 @@ func WriteSyncNoop(ctx context.Context, 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 @@ -1106,16 +1105,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 @@ -1127,7 +1123,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
34 changes: 23 additions & 11 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ import (
// merge [ts=<int>[,<int>]] k=<key> v=<string> [raw]
//
// clear_range k=<key> end=<key>
// clear_rangekey k=<key> end=<key> ts=<int>[,<int>]
//
// Where `<key>` can be a simple string, or a string
// prefixed by the following characters:
Expand Down Expand Up @@ -471,17 +472,18 @@ var commands = map[string]cmd{
// TODO(nvanbenschoten): test "resolve_intent_range".
"check_intent": {typReadOnly, cmdCheckIntent},

"clear_range": {typDataUpdate, cmdClearRange},
"cput": {typDataUpdate, cmdCPut},
"del": {typDataUpdate, cmdDelete},
"del_range": {typDataUpdate, cmdDeleteRange},
"del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone},
"get": {typReadOnly, cmdGet},
"increment": {typDataUpdate, cmdIncrement},
"merge": {typDataUpdate, cmdMerge},
"put": {typDataUpdate, cmdPut},
"put_rangekey": {typDataUpdate, cmdPutRangeKey},
"scan": {typReadOnly, cmdScan},
"clear_range": {typDataUpdate, cmdClearRange},
"clear_rangekey": {typDataUpdate, cmdClearRangeKey},
"cput": {typDataUpdate, cmdCPut},
"del": {typDataUpdate, cmdDelete},
"del_range": {typDataUpdate, cmdDeleteRange},
"del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone},
"get": {typReadOnly, cmdGet},
"increment": {typDataUpdate, cmdIncrement},
"merge": {typDataUpdate, cmdMerge},
"put": {typDataUpdate, cmdPut},
"put_rangekey": {typDataUpdate, cmdPutRangeKey},
"scan": {typReadOnly, cmdScan},

"iter_new": {typReadOnly, cmdIterNew},
"iter_seek_ge": {typReadOnly, cmdIterSeekGE},
Expand Down Expand Up @@ -667,9 +669,19 @@ func cmdCheckIntent(e *evalCtx) error {

func cmdClearRange(e *evalCtx) error {
key, endKey := e.getKeyRange()
if util.ConstantWithMetamorphicTestBool("clear-range-using-iterator", false) {
return e.engine.ClearMVCCIteratorRange(key, endKey)
}
return e.engine.ClearMVCCRange(key, endKey)
}

func cmdClearRangeKey(e *evalCtx) error {
key, endKey := e.getKeyRange()
ts := e.getTs(nil)
return e.engine.ExperimentalClearMVCCRangeKey(
MVCCRangeKey{StartKey: key, EndKey: endKey, Timestamp: ts})
}

func cmdCPut(e *evalCtx) error {
txn := e.getTxn(optional)
ts := e.getTs(txn)
Expand Down
15 changes: 7 additions & 8 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -1210,7 +1210,12 @@ func (p *Pebble) SingleClearEngineKey(key EngineKey) error {

// ClearRawRange implements the Engine interface.
func (p *Pebble) ClearRawRange(start, end roachpb.Key) error {
return p.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end})
startBuf := EncodeMVCCKey(MVCCKey{Key: start})
endBuf := EncodeMVCCKey(MVCCKey{Key: end})
if err := p.db.DeleteRange(startBuf, endBuf, pebble.Sync); err != nil {
return err
}
return p.ExperimentalClearAllMVCCRangeKeys(start, end)
}

// ClearMVCCRange implements the Engine interface.
Expand All @@ -1221,13 +1226,7 @@ func (p *Pebble) ClearMVCCRange(start, end roachpb.Key) error {

// ClearMVCCVersions implements the Engine interface.
func (p *Pebble) ClearMVCCVersions(start, end MVCCKey) error {
return p.clearRange(start, end)
}

func (p *Pebble) clearRange(start, end MVCCKey) error {
bufStart := EncodeMVCCKey(start)
bufEnd := EncodeMVCCKey(end)
return p.db.DeleteRange(bufStart, bufEnd, pebble.Sync)
return p.db.DeleteRange(EncodeMVCCKey(start), EncodeMVCCKey(end), pebble.Sync)
}

// ClearMVCCIteratorRange implements the Engine interface.
Expand Down
13 changes: 7 additions & 6 deletions pkg/storage/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -363,7 +363,12 @@ func (p *pebbleBatch) SingleClearEngineKey(key EngineKey) error {

// ClearRawRange implements the Batch interface.
func (p *pebbleBatch) ClearRawRange(start, end roachpb.Key) error {
return p.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end})
p.buf = EncodeMVCCKeyToBuf(p.buf[:0], MVCCKey{Key: start})
endBuf := EncodeMVCCKey(MVCCKey{Key: end})
if err := p.batch.DeleteRange(p.buf, endBuf, nil); err != nil {
return err
}
return p.ExperimentalClearAllMVCCRangeKeys(start, end)
}

// ClearMVCCRange implements the Batch interface.
Expand All @@ -375,10 +380,6 @@ func (p *pebbleBatch) ClearMVCCRange(start, end roachpb.Key) error {

// ClearMVCCVersions implements the Batch interface.
func (p *pebbleBatch) ClearMVCCVersions(start, end MVCCKey) error {
return p.clearRange(start, end)
}

func (p *pebbleBatch) clearRange(start, end MVCCKey) error {
p.buf = EncodeMVCCKeyToBuf(p.buf[:0], start)
buf2 := EncodeMVCCKey(end)
return p.batch.DeleteRange(p.buf, buf2, nil)
Expand All @@ -405,7 +406,7 @@ func (p *pebbleBatch) ClearMVCCIteratorRange(start, end roachpb.Key) error {
return err
}
}
return nil
return p.ExperimentalClearAllMVCCRangeKeys(start, end)
}

// ExperimentalClearMVCCRangeKey implements the Engine interface.
Expand Down
6 changes: 5 additions & 1 deletion pkg/storage/sst_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,8 @@ func (fw *SSTWriter) Finish() error {
}

// ClearRawRange implements the Writer interface.
//
// TODO(erikgrinaker): This must clear range keys when SSTs support them.
func (fw *SSTWriter) ClearRawRange(start, end roachpb.Key) error {
return fw.clearRange(MVCCKey{Key: start}, MVCCKey{Key: end})
}
Expand All @@ -153,8 +155,10 @@ func (fw *SSTWriter) ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error
}

// ExperimentalClearAllMVCCRangeKeys implements the Writer interface.
//
// TODO(erikgrinaker): This must clear range keys when SSTs support them.
func (fw *SSTWriter) ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) error {
panic("not implemented")
return nil
}

func (fw *SSTWriter) clearRange(start, end MVCCKey) error {
Expand Down
Loading

0 comments on commit ab2a563

Please sign in to comment.