Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
58897: tracing: enable always-on tracing by default r=irfansharif a=irfansharif

This is follow-up work from #58712, where we measured the overhead for
always-on tracing and found it to be minimal/acceptable. Lets switch
this on by default to shake the implications of doing so. We can
reasonably expect two kinds of fallout:

1. Unexpected blow up in memory usage due to resource leakage (which is
a can be problem now that we're always maintaining open spans in an
internal registry, see #58721)

2. Performance degradataion due to tracing overhead per-request
(something #58712) was spot checking for.

For 1 we'll introduce a future test in a separate PR. For 2, we'll
monitor roachperf over the next few weeks.

---

Also moved some of the documentation for the cluster setting into a
comment form above. Looking at what's rendered in our other cluster
settings (`SHOW ALL CLUSTER SETTINGS`), we default to a very pity,
unwrapped description.

Release note: None

58974: opt: suppress logs in benchmarks r=rytaft a=mgartner

As of #57134 passing `-logtostderr=false` as a `TESTFLAG` in benchmarks
errs: `flag provided but not defined: -logtostderr`. The preferred
method for suppressing logs in tests and benchmarks to is add
`defer log.Scope(t).Close(t)` to the top of the test/benchmark
(see #57979).

This commit uses this new method to suppress logs in optimizer
benchmarks.

Release note: None

59009: kv/batcheval: only expose immutable range state to commands r=nvanbenschoten a=nvanbenschoten

The DeclareKeysFunc has always included a full RangeDescriptor, but it
has never been clear which fields in this descriptor are safe to use and
which are not when declaring keys for a request. The concern is that any
property of the RangeDescriptor that is not immutable may change between
the time that a request declares its keys to latch and the time that it
evaluates, so any assumptions based on these mutable fields may not
hold.

The quintessential example of a property of a Range that is not
immutable is its end key. It would be incorrect to declare keys between
a Range's start key and its current end key as a means of latching the
entire range, because a merge of a right-hand neighbor could complete in
between the time that a request declares its keys and the time that it
evaluates. This could lead to a violation of the mutual exclusion that
the command was expecting to have.

This commit makes these kinds of mistakes impossible to make by putting
the RangeDescriptor behind an interface that only exposes the properties
of a Range that cannot change across a Range's lifetime.

59099: kvserver: fix rangelog event logging for non-voter additions r=aayushshah15 a=aayushshah15

Before this patch, we were incorrectly logging non-voter additions as
removals.

Release note: None

59142: sql: fix indentation in information_schema.columns schema r=otan a=arulajmani

Reviewable lied to me.

Release note: None

Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Aayush Shah <[email protected]>
Co-authored-by: arulajmani <[email protected]>
  • Loading branch information
6 people committed Jan 19, 2021
6 parents 731d083 + 17ea476 + a628983 + b7fb524 + d8fd2f9 + 26d0d52 commit debeb0b
Show file tree
Hide file tree
Showing 39 changed files with 176 additions and 173 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/storageccl/export.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,12 +57,12 @@ func init() {
}

func declareKeysExport(
desc *roachpb.RangeDescriptor,
rs batcheval.ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
batcheval.DefaultDeclareIsolatedKeys(desc, header, req, latchSpans, lockSpans)
batcheval.DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLastGCKey(header.RangeID)})
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,15 @@ func init() {
}

func declareKeysClearRange(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
DefaultDeclareKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
}

// ClearRange wipes all MVCC versions of keys covered by the specified
Expand Down
8 changes: 2 additions & 6 deletions pkg/kv/kvserver/batcheval/cmd_compute_checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,7 @@ func init() {
}

func declareKeysComputeChecksum(
desc *roachpb.RangeDescriptor,
_ roachpb.Header,
_ roachpb.Request,
latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
// The correctness of range merges depends on the lease applied index of a
// range not being bumped while the RHS is subsumed. ComputeChecksum bumps a
Expand All @@ -42,8 +39,7 @@ func declareKeysComputeChecksum(
// at the end of Subsume() in cmd_subsume.go for details. Thus, it must
// declare access over at least one key. We choose to declare read-only access
// over the range descriptor key.
rdKey := keys.RangeDescriptorKey(desc.StartKey)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: rdKey})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
}

// Version numbers for Replica checksum computation. Requests silently no-op
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_conditional_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,16 @@ func init() {
}

func declareKeysConditionalPut(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
args := req.(*roachpb.ConditionalPutRequest)
if args.Inline {
DefaultDeclareKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
} else {
DefaultDeclareIsolatedKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
}
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,16 @@ func init() {
}

func declareKeysDeleteRange(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
args := req.(*roachpb.DeleteRangeRequest)
if args.Inline {
DefaultDeclareKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
} else {
DefaultDeclareIsolatedKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
}
}

Expand Down
21 changes: 10 additions & 11 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,7 @@ func init() {
// declareKeysWriteTransaction is the shared portion of
// declareKeys{End,Heartbeat}Transaction.
func declareKeysWriteTransaction(
_ *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans *spanset.SpanSet,
_ ImmutableRangeState, header roachpb.Header, req roachpb.Request, latchSpans *spanset.SpanSet,
) {
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
Expand All @@ -55,13 +52,13 @@ func declareKeysWriteTransaction(
}

func declareKeysEndTxn(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
) {
et := req.(*roachpb.EndTxnRequest)
declareKeysWriteTransaction(desc, header, req, latchSpans)
declareKeysWriteTransaction(rs, header, req, latchSpans)
var minTxnTS hlc.Timestamp
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
Expand All @@ -76,7 +73,7 @@ func declareKeysEndTxn(
abortSpanAccess = spanset.SpanReadWrite
}
latchSpans.AddNonMVCC(abortSpanAccess, roachpb.Span{
Key: keys.AbortSpanKey(header.RangeID, header.Txn.ID),
Key: keys.AbortSpanKey(rs.GetRangeID(), header.Txn.ID),
})
}

Expand All @@ -86,7 +83,9 @@ func declareKeysEndTxn(
// All requests that intend on resolving local locks need to depend on
// the range descriptor because they need to determine which locks are
// within the local range.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.RangeDescriptorKey(rs.GetStartKey()),
})

// The spans may extend beyond this Range, but it's ok for the
// purpose of acquiring latches. The parts in our Range will
Expand Down Expand Up @@ -120,7 +119,7 @@ func declareKeysEndTxn(
EndKey: keys.MakeRangeKeyPrefix(st.RightDesc.EndKey).PrefixEnd(),
})

leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(header.RangeID)
leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(rs.GetRangeID())
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: leftRangeIDPrefix,
EndKey: leftRangeIDPrefix.PrefixEnd(),
Expand All @@ -145,8 +144,8 @@ func declareKeysEndTxn(
})

latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: abortspan.MinKey(header.RangeID),
EndKey: abortspan.MaxKey(header.RangeID),
Key: abortspan.MinKey(rs.GetRangeID()),
EndKey: abortspan.MaxKey(rs.GetRangeID()),
})
}
if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil {
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ func init() {
}

func declareKeysGC(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
Expand All @@ -45,10 +45,10 @@ func declareKeysGC(
// request first to bump the thresholds, and then another one that actually does work
// but can avoid declaring these keys below.
if !gcr.Threshold.IsEmpty() {
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLastGCKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLastGCKey(rs.GetRangeID())})
}
// Needed for Range bounds checks in calls to EvalContext.ContainsKey.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
}

// GC iterates through the list of keys to garbage collect
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,12 @@ func init() {
}

func declareKeysHeartbeatTransaction(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
) {
declareKeysWriteTransaction(desc, header, req, latchSpans)
declareKeysWriteTransaction(rs, header, req, latchSpans)
}

// HeartbeatTxn updates the transaction status and heartbeat
Expand Down
7 changes: 2 additions & 5 deletions pkg/kv/kvserver/batcheval/cmd_lease_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,9 @@ func init() {
}

func declareKeysLeaseInfo(
_ *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
}

// LeaseInfo returns information about the lease holder for the range.
Expand Down
9 changes: 3 additions & 6 deletions pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,18 +25,15 @@ func init() {
}

func declareKeysRequestLease(
desc *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
// NOTE: RequestLease is run on replicas that do not hold the lease, so
// acquiring latches would not help synchronize with other requests. As
// such, the request does not actually acquire latches over these spans
// (see concurrency.shouldAcquireLatches). However, we continue to
// declare the keys in order to appease SpanSet assertions under race.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
}

// RequestLease sets the range lease for this range. The command fails
Expand Down
17 changes: 7 additions & 10 deletions pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,25 +21,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func init() {
RegisterReadWriteCommand(roachpb.TransferLease, declareKeysTransferLease, TransferLease)
}

func declareKeysTransferLease(
desc *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
// Cover the entire addressable key space with a latch to prevent any writes
// from overlapping with lease transfers. In principle we could just use the
// current range descriptor (desc) but it could potentially change due to an
// as of yet unapplied merge.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey})
}

func init() {
RegisterReadWriteCommand(roachpb.TransferLease, declareKeysTransferLease, TransferLease)
}

// TransferLease sets the lease holder for the range.
// Unlike with RequestLease(), the new lease is allowed to overlap the old one,
// the contract being that the transfer must have been initiated by the (soon
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvserver/batcheval/cmd_migrate.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@ func init() {
}

func declareKeysMigrate(
desc *roachpb.RangeDescriptor,
header roachpb.Header,
rs ImmutableRangeState,
_ roachpb.Header,
_ roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
Expand All @@ -40,10 +40,10 @@ func declareKeysMigrate(
// define the allow authors for specific set of keys each migration needs to
// grab latches and locks over.

latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeVersionKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(header.RangeID)})
lockSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeVersionKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(rs.GetRangeID())})
lockSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(rs.GetRangeID())})
}

// migrationRegistry is a global registry of all KV-level migrations. See
Expand Down
7 changes: 2 additions & 5 deletions pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,11 @@ func init() {
}

func declareKeysPushTransaction(
_ *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
rs ImmutableRangeState, _ roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
pr := req.(*roachpb.PushTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(pr.PusheeTxn.Key, pr.PusheeTxn.ID)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(header.RangeID, pr.PusheeTxn.ID)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(rs.GetRangeID(), pr.PusheeTxn.ID)})
}

// PushTxn resolves conflicts between concurrent txns (or between
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,16 @@ func init() {
}

func declareKeysPut(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
args := req.(*roachpb.PutRequest)
if args.Inline {
DefaultDeclareKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
} else {
DefaultDeclareIsolatedKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
}
}

Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/batcheval/cmd_query_intent.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,7 @@ func init() {
}

func declareKeysQueryIntent(
_ *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState, _ roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
// QueryIntent requests read the specified keys at the maximum timestamp in
// order to read any intent present, if one exists, regardless of the
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/batcheval/cmd_query_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,7 @@ func init() {
}

func declareKeysQueryTransaction(
_ *roachpb.RangeDescriptor,
header roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ ImmutableRangeState, _ roachpb.Header, req roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
qr := req.(*roachpb.QueryTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(qr.Txn.Key, qr.Txn.ID)})
Expand Down
16 changes: 8 additions & 8 deletions pkg/kv/kvserver/batcheval/cmd_range_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,20 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
)

func init() {
RegisterReadOnlyCommand(roachpb.RangeStats, declareKeysRangeStats, RangeStats)
}

func declareKeysRangeStats(
desc *roachpb.RangeDescriptor,
rs ImmutableRangeState,
header roachpb.Header,
req roachpb.Request,
latchSpans, lockSpans *spanset.SpanSet,
) {
DefaultDeclareKeys(desc, header, req, latchSpans, lockSpans)
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans)
// The request will return the descriptor and lease.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(header.RangeID)})
}

func init() {
RegisterReadOnlyCommand(roachpb.RangeStats, declareKeysRangeStats, RangeStats)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
}

// RangeStats returns the MVCC statistics for a range.
Expand Down
Loading

0 comments on commit debeb0b

Please sign in to comment.