diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 98c7b471a61f..f284425a0709 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -677,7 +677,7 @@ func (ds *DistSender) initAndVerifyBatch( switch inner.(type) { case *roachpb.ScanRequest, *roachpb.ResolveIntentRangeRequest, *roachpb.DeleteRangeRequest, *roachpb.RevertRangeRequest, - *roachpb.ExportRequest, *roachpb.QueryLocksRequest: + *roachpb.ExportRequest, *roachpb.QueryLocksRequest, *roachpb.IsSpanEmptyRequest: // Accepted forward range requests. foundForward = true diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 426210306029..400bea2b7d69 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "cmd_heartbeat_txn.go", "cmd_increment.go", "cmd_init_put.go", + "cmd_is_span_empty.go", "cmd_lease.go", "cmd_lease_info.go", "cmd_lease_request.go", @@ -107,6 +108,7 @@ go_test( "cmd_end_transaction_test.go", "cmd_export_test.go", "cmd_get_test.go", + "cmd_is_span_empty_test.go", "cmd_lease_test.go", "cmd_query_intent_test.go", "cmd_query_resolved_timestamp_test.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go new file mode 100644 index 000000000000..c2bbfa19ae1c --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty.go @@ -0,0 +1,48 @@ +// 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 batcheval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +func init() { + RegisterReadOnlyCommand(roachpb.IsSpanEmpty, DefaultDeclareKeys, IsSpanEmpty) +} + +// IsSpanEmpty determines whether there are any keys in the key span requested +// at any time. If there are any keys, the response header will have a NumKeys +// value of 1. +func IsSpanEmpty( + ctx context.Context, reader storage.Reader, cArgs CommandArgs, resp roachpb.Response, +) (result.Result, error) { + args := cArgs.Args.(*roachpb.IsSpanEmptyRequest) + reply := resp.(*roachpb.IsSpanEmptyResponse) + isEmpty, err := storage.MVCCIsSpanEmpty(ctx, reader, storage.MVCCIsSpanEmptyOptions{ + StartKey: args.Key, + EndKey: args.EndKey, + StartTS: hlc.MinTimestamp, // beginning of time + EndTS: hlc.MaxTimestamp, // end of time + }) + if err != nil { + return result.Result{}, errors.Wrap(err, "IsSpanEmpty") + } + if !isEmpty { + reply.NumKeys++ + } + return result.Result{}, nil +} diff --git a/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go new file mode 100644 index 000000000000..bb571a80267a --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_is_span_empty_test.go @@ -0,0 +1,97 @@ +// 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 batcheval_test + +import ( + "context" + "sync/atomic" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestIsSpanEmpty(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + var sentIsSpanEmptyRequests int64 + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + TestingRequestFilter: func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { + if _, exists := request.GetArg(roachpb.IsSpanEmpty); exists { + atomic.AddInt64(&sentIsSpanEmptyRequests, 1) + } + return nil + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + kvDB := tc.Server(0).DB() + scratchKey := tc.ScratchRange(t) + mkKey := func(suffix string) roachpb.Key { + return append(scratchKey[:len(scratchKey):len(scratchKey)], suffix...) + } + + checkIsEmpty := func(t *testing.T, exp bool, from, to roachpb.Key) { + var ba kv.Batch + ba.Header.MaxSpanRequestKeys = 1 + ba.AddRawRequest(&roachpb.IsSpanEmptyRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: from, EndKey: to, + }, + }) + require.NoError(t, kvDB.Run(ctx, &ba)) + require.Equal(t, exp, ba.RawResponse().Responses[0].GetIsSpanEmpty().IsEmpty()) + } + + requireEmpty := func(t *testing.T, from, to roachpb.Key) { + checkIsEmpty(t, true, from, to) + } + requireNotEmpty := func(t *testing.T, from, to roachpb.Key) { + checkIsEmpty(t, false, from, to) + } + + requireEmpty(t, mkKey(""), mkKey("").PrefixEnd()) + + tc.SplitRangeOrFatal(t, mkKey("c")) + requireEmpty(t, mkKey(""), mkKey("").PrefixEnd()) + + require.NoError(t, kvDB.Put(ctx, mkKey("x"), "foo")) + requireEmpty(t, mkKey(""), mkKey("x")) + requireNotEmpty(t, mkKey(""), mkKey("").PrefixEnd()) + + require.NoError(t, kvDB.Del(ctx, mkKey("x"))) + requireEmpty(t, mkKey(""), mkKey("x")) + requireNotEmpty(t, mkKey(""), mkKey("").PrefixEnd()) + + // We want to make sure that the DistSender stops iterating ranges once + // the first range with any keys is found. + checkIsCalled := func(t *testing.T, expEmpty bool, delta int64, from, to roachpb.Key) { + before := atomic.LoadInt64(&sentIsSpanEmptyRequests) + checkIsEmpty(t, expEmpty, from, to) + require.Equal(t, delta, atomic.LoadInt64(&sentIsSpanEmptyRequests)-before) + } + checkIsCalled(t, false, 2, mkKey(""), mkKey("").PrefixEnd()) + tc.SplitRangeOrFatal(t, mkKey("a")) + tc.SplitRangeOrFatal(t, mkKey("b")) + checkIsCalled(t, false, 4, mkKey(""), mkKey("").PrefixEnd()) +} diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 8bc9e4a47b69..fce6515dfcde 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -550,6 +550,25 @@ func (r *QueryLocksResponse) combine(c combinable) error { var _ combinable = &QueryLocksResponse{} +// combine implements the combinable interface. +func (r *IsSpanEmptyResponse) combine(c combinable) error { + otherR := c.(*IsSpanEmptyResponse) + if r != nil { + if err := r.ResponseHeader.combine(otherR.Header()); err != nil { + return err + } + // Given the request doesn't actually count anything, and instead + // hijacks NumKeys to indicate whether there is any data, there's + // no good reason to have it take on a value greater than 1. + if r.ResponseHeader.NumKeys > 1 { + r.ResponseHeader.NumKeys = 1 + } + } + return nil +} + +var _ combinable = &IsSpanEmptyResponse{} + // Header implements the Request interface. func (rh RequestHeader) Header() RequestHeader { return rh @@ -787,6 +806,9 @@ func (*ScanInterleavedIntentsRequest) Method() Method { return ScanInterleavedIn // Method implements the Request interface. func (*BarrierRequest) Method() Method { return Barrier } +// Method implements the Request interface. +func (*IsSpanEmptyRequest) Method() Method { return IsSpanEmpty } + // ShallowCopy implements the Request interface. func (gr *GetRequest) ShallowCopy() Request { shallowCopy := *gr @@ -1075,6 +1097,12 @@ func (r *BarrierRequest) ShallowCopy() Request { return &shallowCopy } +// ShallowCopy implements the Request interface. +func (r *IsSpanEmptyRequest) ShallowCopy() Request { + shallowCopy := *r + return &shallowCopy +} + // NewGet returns a Request initialized to get the value at key. If // forUpdate is true, an unreplicated, exclusive lock is acquired on on // the key, if it exists. @@ -1454,6 +1482,7 @@ func (*QueryResolvedTimestampRequest) flags() flag { } func (*ScanInterleavedIntentsRequest) flags() flag { return isRead | isRange } func (*BarrierRequest) flags() flag { return isWrite | isRange } +func (*IsSpanEmptyRequest) flags() flag { return isRead | isRange } // IsParallelCommit returns whether the EndTxn request is attempting to perform // a parallel commit. See txn_interceptor_committer.go for a discussion about @@ -1657,6 +1686,12 @@ func (r *JoinNodeResponse) CreateStoreIdent() (StoreIdent, error) { return sIdent, nil } +// IsEmpty returns true if the NumKeys field of the ResponseHeader is 0, +// indicating that the span is empty. +func (r *IsSpanEmptyResponse) IsEmpty() bool { + return r.NumKeys == 0 +} + // SafeFormat implements redact.SafeFormatter. func (c *ContentionEvent) SafeFormat(w redact.SafePrinter, _ rune) { w.Printf("conflicted with %s on %s for %.3fs", c.TxnMeta.ID, c.Key, c.Duration.Seconds()) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 90a9c8e67380..5a4cf28bf1de 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -193,6 +193,25 @@ message ProbeResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; } +// IsSpanEmptyRequest is used to determine whether a span contains any keys, +// garbage or otherwise. It is used to determine whether data deleted by a +// DeleteRange tombstone has been fully removed. +// +// Generally, the caller should set the MaxSpanKeys header on the BatchRequest +// to 1 so that the DistSender will process the overlapping ranges sequentially +// and stop after the first non-empty range. +message IsSpanEmptyRequest { + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + +// IsSpanEmptyResponse is the response to an IsSpanEmptyRequest. +// If there is any data in the queried span, the NumKeys field of the +// ResponseHeader will have a positive value; if NumKeys is zero, then the +// span is empty. +message IsSpanEmptyResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + // A PutRequest is the argument to the Put() method. message PutRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; @@ -2144,6 +2163,7 @@ message RequestUnion { ScanInterleavedIntentsRequest scan_interleaved_intents = 52; BarrierRequest barrier = 53; ProbeRequest probe = 54; + IsSpanEmptyRequest is_span_empty = 56; } reserved 8, 15, 23, 25, 27, 31, 34; } @@ -2199,6 +2219,7 @@ message ResponseUnion { ScanInterleavedIntentsResponse scan_interleaved_intents = 52; BarrierResponse barrier = 53; ProbeResponse probe = 54; + IsSpanEmptyResponse is_span_empty = 56; } reserved 8, 15, 23, 25, 27, 28, 31, 34; } diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index c90b3f72b258..5bba1a23af0d 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -182,6 +182,8 @@ func (ru RequestUnion) GetInner() Request { return t.Barrier case *RequestUnion_Probe: return t.Probe + case *RequestUnion_IsSpanEmpty: + return t.IsSpanEmpty default: return nil } @@ -284,6 +286,8 @@ func (ru ResponseUnion) GetInner() Response { return t.Barrier case *ResponseUnion_Probe: return t.Probe + case *ResponseUnion_IsSpanEmpty: + return t.IsSpanEmpty default: return nil } @@ -467,6 +471,8 @@ func (ru *RequestUnion) MustSetInner(r Request) { union = &RequestUnion_Barrier{t} case *ProbeRequest: union = &RequestUnion_Probe{t} + case *IsSpanEmptyRequest: + union = &RequestUnion_IsSpanEmpty{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } @@ -572,13 +578,15 @@ func (ru *ResponseUnion) MustSetInner(r Response) { union = &ResponseUnion_Barrier{t} case *ProbeResponse: union = &ResponseUnion_Probe{t} + case *IsSpanEmptyResponse: + union = &ResponseUnion_IsSpanEmpty{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } ru.Value = union } -type reqCounts [48]int32 +type reqCounts [49]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -682,6 +690,8 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[46]++ case *RequestUnion_Probe: counts[47]++ + case *RequestUnion_IsSpanEmpty: + counts[48]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -738,6 +748,7 @@ var requestNames = []string{ "ScanInterleavedIntents", "Barrier", "Probe", + "IsSpanEmpty", } // Summary prints a short summary of the requests in a batch. @@ -961,6 +972,10 @@ type probeResponseAlloc struct { union ResponseUnion_Probe resp ProbeResponse } +type isSpanEmptyResponseAlloc struct { + union ResponseUnion_IsSpanEmpty + resp IsSpanEmptyResponse +} // CreateReply creates replies for each of the contained requests, wrapped in a // BatchResponse. The response objects are batch allocated to minimize @@ -1019,6 +1034,7 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf45 []scanInterleavedIntentsResponseAlloc var buf46 []barrierResponseAlloc var buf47 []probeResponseAlloc + var buf48 []isSpanEmptyResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -1358,6 +1374,13 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf47[0].union.Probe = &buf47[0].resp br.Responses[i].Value = &buf47[0].union buf47 = buf47[1:] + case *RequestUnion_IsSpanEmpty: + if buf48 == nil { + buf48 = make([]isSpanEmptyResponseAlloc, counts[48]) + } + buf48[0].union.IsSpanEmpty = &buf48[0].resp + br.Responses[i].Value = &buf48[0].union + buf48 = buf48[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } @@ -1464,6 +1487,8 @@ func CreateRequest(method Method) Request { return &BarrierRequest{} case Probe: return &ProbeRequest{} + case IsSpanEmpty: + return &IsSpanEmptyRequest{} default: panic(fmt.Sprintf("unsupported method: %+v", method)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index ea3d643e6325..abf87e68b479 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -174,6 +174,9 @@ const ( // Probe is a noop write request used to test the ability to make // progress at the replication layer. Probe + // IsSpanEmpty is a non-transaction read request used to determine whether + // a span contains any keys whatsoever (garbage or otherwise). + IsSpanEmpty // NumMethods represents the total number of API methods. NumMethods ) diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index 24ea3d64b0b9..bff55781d521 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -57,12 +57,13 @@ func _() { _ = x[ScanInterleavedIntents-46] _ = x[Barrier-47] _ = x[Probe-48] - _ = x[NumMethods-49] + _ = x[IsSpanEmpty-49] + _ = x[NumMethods-50] } -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryLocksQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportAdminScatterAddSSTableMigrateRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampQueryResolvedTimestampScanInterleavedIntentsBarrierProbeNumMethods" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryLocksQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportAdminScatterAddSSTableMigrateRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampQueryResolvedTimestampScanInterleavedIntentsBarrierProbeIsSpanEmptyNumMethods" -var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 216, 224, 235, 248, 266, 271, 282, 294, 307, 316, 331, 347, 354, 364, 370, 382, 392, 399, 413, 420, 432, 439, 449, 478, 500, 522, 529, 534, 544} +var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 216, 224, 235, 248, 266, 271, 282, 294, 307, 316, 331, 347, 354, 364, 370, 382, 392, 399, 413, 420, 432, 439, 449, 478, 500, 522, 529, 534, 545, 555} func (i Method) String() string { if i < 0 || i >= Method(len(_Method_index)-1) { diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index b3e1d26312d9..68dc1e98a509 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -5524,6 +5524,28 @@ func ComputeStatsForRangeWithVisitors( return ms, nil } +// MVCCIsSpanEmpty returns true if there are no MVCC keys whatsoever in the +// key span in the requested time interval. +func MVCCIsSpanEmpty( + ctx context.Context, reader Reader, opts MVCCIsSpanEmptyOptions, +) (isEmpty bool, _ error) { + iter := NewMVCCIncrementalIterator(reader, MVCCIncrementalIterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + StartKey: opts.StartKey, + EndKey: opts.EndKey, + StartTime: opts.StartTS, + EndTime: opts.EndTS, + IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, + }) + defer iter.Close() + iter.SeekGE(MVCCKey{Key: opts.StartKey}) + valid, err := iter.Valid() + if err != nil { + return false, err + } + return !valid, nil +} + // MVCCExportToSST exports changes to the keyrange [StartKey, EndKey) over the // interval (StartTS, EndTS] as a Pebble SST. See MVCCExportOptions for options. // @@ -5922,6 +5944,16 @@ type MVCCExportOptions struct { ResourceLimiter ResourceLimiter } +// MVCCIsSpanEmptyOptions configures the MVCCIsSpanEmpty function. +type MVCCIsSpanEmptyOptions struct { + // StartKey determines start of the checked span. + StartKey roachpb.Key + // EndKey determines the end of exported interval (exclusive). + EndKey roachpb.Key + // StartTS and EndTS determine the scanned time range as (startTS, endTS]. + StartTS, EndTS hlc.Timestamp +} + // PeekRangeKeysLeft peeks for any range keys to the left of the given key. // It returns the relative position of any range keys to the peek key, along // with the (unsafe) range key stack: diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index fef765e722ce..3efde0013836 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -671,6 +671,7 @@ var commands = map[string]cmd{ "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 @@ -1210,6 +1211,21 @@ func cmdPut(e *evalCtx) error { }) } +func cmdIsSpanEmpty(e *evalCtx) error { + key, endKey := e.getKeyRange() + isEmpty, err := MVCCIsSpanEmpty(e.ctx, e.engine, MVCCIsSpanEmptyOptions{ + StartKey: key, + EndKey: endKey, + StartTS: e.getTsWithName("startTs"), + EndTS: e.getTs(nil), + }) + if err != nil { + return err + } + e.results.buf.Print(isEmpty) + return nil +} + func cmdExport(e *evalCtx) error { key, endKey := e.getKeyRange() opts := MVCCExportOptions{ diff --git a/pkg/storage/testdata/mvcc_histories/clear_range b/pkg/storage/testdata/mvcc_histories/clear_range index fc430d912f73..42dec063413c 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_range +++ b/pkg/storage/testdata/mvcc_histories/clear_range @@ -18,6 +18,15 @@ data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +run ok +is_span_empty k=a end=+a +---- +false + +run ok +is_span_empty k=a end=z +---- +false run ok clear_range k=a end=+a @@ -28,6 +37,16 @@ data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +run ok +is_span_empty k=a end=+a +---- +true + +run ok +is_span_empty k=a end=-a +---- +false + run ok clear_range k=a end=-a ---- @@ -36,6 +55,11 @@ data: "b"/44.000000000,0 -> /BYTES/abc data: "b/123"/44.000000000,0 -> /BYTES/abc data: "c"/44.000000000,0 -> /BYTES/abc +run ok +is_span_empty k=a end=-a +---- +true + run ok clear_range k=a end==b ---- diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 1846d76de203..e4517eb923f6 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -355,6 +355,7 @@ var charts = []sectionDescription{ "rpc.method.transferlease.recv", "rpc.method.truncatelog.recv", "rpc.method.writebatch.recv", + "rpc.method.isspanempty.recv", }, }, { @@ -409,6 +410,7 @@ var charts = []sectionDescription{ "distsender.rpc.probe.sent", "distsender.rpc.truncatelog.sent", "distsender.rpc.writebatch.sent", + "distsender.rpc.isspanempty.sent", }, }, {