Skip to content

Commit

Permalink
Merge #85505 #85522
Browse files Browse the repository at this point in the history
85505: gossip: provide online method to clear leaked gossip infos r=knz a=nvanbenschoten

Fixes #85013.
Needed (in v21.2.X) for cockroachlabs/support#1709.

This commit introduces a new `crdb_internal.unsafe_clear_gossip_info` builtin
function which allows admin users to manually clear info objects from the
cluster's gossip network. The function does so by re-gossiping an identical
value for the specified key but with a TTL that is long enough to reasonably
ensure full propagation to all nodes in the cluster but short enough to expire
quickly once propagated.

The function is best-effort. It is possible for the info object with the low
TTL to fail to reach full propagation before reaching its TTL. For instance,
this is possible during a transient network partition. The effect of this is
that the existing gossip info object with a higher (or no) TTL would remain
in the gossip network on some nodes and may eventually propagate back out to
other nodes once the partition heals.

`@knz:` I'm assigning this to you for a review both because you're as good a
person as any to look at gossip-related changes, and because limited SQL
access to the cluster's gossip network is a nuanced subject in the
context of multi-tenancy.

Release note: None

85522: storage: optimize `DeleteRange` when deleting entire Raft range r=aliher1911 a=erikgrinaker

This patch adds a fast path for `DeleteRange` when deleting an entire
Raft range, by simply marking all live data as deleted in MVCC stats
instead of scanning across all point keys. It will still perform a
time-bound scan to look for conflicts with newer writes, and a range key
scan to take range key fragmentation into account for stats.

There are no behavioral changes. The fast path is therefore tested
comprehensively by adding a metamorphic parameter for it in
`TestMVCCHistories`.

Benchmarks confirm that the fast path is ~constant, while the slow path
is asymptotically linear.

```
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=1000/valueSize=64/entireRange=false-24         	     499	   2319384 ns/op	  48.29 MB/s
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=1000/valueSize=64/entireRange=true-24          	     577	   1965157 ns/op	  56.99 MB/s
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=10000/valueSize=64/entireRange=false-24        	     216	   5531790 ns/op	 202.47 MB/s
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=10000/valueSize=64/entireRange=true-24         	     576	   2014470 ns/op	 555.98 MB/s
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=100000/valueSize=64/entireRange=false-24       	      32	  37814215 ns/op	 296.18 MB/s
BenchmarkMVCCDeleteRangeUsingTombstone_Pebble/numKeys=100000/valueSize=64/entireRange=true-24        	     589	   2022481 ns/op	5537.75 MB/s
```

Resolves #83696.

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
  • Loading branch information
3 people committed Aug 8, 2022
3 parents 773332f + dfcb6ef + 85146a8 commit 49494e5
Show file tree
Hide file tree
Showing 26 changed files with 326 additions and 35 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3138,6 +3138,8 @@ table. Returns an error if validation fails.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="crdb_internal.trim_tenant_prefix"></a><code>crdb_internal.trim_tenant_prefix(keys: <a href="bytes.html">bytes</a>[]) &rarr; <a href="bytes.html">bytes</a>[]</code></td><td><span class="funcdesc"><p>This function assumes the given bytes are a CockroachDB key and trims any tenant prefix from the key.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="crdb_internal.unsafe_clear_gossip_info"></a><code>crdb_internal.unsafe_clear_gossip_info(key: <a href="string.html">string</a>) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used only by CockroachDB’s developers for testing purposes.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.validate_session_revival_token"></a><code>crdb_internal.validate_session_revival_token(token: <a href="bytes.html">bytes</a>) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>Validate a token that was created by create_session_revival_token. Intended for testing.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.validate_ttl_scheduled_jobs"></a><code>crdb_internal.validate_ttl_scheduled_jobs() &rarr; void</code></td><td><span class="funcdesc"><p>Validate all TTL tables have a valid scheduled job attached.</p>
Expand Down
37 changes: 37 additions & 0 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -1071,6 +1071,43 @@ func (g *Gossip) GetInfoProto(key string, msg protoutil.Message) error {
return protoutil.Unmarshal(bytes, msg)
}

// TryClearInfo attempts to clear an info object from the cluster's gossip
// network. It does so by retrieving the object with the corresponding key. If
// one does not exist, there's nothing to do and the method returns false.
// Otherwise, the method re-gossips the same key-value pair with a TTL that is
// long enough to reasonably ensure full propagation to all nodes in the cluster
// but short enough to expire quickly once propagated.
//
// The method is best-effort. It is possible for the info object with the low
// TTL to fail to reach full propagation before reaching its TTL. For instance,
// this is possible during a transient network partition. The effect of this is
// that the existing gossip info object with a higher (or no) TTL would remain
// in the gossip network on some nodes and may eventually propagate back out to
// other nodes once the partition heals.
func (g *Gossip) TryClearInfo(key string) (bool, error) {
// Long enough to propagate to all nodes, short enough to expire quickly.
const ttl = 1 * time.Minute
return g.tryClearInfoWithTTL(key, ttl)
}

func (g *Gossip) tryClearInfoWithTTL(key string, ttl time.Duration) (bool, error) {
val, err := g.GetInfo(key)
if err != nil {
if errors.HasType(err, KeyNotPresentError{}) {
// Info object not known on this node. We can't force a deletion
// preemptively, e.g. with a poison entry, because we do not have a valid
// value object to populate and consumers may make assumptions about the
// format of the value.
return false, nil
}
return false, err
}
if err := g.AddInfo(key, val, ttl); err != nil {
return false, err
}
return true, nil
}

// InfoOriginatedHere returns true iff the latest info for the provided key
// originated on this node. This is useful for ensuring that the system config
// is regossiped as soon as possible when its lease changes hands.
Expand Down
20 changes: 20 additions & 0 deletions pkg/gossip/gossip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -848,6 +848,26 @@ func TestGossipPropagation(t *testing.T) {
}
return nil
})

mustClear := func(g *Gossip, key string) {
if _, err := g.tryClearInfoWithTTL(key, 3*time.Second); err != nil {
t.Fatal(err)
}
}

// Clear both entries. Verify that both are removed from the gossip network.
mustClear(local, "local")
mustClear(remote, "remote")
testutils.SucceedsSoon(t, func() error {
for gName, g := range map[string]*Gossip{"local": local, "remote": remote} {
for _, key := range []string{"local", "remote"} {
if getInfo(g, key) != nil {
return fmt.Errorf("%s info %q not cleared", gName, key)
}
}
}
return nil
})
}

// Test whether propagation of an info that was generated by a prior
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -848,7 +848,7 @@ func TestEvalAddSSTable(t *testing.T) {
require.NoError(t, err)
require.True(t, v.IsTombstone(), "MVCC range keys must be tombstones")
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(
ctx, b, nil, kv.RangeKey.StartKey, kv.RangeKey.EndKey, kv.RangeKey.Timestamp, v.MVCCValueHeader.LocalTimestamp, nil, nil, 0))
ctx, b, nil, kv.RangeKey.StartKey, kv.RangeKey.EndKey, kv.RangeKey.Timestamp, v.MVCCValueHeader.LocalTimestamp, nil, nil, 0, nil))
default:
t.Fatalf("unknown KV type %T", kv)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ func TestCmdClearRange(t *testing.T) {
for _, rk := range rangeTombstones {
localTS := hlc.ClockTimestamp{WallTime: rk.Timestamp.WallTime - 1e9} // give range key a value if > 0
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(
ctx, eng, nil, rk.StartKey, rk.EndKey, rk.Timestamp, localTS, nil, nil, 0))
ctx, eng, nil, rk.StartKey, rk.EndKey, rk.Timestamp, localTS, nil, nil, 0, nil))
}

// Write some random point keys within the cleared span, above the range tombstones.
Expand Down
15 changes: 13 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -93,10 +94,20 @@ func DeleteRange(
args.Key, args.EndKey, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey())
maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV)

// If no predicate parameters are passed, use the fast path. If we're
// deleting the entire Raft range, use an even faster path that avoids a
// point key scan to update MVCC stats.
if args.Predicates == (roachpb.DeleteRangePredicates{}) {
// If no predicate parameters are passed, use the fast path.
var statsCovered *enginepb.MVCCStats
if args.Key.Equal(desc.StartKey.AsRawKey()) && args.EndKey.Equal(desc.EndKey.AsRawKey()) {
// NB: We take the fast path even if stats are estimates, because the
// slow path will likely end up with similarly poor stats anyway.
s := cArgs.EvalCtx.GetMVCCStats()
statsCovered = &s
}
err := storage.MVCCDeleteRangeUsingTombstone(ctx, readWriter, cArgs.Stats,
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents)
args.Key, args.EndKey, h.Timestamp, cArgs.Now, leftPeekBound, rightPeekBound, maxIntents,
statsCovered)
return result.Result{}, err
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_delete_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,9 @@ func TestDeleteRangeTombstone(t *testing.T) {
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2e9}, localTS, roachpb.MakeValueFromString("d2"), nil))
require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3e9}, localTS, nil))
require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5e9}, localTS, roachpb.MakeValueFromString("i5"), &txn))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, 0))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3e9}, localTS, nil, nil, 0, nil))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("Z"), roachpb.Key("a"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0, nil))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("z"), roachpb.Key("|"), hlc.Timestamp{WallTime: 100e9}, localTS, nil, nil, 0, nil))
}

now := hlc.ClockTimestamp{Logical: 9}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestRefreshRange(t *testing.T) {
require.NoError(t, storage.MVCCPut(
ctx, eng, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, roachpb.Value{}, nil))
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(
ctx, eng, nil, roachpb.Key("d"), roachpb.Key("f"), hlc.Timestamp{WallTime: 7}, hlc.ClockTimestamp{}, nil, nil, 0))
ctx, eng, nil, roachpb.Key("d"), roachpb.Key("f"), hlc.Timestamp{WallTime: 7}, hlc.ClockTimestamp{}, nil, nil, 0, nil))

testcases := map[string]struct {
start, end string
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/data_distribution_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func (ds dataDistribution) setupTest(
"invalid test data, range can't be used together with value: key=%s, rangeKey=%s",
kv.Key.String(), rangeKey.String())
err := storage.MVCCDeleteRangeUsingTombstone(ctx, eng, &ms, rangeKey.StartKey,
rangeKey.EndKey, rangeKey.Timestamp, hlc.ClockTimestamp{}, nil, nil, 1)
rangeKey.EndKey, rangeKey.Timestamp, hlc.ClockTimestamp{}, nil, nil, 1, nil)
require.NoError(t, err, "failed to put delete range")
} else if txn == nil {
if kv.Key.Timestamp.IsEmpty() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_consistency_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func TestReplicaChecksumSHA512(t *testing.T) {

if len(endKey) > 0 {
require.NoError(t, storage.MVCCDeleteRangeUsingTombstone(
ctx, eng, nil, key, endKey, ts, localTS, nil, nil, 0))
ctx, eng, nil, key, endKey, ts, localTS, nil, nil, 0, nil))
} else {
require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, ts, localTS, value, nil))
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ go_library(
"explain_vec.go",
"export.go",
"filter.go",
"gossip.go",
"grant_revoke.go",
"grant_revoke_system.go",
"grant_role.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2709,6 +2709,7 @@ func (ex *connExecutor) initEvalCtx(ctx context.Context, evalCtx *extendedEvalCo
Tenant: p,
Regions: p,
JoinTokenCreator: p,
Gossip: p,
PreparedStatementState: &ex.extraTxnState.prepStmtsNamespace,
SessionDataStack: ex.sessionDataStack,
ReCache: ex.server.reCache,
Expand Down
25 changes: 25 additions & 0 deletions pkg/sql/gossip.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql

import "context"

// TryClearGossipInfo implements the tree.GossipOperator interface.
func (p *planner) TryClearGossipInfo(ctx context.Context, key string) (bool, error) {
g, err := p.ExecCfg().Gossip.OptionalErr(0 /* issue */)
if err != nil {
return false, err
}
if err := p.RequireAdminRole(ctx, "try clear gossip info"); err != nil {
return false, err
}
return g.TryClearInfo(key)
}
6 changes: 6 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -1146,3 +1146,9 @@ query I
SELECT crdb_internal.num_inverted_index_entries(NULL::STRING, 0)
----
0

# Exercise unsafe gossip builtin functions.
query B
SELECT crdb_internal.unsafe_clear_gossip_info('unknown key')
----
false
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4634,7 +4634,7 @@ FROM pg_proc p
JOIN pg_type t ON t.typinput = p.oid
WHERE t.typname = '_int4'
----
2004 array_in array_in
2005 array_in array_in

## #16285
## int2vectors should be 0-indexed
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -385,6 +385,7 @@ func newInternalPlanner(
p.extendedEvalCtx.Tenant = p
p.extendedEvalCtx.Regions = p
p.extendedEvalCtx.JoinTokenCreator = p
p.extendedEvalCtx.Gossip = p
p.extendedEvalCtx.ClusterID = execCfg.NodeInfo.LogicalClusterID()
p.extendedEvalCtx.ClusterName = execCfg.RPCContext.ClusterName()
p.extendedEvalCtx.NodeID = execCfg.NodeInfo.NodeID
Expand Down
21 changes: 21 additions & 0 deletions pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -4988,6 +4988,27 @@ value if you rely on the HLC for accuracy.`,
},
),

"crdb_internal.unsafe_clear_gossip_info": makeBuiltin(
tree.FunctionProperties{Category: builtinconstants.CategorySystemInfo},
tree.Overload{
Types: tree.ArgTypes{{"key", types.String}},
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *eval.Context, args tree.Datums) (tree.Datum, error) {
key, ok := tree.AsDString(args[0])
if !ok {
return nil, errors.Newf("expected string value, got %T", args[0])
}
ok, err := ctx.Gossip.TryClearGossipInfo(ctx.Context, string(key))
if err != nil {
return nil, err
}
return tree.MakeDBool(tree.DBool(ok)), nil
},
Info: "This function is used only by CockroachDB's developers for testing purposes.",
Volatility: volatility.Volatile,
},
),

"crdb_internal.encode_key": makeBuiltin(
tree.FunctionProperties{Category: builtinconstants.CategorySystemInfo},
tree.Overload{
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/sem/eval/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,8 @@ type Context struct {

JoinTokenCreator JoinTokenCreator

Gossip GossipOperator

PreparedStatementState PreparedStatementState

// The transaction in which the statement is executing.
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/sem/eval/deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -531,6 +531,15 @@ type JoinTokenCreator interface {
CreateJoinToken(ctx context.Context) (string, error)
}

// GossipOperator is capable of manipulating the cluster's gossip network. The
// methods will return errors when run by any tenant other than the system
// tenant.
type GossipOperator interface {
// TryClearGossipInfo attempts to clear an info object from the cluster's
// gossip network.
TryClearGossipInfo(ctx context.Context, key string) (bool, error)
}

// SQLStatsController is an interface embedded in EvalCtx which can be used by
// the builtins to reset SQL stats in the cluster. This interface is introduced
// to avoid circular dependency.
Expand Down
18 changes: 18 additions & 0 deletions pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -361,6 +361,24 @@ func BenchmarkMVCCDeleteRange_Pebble(b *testing.B) {
}
}

func BenchmarkMVCCDeleteRangeUsingTombstone_Pebble(b *testing.B) {
skip.UnderShort(b)
ctx := context.Background()
for _, numKeys := range []int{1000, 10000, 100000} {
b.Run(fmt.Sprintf("numKeys=%d", numKeys), func(b *testing.B) {
for _, valueSize := range []int{64} {
b.Run(fmt.Sprintf("valueSize=%d", valueSize), func(b *testing.B) {
for _, entireRange := range []bool{false, true} {
b.Run(fmt.Sprintf("entireRange=%t", entireRange), func(b *testing.B) {
runMVCCDeleteRangeUsingTombstone(ctx, b, setupMVCCPebble, numKeys, valueSize, entireRange)
})
}
})
}
})
}
}

func BenchmarkClearMVCCVersions_Pebble(b *testing.B) {
skip.UnderShort(b)
ctx := context.Background()
Expand Down
69 changes: 68 additions & 1 deletion pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -718,7 +718,7 @@ func setupMVCCData(
startKey := roachpb.Key(encoding.EncodeUvarintAscending([]byte("key-"), uint64(start)))
endKey := roachpb.Key(encoding.EncodeUvarintAscending([]byte("key-"), uint64(end)))
require.NoError(b, MVCCDeleteRangeUsingTombstone(
ctx, batch, nil, startKey, endKey, ts, hlc.ClockTimestamp{}, nil, nil, 0))
ctx, batch, nil, startKey, endKey, ts, hlc.ClockTimestamp{}, nil, nil, 0, nil))
}
require.NoError(b, batch.Commit(false /* sync */))
batch.Close()
Expand Down Expand Up @@ -1306,6 +1306,73 @@ func runMVCCDeleteRange(ctx context.Context, b *testing.B, emk engineMaker, valu
}
}

func runMVCCDeleteRangeUsingTombstone(
ctx context.Context, b *testing.B, emk engineMaker, numKeys int, valueBytes int, entireRange bool,
) {
eng, dir := setupMVCCData(ctx, b, emk, benchDataOptions{
numVersions: 1,
numKeys: numKeys,
valueBytes: valueBytes,
})
require.NoError(b, eng.Compact())

var msCovered *enginepb.MVCCStats
var leftPeekBound, rightPeekBound roachpb.Key
if entireRange {
iter := eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{
KeyTypes: IterKeyTypePointsAndRanges,
LowerBound: keys.LocalMax,
UpperBound: keys.MaxKey,
})
ms, err := ComputeStatsForRange(iter, keys.LocalMax, keys.MaxKey, 0)
iter.Close()
require.NoError(b, err)

leftPeekBound = keys.LocalMax
rightPeekBound = keys.MaxKey
msCovered = &ms
}

eng.Close()

b.SetBytes(int64(numKeys) * int64(overhead+valueBytes))
b.StopTimer()
b.ResetTimer()

locDirty := dir + "_dirty"

for i := 0; i < b.N; i++ {
if err := os.RemoveAll(locDirty); err != nil {
b.Fatal(err)
}
if err := fileutil.CopyDir(dir, locDirty); err != nil {
b.Fatal(err)
}
func() {
eng := emk(b, locDirty)
defer eng.Close()

b.StartTimer()
if err := MVCCDeleteRangeUsingTombstone(
ctx,
eng,
&enginepb.MVCCStats{},
keys.LocalMax,
roachpb.KeyMax,
hlc.MaxTimestamp,
hlc.ClockTimestamp{},
leftPeekBound,
rightPeekBound,
0,
msCovered,
); err != nil {
b.Fatal(err)
}
b.StopTimer()
}()
}
}

func runClearRange(
ctx context.Context,
b *testing.B,
Expand Down
Loading

0 comments on commit 49494e5

Please sign in to comment.