Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,9 +150,9 @@ func ClearRange(
// If we're writing Pebble range tombstones, use ClearRangeWithHeuristic to
// avoid writing tombstones across empty spans -- in particular, across the
// range key span, since we expect range keys to be rare.
const pointKeyThreshold, rangeKeyThreshold = 2, 2
const pointKeyThreshold = 2
if err := storage.ClearRangeWithHeuristic(
ctx, readWriter, readWriter, from, to, pointKeyThreshold, rangeKeyThreshold,
ctx, readWriter, readWriter, from, to, pointKeyThreshold,
); err != nil {
return result.Result{}, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4023,7 +4023,7 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) {
EndKey: roachpb.RKey(keyEnd),
}
if err := storage.ClearRangeWithHeuristic(
ctx, receivingEng, &sst, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), 64, 8,
ctx, receivingEng, &sst, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), 64,
); err != nil {
return err
}
Expand Down
11 changes: 3 additions & 8 deletions pkg/kv/kvserver/kvstorage/destroy.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,6 @@ const (
// perhaps we should fix Pebble to handle large numbers of range tombstones in
// an sstable better.
ClearRangeThresholdPointKeys = 64

// ClearRangeThresholdRangeKeys is the threshold (as number of range keys)
// beyond which we'll clear range data using a single RANGEKEYDEL across the
// span rather than clearing individual range keys.
ClearRangeThresholdRangeKeys = 8
)

// ClearRangeDataOptions specify which parts of a Replica are to be destroyed.
Expand Down Expand Up @@ -78,14 +73,14 @@ func ClearRangeData(
UnreplicatedByRangeID: opts.ClearUnreplicatedByRangeID,
})

pointKeyThreshold, rangeKeyThreshold := ClearRangeThresholdPointKeys, ClearRangeThresholdRangeKeys
pointKeyThreshold := ClearRangeThresholdPointKeys
if opts.MustUseClearRange {
pointKeyThreshold, rangeKeyThreshold = 1, 1
pointKeyThreshold = 1
}

for _, keySpan := range keySpans {
if err := storage.ClearRangeWithHeuristic(
ctx, reader, writer, keySpan.Key, keySpan.EndKey, pointKeyThreshold, rangeKeyThreshold,
ctx, reader, writer, keySpan.Key, keySpan.EndKey, pointKeyThreshold,
); err != nil {
return err
}
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/snapshot_apply_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,6 @@ func clearSubsumedReplicaDiskData(
keySpans[i].EndKey,
totalKeySpans[i].EndKey,
kvstorage.ClearRangeThresholdPointKeys,
kvstorage.ClearRangeThresholdRangeKeys,
); err != nil {
subsumedReplSST.Close()
return nil, err
Expand Down
84 changes: 30 additions & 54 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -1589,24 +1589,22 @@ func WriteSyncNoop(eng Engine) error {
// either write a Pebble range tombstone or clear individual keys. If it uses
// a range tombstone, it will tighten the span to the first encountered key.
//
// pointKeyThreshold and rangeKeyThreshold specify the number of point/range
// keys respectively where it will switch from clearing individual keys to
// Pebble range tombstones (RANGEDEL or RANGEKEYDEL respectively). A threshold
// of 0 disables checking for and clearing that key type.
// The pointKeyThreshold parameter specifies the number of point keys where it
// will switch from clearing individual keys using point tombstones to clearing
// the entire range using Pebble range tombstones (RANGEDELs). The
// pointKeyThreshold value must be at least 1. NB: An initial scan will be done
// to determine the type of clear, so a large threshold will potentially involve
// scanning a large number of keys.
//
// NB: An initial scan will be done to determine the type of clear, so a large
// threshold will potentially involve scanning a large number of keys twice.
//
// TODO(erikgrinaker): Consider tightening the end of the range tombstone span
// too, by doing a SeekLT when we reach the threshold. It's unclear whether it's
// really worth it.
// ClearRangeWithHeuristic will also check for the existence of range keys, and
// if any exist, it will write a RANGEKEYDEL clearing all range keys in the span.
func ClearRangeWithHeuristic(
ctx context.Context,
r Reader,
w Writer,
start, end roachpb.Key,
pointKeyThreshold, rangeKeyThreshold int,
ctx context.Context, r Reader, w Writer, start, end roachpb.Key, pointKeyThreshold int,
) error {
if pointKeyThreshold < 1 {
return errors.AssertionFailedf("pointKeyThreshold must be at least 1")
}

clearPointKeys := func(r Reader, w Writer, start, end roachpb.Key, threshold int) error {
iter, err := r.NewEngineIterator(ctx, IterOptions{
KeyTypes: IterKeyTypePointsOnly,
Expand Down Expand Up @@ -1655,7 +1653,7 @@ func ClearRangeWithHeuristic(
return err
}

clearRangeKeys := func(r Reader, w Writer, start, end roachpb.Key, threshold int) error {
clearRangeKeys := func(r Reader, w Writer, start, end roachpb.Key) error {
iter, err := r.NewEngineIterator(ctx, IterOptions{
KeyTypes: IterKeyTypeRangesOnly,
LowerBound: start,
Expand All @@ -1666,51 +1664,29 @@ func ClearRangeWithHeuristic(
}
defer iter.Close()

// Scan, and drop a RANGEKEYDEL if we reach the threshold.
var ok bool
var count int
var firstKey roachpb.Key
for ok, err = iter.SeekEngineKeyGE(EngineKey{Key: start}); ok; ok, err = iter.NextEngineKey() {
count += len(iter.EngineRangeKeys())
if len(firstKey) == 0 {
bounds, err := iter.EngineRangeBounds()
if err != nil {
return err
}
firstKey = bounds.Key.Clone()
}
if count >= threshold {
return w.ClearRawRange(firstKey, end, false /* pointKeys */, true /* rangeKeys */)
}
}
if err != nil || count == 0 {
ok, err := iter.SeekEngineKeyGE(EngineKey{Key: start})
if err != nil {
return err
}
// Clear individual range keys.
for ok, err = iter.SeekEngineKeyGE(EngineKey{Key: start}); ok; ok, err = iter.NextEngineKey() {
bounds, err := iter.EngineRangeBounds()
if err != nil {
return err
}
for _, v := range iter.EngineRangeKeys() {
if err := w.ClearEngineRangeKey(bounds.Key, bounds.EndKey, v.Version); err != nil {
return err
}
}
if !ok {
// No range keys in the span.
return nil
}
return err
}

if pointKeyThreshold > 0 {
if err := clearPointKeys(r, w, start, end, pointKeyThreshold); err != nil {
bounds, err := iter.EngineRangeBounds()
if err != nil {
return err
}
// TODO(erikgrinaker): Consider tightening the end of the range
// tombstone span too, by doing a SeekLT when we reach the threshold.
// It's unclear whether it's really worth it.
return w.ClearRawRange(bounds.Key, end, false /* pointKeys */, true /* rangeKeys */)
}

if rangeKeyThreshold > 0 {
if err := clearRangeKeys(r, w, start, end, rangeKeyThreshold); err != nil {
return err
}
if err := clearPointKeys(r, w, start, end, pointKeyThreshold); err != nil {
return err
}
if err := clearRangeKeys(r, w, start, end); err != nil {
return err
}

return nil
Expand Down
30 changes: 3 additions & 27 deletions pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1813,49 +1813,25 @@ func TestEngineClearRange(t *testing.T) {

"ClearRangeWithHeuristic individual": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, math.MaxInt, math.MaxInt)
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, math.MaxInt)
},
clearsPointKeys: true,
clearsRangeKeys: true,
clearsIntents: false,
},
"ClearRangeWithHeuristic ranged": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, 1, 1)
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, 1)
},
clearsPointKeys: true,
clearsRangeKeys: true,
clearsIntents: false,
},
"ClearRangeWithHeuristic point keys individual": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, math.MaxInt, 0)
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, math.MaxInt)
},
clearsPointKeys: true,
clearsRangeKeys: false,
clearsIntents: false,
},
"ClearRangeWithHeuristic point keys ranged": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, 1, 0)
},
clearsPointKeys: true,
clearsRangeKeys: false,
clearsIntents: false,
},
"ClearRangeWithHeuristic range keys individual": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, 0, math.MaxInt)
},
clearsPointKeys: false,
clearsRangeKeys: true,
clearsIntents: false,
},
"ClearRangeWithHeuristic range keys ranged": {
clearRange: func(rw ReadWriter, start, end roachpb.Key) error {
return ClearRangeWithHeuristic(ctx, rw, rw, start, end, 0, 1)
},
clearsPointKeys: false,
clearsRangeKeys: true,
clearsIntents: false,
},
Expand Down
Loading