Skip to content

Commit

Permalink
Merge #59086
Browse files Browse the repository at this point in the history
59086: kv: move range lease checks and transfers below latching r=nvanbenschoten a=nvanbenschoten

Needed for #57688.

This PR reworks interactions between range leases and requests, pulling the consultation of a replica's lease down below the level of latching while keeping heavy-weight operations like lease acquisitions above the level of latching. Doing so comes with several benefits, some related specifically to non-blocking transactions and some more general.

### Background

Before discussing the change here, let's discuss how lease checks, lease acquisitions, lease redirection, and lease transfers currently work. Today, requests consult a replica's range lease before acquiring latches. If the lease is good to go, the request proceeds to acquire latches. If the lease is not currently held by any replica, the lease is acquired (again, above latches) through a coalesced `RequestLeaseRequest`. If the lease is currently held by a different replica, the request is redirected to that replica using a `NotLeaseHolderError`. Finally, if the lease check notices a lease transfer in progress, the request is optimistically redirected to the prospective new leaseholder.

This all works, but only because it's been around for so long. Due to the lease check above latching, we're forced to go to great lengths to get the synchronization with in-flight requests right, which leads to very subtle logic. This is most apparent with lease transfers, which properly synchronize with ongoing requests through a delicate dance with the HLC clock and some serious "spooky action at a distance". Every request bumps the local HLC clock in `Store.Send`, then grabs the replica mutex, checks for an ongoing lease transfer, drops the replica mutex, then evaluates. Lease transfers grab the replica mutex, grab a clock reading from the local HLC clock, bump the minLeaseProposedTS to stop using the current lease, drops the replica mutex, then proposes a new lease using this clock reading as its start time. This works only because each request bumps the HLC clock _before_ checking the lease, so the HLC clock can serve as an upper bound on every request that has made it through the lease check by the time the lease transfer begins.

This structure is inflexible, subtle, and falls over as soon as we try to extend it.

### Motivation

The primary motivation for pulling lease checks and transfers below latching is that the interaction between requests and lease transfers is incompatible with future-time operations, a key part of the non-blocking transaction project. This is because the structure relies on the HLC clock providing an upper bound on the time of any request served by an outgoing leaseholder, which is attached to lease transfers to ensure that the new leaseholder does not violate any request served on the old leaseholder. But this is quickly violated once we start serving future-time operations, which don't bump the HLC clock.

So we quickly need to look elsewhere for this information. The obvious place to look for this information is the timestamp cache, which records the upper bound read time of each key span in a range, even if this upper bound time is synthetic. If we could scan the timestamp cache and attach the maximum read time to a lease transfer (through a new field, not as the lease start time), we'd be good. But this runs into a problem, because if we just read the timestamp cache under the lease transfer's lock, we can't be sure we didn't miss any in-progress operations that had passed the lease check previously but had not yet bumped the timestamp cache. Maybe they are still reading? So the custom locking quickly runs into problems (I said it was inflexible!).

### Solution

The solution here is to stop relying on custom locking for lease transfers by pulling the lease check below latching and by pulling the determination of the transfer's start time below latching. This ensures that during a lease transfer, we don't only block new requests, but we also flush out in-flight requests. This means that by the time we look at the timestamp cache during the evaluation of a lease transfer, we know it has already been updated by any request that will be served under the current lease.

This commit doesn't make the switch from consulting the HLC clock to consulting the timestamp cache during TransferLease request evaluation, but a future commit will.

### Other benefits

Besides this primary change, a number of other benefits fall out of this restructuring.

1. we avoid relying on custom synchronization around leases, instead relying on more the more general latching mechanism.
2. we more closely aligns `TransferLeaseRequest` and `SubsumeRequest`, which now both grab clock readings during evaluation and will both need to forward their clock reading by the upper-bound of a range's portion of the timestamp cache. It makes sense that these two requests would be very similar, as both are responsible for renouncing the current leaseholder's powers and passing them elsewhere.
3. we more closely aligns the lease acquisition handling with the handling of `MergeInProgressError` by classifying a new `InvalidLeaseError` as a "concurrencyRetryError" (see isConcurrencyRetryError). This fits the existing structure of: grab latches, check range state, drop latches and wait if necessary, retry.
4. in doing so, we fuse the critical section of lease checks and the rest of the checks in `checkExecutionCanProceed`. So we grab the replica read lock one fewer time in the request path.
5. we move one step closer to a world where we can "ship a portion of the timestamp cache" during lease transfers (and range merges) to avoid retry errors / transaction aborts on the new leaseholder. This commit will be followed up by one that ships a very basic summary of a leaseholder's timestamp cache during lease transfers. However, this would now be trivial to extend with higher resolution information, given some size limit. Perhaps we prioritize the local portion of the timestamp cache to avoid txn aborts?
6. now that leases are checked below latching, we no longer have the potential for an arbitrary delay due to latching and waiting on locks between when the lease is checked and when a request evaluates, so we no longer need checks like [this](https://github.com/cockroachdb/cockroach/blob/7bcb2cef794da56f6993f1b27d5b6a036016242b/pkg/kv/kvserver/replica_write.go#L119).
7. we pull observed timestamp handling a layer down, which will be useful to address plumbing comments on #57077.

### Other behavioral changes

There are two auxiliary behavioral changes made by this commit that deserve attention.

The first is that during a lease transfer, operations now block on the outgoing leaseholder instead of immediately redirecting to the expected next leaseholder. This has trade-offs. On one hand, this delays redirection, which may make lease transfers more disruptive to ongoing traffic. On the other, we've seen in the past that the optimistic redirection is not an absolute win. In many cases, it can lead to thrashing and lots of wasted work, as the outgoing leaseholder and the incoming leaseholder both point at each other and requests ping-pong between them. We've seen this cause serious issues like #22837 and #32367, which we addressed by adding exponential backoff in the client in 89d349a. So while this change may make average-case latency during lease transfers slightly worse, it will keep things much more orderly, avoid wasted work, and reduce worst-case latency during lease transfers.

The other behavioral changes made by this commit is that observed timestamps are no longer applied to a request to reduce its MaxOffset until after latching and locking, instead of before. This sounds concerning, but it's actually not for two reasons. First, as of #57136, a transactions uncertainty interval is no longer considered by the lock table because locks in a transaction's uncertainty interval are no longer considered write-read conflicts. Instead, those locks' provisional values are considered at evaluation time to be uncertain. Second, the fact that the observed timestamp-limited MaxOffset was being used for latching is no longer correct in a world with synthetic timestamps (see #57077), so we would have had to make this change anyway. So put together, this behavioral change isn't meaningful.

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Feb 6, 2021
2 parents b5f8b8c + c44b357 commit 81a2c26
Show file tree
Hide file tree
Showing 47 changed files with 1,595 additions and 857 deletions.
14 changes: 7 additions & 7 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
// These constants are single bytes for performance. They allow single-byte
// comparisons which are considerably faster than bytes.HasPrefix.
const (
localPrefixByte = '\x01'
LocalPrefixByte = '\x01'
localMaxByte = '\x02'
meta1PrefixByte = localMaxByte
meta2PrefixByte = '\x03'
Expand All @@ -42,8 +42,8 @@ var (
// MaxKey is the infinity marker which is larger than any other key.
MaxKey = roachpb.KeyMax

// localPrefix is the prefix for all local keys.
localPrefix = roachpb.Key{localPrefixByte}
// LocalPrefix is the prefix for all local keys.
LocalPrefix = roachpb.Key{LocalPrefixByte}
// LocalMax is the end of the local key range. It is itself a global
// key.
LocalMax = roachpb.Key{localMaxByte}
Expand All @@ -64,7 +64,7 @@ var (
// metadata is identified by one of the suffixes listed below, along
// with potentially additional encoded key info, for instance in the
// case of AbortSpan entry.
LocalRangeIDPrefix = roachpb.RKey(makeKey(localPrefix, roachpb.Key("i")))
LocalRangeIDPrefix = roachpb.RKey(makeKey(LocalPrefix, roachpb.Key("i")))
// LocalRangeIDReplicatedInfix is the post-Range ID specifier for all Raft
// replicated per-range data. By appending this after the Range ID, these
// keys will be sorted directly before the local unreplicated keys for the
Expand Down Expand Up @@ -134,7 +134,7 @@ var (
// specific sort of per-range metadata is identified by one of the
// suffixes listed below, along with potentially additional encoded
// key info, such as the txn ID in the case of a transaction record.
LocalRangePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("k")))
LocalRangePrefix = roachpb.Key(makeKey(LocalPrefix, roachpb.RKey("k")))
LocalRangeMax = LocalRangePrefix.PrefixEnd()
// LocalQueueLastProcessedSuffix is the suffix for replica queue state keys.
LocalQueueLastProcessedSuffix = roachpb.RKey("qlpt")
Expand All @@ -148,7 +148,7 @@ var (
// 4. Store local keys
//
// LocalStorePrefix is the prefix identifying per-store data.
LocalStorePrefix = makeKey(localPrefix, roachpb.Key("s"))
LocalStorePrefix = makeKey(LocalPrefix, roachpb.Key("s"))
// localStoreSuggestedCompactionSuffix stores suggested compactions to
// be aggregated and processed on the store.
localStoreSuggestedCompactionSuffix = []byte("comp")
Expand Down Expand Up @@ -200,7 +200,7 @@ var (
// double duty as a reference to a provisional MVCC value.
// TODO(sumeer): remember to adjust this comment when adding locks of
// other strengths, or range locks.
LocalRangeLockTablePrefix = roachpb.Key(makeKey(localPrefix, roachpb.RKey("z")))
LocalRangeLockTablePrefix = roachpb.Key(makeKey(LocalPrefix, roachpb.RKey("z")))
LockTableSingleKeyInfix = []byte("k")
// LockTableSingleKeyStart is the inclusive start key of the key range
// containing single key locks.
Expand Down
4 changes: 2 additions & 2 deletions pkg/keys/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,8 +177,8 @@ var _ = [...]interface{}{
// `LocalRangeLockTablePrefix`.
//
// `LocalRangeIDPrefix`, `localRangePrefix`, `localStorePrefix`, and
// `LocalRangeLockTablePrefix` all in turn share `localPrefix`.
// `localPrefix` was chosen arbitrarily. Local keys would work just as well
// `LocalRangeLockTablePrefix` all in turn share `LocalPrefix`.
// `LocalPrefix` was chosen arbitrarily. Local keys would work just as well
// with a different prefix, like 0xff, or even with a suffix.

// 1. Replicated range-ID local keys: These store metadata pertaining to a
Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -478,7 +478,7 @@ func DecodeLockTableSingleKey(key roachpb.Key) (lockedKey roachpb.Key, err error
// opposed to "user") keys, but unfortunately that name has already been
// claimed by a related (but not identical) concept.
func IsLocal(k roachpb.Key) bool {
return bytes.HasPrefix(k, localPrefix)
return bytes.HasPrefix(k, LocalPrefix)
}

// Addr returns the address for the key, used to lookup the range containing the
Expand Down
4 changes: 2 additions & 2 deletions pkg/keys/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func TestKeySorting(t *testing.T) {
roachpb.RKey("\x01").Less(roachpb.RKey("\x01\x00"))) {
t.Fatalf("something is seriously wrong with this machine")
}
if bytes.Compare(localPrefix, Meta1Prefix) >= 0 {
if bytes.Compare(LocalPrefix, Meta1Prefix) >= 0 {
t.Fatalf("local key spilling into replicated ranges")
}
if !bytes.Equal(roachpb.Key(""), roachpb.Key(nil)) {
Expand Down Expand Up @@ -140,7 +140,7 @@ func TestKeyAddressError(t *testing.T) {
RangeLastReplicaGCTimestampKey(0),
},
"local key .* malformed": {
makeKey(localPrefix, roachpb.Key("z")),
makeKey(LocalPrefix, roachpb.Key("z")),
},
}
for regexp, keyList := range testCases {
Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ var (

// KeyDict drives the pretty-printing and pretty-scanning of the key space.
KeyDict = KeyComprehensionTable{
{Name: "/Local", start: localPrefix, end: LocalMax, Entries: []DictEntry{
{Name: "/Local", start: LocalPrefix, end: LocalMax, Entries: []DictEntry{
{Name: "/Store", prefix: roachpb.Key(LocalStorePrefix),
ppFunc: localStoreKeyPrint, PSFunc: localStoreKeyParse},
{Name: "/RangeID", prefix: roachpb.Key(LocalRangeIDPrefix),
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -92,8 +92,8 @@ go_test(
"cmd_resolve_intent_test.go",
"cmd_revert_range_test.go",
"cmd_scan_test.go",
"cmd_subsume_test.go",
"cmd_truncate_log_test.go",
"declare_test.go",
"intent_test.go",
"main_test.go",
"transaction_test.go",
Expand Down
64 changes: 64 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,10 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -120,10 +123,13 @@ func TestLeaseCommandLearnerReplica(t *testing.T) {
}
desc := roachpb.RangeDescriptor{}
desc.SetReplicas(roachpb.MakeReplicaSet(replicas))
manual := hlc.NewManualClock(123)
clock := hlc.NewClock(manual.UnixNano, time.Nanosecond)
cArgs := CommandArgs{
EvalCtx: (&MockEvalCtx{
StoreID: voterStoreID,
Desc: &desc,
Clock: clock,
}).EvalContext(),
Args: &roachpb.TransferLeaseRequest{
Lease: roachpb.Lease{
Expand Down Expand Up @@ -157,6 +163,64 @@ func TestLeaseCommandLearnerReplica(t *testing.T) {
require.EqualError(t, err, expForLearner)
}

// TestLeaseTransferForwardsStartTime tests that during a lease transfer, the
// start time of the new lease is determined during evaluation, after latches
// have granted the lease transfer full mutual exclusion over the leaseholder.
func TestLeaseTransferForwardsStartTime(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testutils.RunTrueAndFalse(t, "epoch", func(t *testing.T, epoch bool) {
ctx := context.Background()
db := storage.NewDefaultInMem()
defer db.Close()
batch := db.NewBatch()
defer batch.Close()

replicas := []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: 1, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1},
{NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 2},
}
desc := roachpb.RangeDescriptor{}
desc.SetReplicas(roachpb.MakeReplicaSet(replicas))
manual := hlc.NewManualClock(123)
clock := hlc.NewClock(manual.UnixNano, time.Nanosecond)

nextLease := roachpb.Lease{
Replica: replicas[1],
Start: clock.NowAsClockTimestamp(),
}
if epoch {
nextLease.Epoch = 1
} else {
exp := nextLease.Start.ToTimestamp().Add(9*time.Second.Nanoseconds(), 0)
nextLease.Expiration = &exp
}
cArgs := CommandArgs{
EvalCtx: (&MockEvalCtx{
StoreID: 1,
Desc: &desc,
Clock: clock,
}).EvalContext(),
Args: &roachpb.TransferLeaseRequest{
Lease: nextLease,
},
}

manual.Increment(1000)
beforeEval := clock.NowAsClockTimestamp()

res, err := TransferLease(ctx, batch, cArgs, nil)
require.NoError(t, err)

// The proposed lease start time should be assigned at eval time.
propLease := res.Replicated.State.Lease
require.NotNil(t, propLease)
require.True(t, nextLease.Start.Less(propLease.Start))
require.True(t, beforeEval.Less(propLease.Start))
})
}

func TestCheckCanReceiveLease(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
43 changes: 32 additions & 11 deletions pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package batcheval
import (
"context"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand All @@ -28,13 +27,26 @@ func init() {
func declareKeysTransferLease(
rs ImmutableRangeState, _ roachpb.Header, _ roachpb.Request, latchSpans, _ *spanset.SpanSet,
) {
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})
// TransferLease must not run concurrently with any other request so it uses
// latches to synchronize with all other reads and writes on the outgoing
// leaseholder. Additionally, it observes the state of the timestamp cache
// and so it uses latches to wait for all in-flight requests to complete.
//
// Because of this, it declares a non-MVCC write over every addressable key
// in the range, even through the only key the TransferLease actually writes
// to is the RangeLeaseKey. This guarantees that it conflicts with any other
// request because every request must declare at least one addressable key.
//
// We could, in principle, declare these latches as MVCC writes at the time
// of the new lease. Doing so would block all concurrent writes but would
// allow reads below the new lease timestamp through. However, doing so
// would only be safe if we also accounted for clock uncertainty in all read
// latches so that any read that may need to observe state on the new
// leaseholder gets blocked. We actually already do this for transactional
// reads (see DefaultDeclareIsolatedKeys), but not for non-transactional
// reads. We'd need to be careful here, so we should only pull on this if we
// decide that doing so is important.
declareAllKeys(latchSpans)
}

// TransferLease sets the lease holder for the range.
Expand All @@ -55,13 +67,22 @@ func TransferLease(
// LeaseRejectedError before going through Raft.
prevLease, _ := cArgs.EvalCtx.GetLease()

// Forward the lease's start time to a current clock reading. At this
// point, we're holding latches across the entire range, we know that
// this time is greater than the timestamps at which any request was
// serviced by the leaseholder before it stopped serving requests (i.e.
// before the TransferLease request acquired latches).
newLease := args.Lease
newLease.Start.Forward(cArgs.EvalCtx.Clock().NowAsClockTimestamp())
args.Lease = roachpb.Lease{} // prevent accidental use below

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
if err := roachpb.CheckCanReceiveLease(args.Lease.Replica, cArgs.EvalCtx.Desc()); err != nil {
if err := roachpb.CheckCanReceiveLease(newLease.Replica, cArgs.EvalCtx.Desc()); err != nil {
return newFailedLeaseTrigger(true /* isTransfer */), err
}

log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, args.Lease)
log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, newLease)
return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats,
args.Lease, prevLease, false /* isExtension */, true /* isTransfer */)
newLease, prevLease, false /* isExtension */, true /* isTransfer */)
}
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,8 @@ func PushTxn(
return result.Result{}, errors.Errorf("request timestamp %s less than pushee txn timestamp %s", h.Timestamp, args.PusheeTxn.WriteTimestamp)
}
now := cArgs.EvalCtx.Clock().Now()
// TODO(nvanbenschoten): remove this limitation. But when doing so,
// keep the h.Timestamp.Less(args.PushTo) check above.
if now.Less(h.Timestamp) {
// The batch's timestamp should have been used to update the clock.
return result.Result{}, errors.Errorf("request timestamp %s less than current clock time %s", h.Timestamp, now)
Expand Down
25 changes: 4 additions & 21 deletions pkg/kv/kvserver/batcheval/cmd_subsume.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,27 +32,10 @@ func declareKeysSubsume(
) {
// Subsume must not run concurrently with any other command. It declares a
// non-MVCC write over every addressable key in the range; this guarantees
// that it conflicts with any other command because every command must declare
// at least one addressable key. It does not, in fact, write any keys.
//
// We use the key bounds from the range descriptor in the request instead
// of the current range descriptor. Either would be fine because we verify
// that these match during the evaluation of the Subsume request.
args := req.(*roachpb.SubsumeRequest)
desc := args.RightDesc
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: desc.StartKey.AsRawKey(),
EndKey: desc.EndKey.AsRawKey(),
})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.MakeRangeKeyPrefix(desc.StartKey),
EndKey: keys.MakeRangeKeyPrefix(desc.EndKey).PrefixEnd(),
})
rangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(desc.RangeID)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: rangeIDPrefix,
EndKey: rangeIDPrefix.PrefixEnd(),
})
// that it conflicts with any other command because every command must
// declare at least one addressable key. It does not, in fact, write any
// keys.
declareAllKeys(latchSpans)
}

// Subsume freezes a range for merging with its left-hand neighbor. When called
Expand Down
14 changes: 14 additions & 0 deletions pkg/kv/kvserver/batcheval/declare.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,20 @@ func DeclareKeysForBatch(
}
}

// declareAllKeys declares a non-MVCC write over every addressable key. This
// guarantees that the caller conflicts with any other command because every
// command must declare at least one addressable key, which is tested against
// in TestRequestsSerializeWithAllKeys.
func declareAllKeys(latchSpans *spanset.SpanSet) {
// NOTE: we don't actually know what the end key of the Range will
// be at the time of request evaluation (see ImmutableRangeState),
// so we simply declare a latch over the entire keyspace. This may
// extend beyond the Range, but this is ok for the purpose of
// acquiring latches.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalPrefix, EndKey: keys.LocalMax})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey})
}

// CommandArgs contains all the arguments to a command.
// TODO(bdarnell): consider merging with kvserverbase.FilterArgs (which
// would probably require removing the EvalCtx field due to import order
Expand Down
Loading

0 comments on commit 81a2c26

Please sign in to comment.