Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
95789: pkg/util/log: don't falsify tenant ID tag in logs if none in ctx r=andreimatei a=abarganier

Previously, I made the decision to always tag a log entry with a tenant ID, even if no tenant ID was found in the context associated with the log entry. In this case, the system tenant ID was used in the tag, instead of omitting a tenant ID tag altogether.

I received some feedback that this is confusing. For example, imagine testing a feature, expecting log entries to come from a secondary tenant, and the context being used in that feature is not annotated with a tenant ID. With the previous behavior, the log entry would default to being tagged with the system tenant ID instead of having empty tags (or at least, no tenant ID tag). In this scenario, how do I tell the actual state of the log entry? Did the log entry indeed come from a goroutine belonging to the system tenant? Or was the context just missing the tenant ID annotation, but otherwise came from the correct tenant?

This ambiguity is not helpful. By falsifying a tenant ID tag we confuse the log reader about the actual state of the system. Furthermore, our eventual goal should be that almost no context objects in the system exist without a tenant ID (except for perhaps at startup before tenant initialization). Tagging with the system tenant ID in the case of a missing tenant ID annotation in the context makes it difficult to track down offending context objects.

This patch removes this default behavior from the logging package. Now, if no tenant ID is found in the context, we do not tag the entry with a tenant ID. Note however that on the *decode* side, we will maintain this default tenant ID tagging behavior. If a log entry does not have a tenant ID tag, then we must assume that only the system tenant has privilege to view said log entry, since the owner is ambiguous.

Release note: none

Epic CRDB-14486

98175: cdc: show all changefeed jobs in `SHOW CHANGEFEED JOBS` r=HonoreDB a=jayshrivastava

### cdc: show all changefeed jobs in SHOW CHANGEFEED JOBS

Release note (general change): Previously, the output of `SHOW CHANGEFEED JOBS` was limited to show unfinished jobs and finished jobs from the last 14 days. This change makes the command show all changefeed jobs, regardless of if they finished and when they finished. Note that jobs still obey the cluster setting `jobs.retention_time`. Completed jobs older than that time are deleted.

Fixes: #97883

### jobs: add virtual index for job_type in crdb_internal.jobs

This change adds a virtual index on the `job_type` column
of `crdb_internal.jobs`. This change should make queries
on that table which filter on job type (such as `SHOW
CHANGEFEED JOBS`) more efficient.

Release note: None

Epic: None

98515: kvserver: deflake test store capacity after split r=andrewbaptist a=kvoli

This commit defales `TestStoreCapacityAfterSplit`. Previously it was possible for the replica load stats which underpins Capacity to be reset. The reset caused the recording duration to fall short of min stats duration, which led to a 0 value being reported for writes in store capacity.

This commit bumps the manual clock twice and removes redundant leaseholder checks within a retry loop. The combination of these two changes makes the test much less likely to flake.

The test is now unskipped.

```
dev test pkg/kv/kvserver -f TestStoreCapacityAfterSplit -v --stress
...
4410 runs so far, 0 failures, over 6m10s
```

Resolves: #92677

Release note: None

98521: ui: don't continue polling endpoints that return 403 errors r=dhartunian a=abarganier

It was brought to our attention that endpoints such as `v1/settings` would continue to be polled by DB Console even if they returned 403 errors.

If an endpoint returns 403 errors, we should not continue to poll it since the required access is not present for the current user.

This patch updates the polling mechanism to short-circuit the `refresh` process if a 403 error is encountered throughout the lifecycle of the poller.

Release note: none

Fixes: #98356

98536: kvserver: deflake learner joint cfg relocate range r=andrewbaptist a=kvoli

Previously, in `TestLearnerOrJointConfigAdminRelocateRange` it was possible for there to be an in-flight snapshot towards a learner, prior to sending `AdminRelocateRange` command. When this occurred, the test would fail as `AdminRelocateRange` returns an error when finding any in-flight snapshots to learners. This situation occurred infrequently, causing the test to flake.

This commit updates the `TestLearnerOrJointConfigAdminRelocateRange` test to first assert that there are the expected number of learners, then assert that there are no in-flight snapshots towards learners before beginning the main testing logic. The test is now unskipped.

```
dev test pkg/kv/kvserver \
      -f TestLearnerOrJointConfigAdminRelocateRange \
      -v --stress
...
5652 runs so far, 0 failures, over 12m30s
```

Resolves: #95500

Release note: None

98542: storage: remove MVCCIterator.Key method r=jbowens a=jbowens

The MVCCIterator interface previously exposed two methods for accessing the current iterator postion as a MVCC key—UnsafeKey and Key. Key() was equivalent to UnsafeKey().Clone().

This commit removes the Key() variant, pushing the onus of key copying onto the caller. This reduces the interface surface area, avoids accidental key copying (some of which is addressed within this commit), and does not impose any unreasonable burden on callers.

Epic: None
Informs #82589.
Release note: None

98543: allocator: fix lease io enforcement setting typo r=andrewbaptist a=kvoli

This commit updates the "do nothing" lease IO overload enforcement (`kv.allocator.lease_io_overload_threshold_enforcement`) setting to be correctly spelled "ignore" rather than "ingore".

Part of: #96508

Release note (ops change): The
`kv.allocator.lease_io_overload_threshold_enforcement` setting value which disables enforcement is updated to be spelled correctly as "ignore" rather than "ingore".

98600: server: change conn close error to warning r=knz,abarganier a=dhartunian

Resolves: #98523
Epic: None
Release note: None

Co-authored-by: Alex Barganier <[email protected]>
Co-authored-by: Jayant Shrivastava <[email protected]>
Co-authored-by: Austen McClernon <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
Co-authored-by: David Hartunian <[email protected]>
  • Loading branch information
6 people committed Mar 14, 2023
9 parents 7f2afbc + bb72aeb + 85d878c + 0d35a08 + 609f5cb + 0a62474 + e5e0895 + c80e7c7 + 75e6c31 commit a36d88b
Show file tree
Hide file tree
Showing 59 changed files with 281 additions and 228 deletions.
5 changes: 0 additions & 5 deletions pkg/base/serverident/server_ident.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,12 +104,7 @@ func GetIdentificationPayload(ctx context.Context) IDPayload {
}

// TenantID returns the tenant ID associated with this idPayload.
// if the idPayload has no tenant ID set, we default to the system
// tenant ID. NB: This function should never return an empty string.
func (ip IDPayload) TenantID() string {
if ip.TenantIDInternal == "" {
return SystemTenantID
}
return ip.TenantIDInternal
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/full_cluster_backup_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,7 @@ CREATE TABLE data2.foo (a int);
it.SeekGE(storage.MVCCKey{Key: startKey})
hasKey, err := it.Valid()
require.NoError(t, err)
require.False(t, hasKey, "did not expect to find a key, found %s", it.Key())
require.False(t, hasKey, "did not expect to find a key, found %s", it.UnsafeKey())
})

// Allow the restore to make progress after we've checked the pre-restore
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func slurpSSTablesLatestKey(
if err != nil {
t.Fatal(err)
}
kvs = append(kvs, storage.MVCCKeyValue{Key: it.Key(), Value: val.Value.RawBytes})
kvs = append(kvs, storage.MVCCKeyValue{Key: it.UnsafeKey().Clone(), Value: val.Value.RawBytes})
}
return kvs
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/replicationutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ func ScanSST(
return err
}
if err = mvccKeyValOp(storage.MVCCKeyValue{
Key: pointIter.Key(),
Key: pointIter.UnsafeKey().Clone(),
Value: v,
}); err != nil {
return err
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -329,18 +329,18 @@ func assertExactlyEqualKVs(
}
break
}
if maxKVTimestampSeen.Less(it.Key().Timestamp) {
maxKVTimestampSeen = it.Key().Timestamp
if maxKVTimestampSeen.Less(it.UnsafeKey().Timestamp) {
maxKVTimestampSeen = it.UnsafeKey().Timestamp
}
newKey := (prevKey != nil && !it.Key().Key.Equal(prevKey)) || prevKey == nil
prevKey = it.Key().Key
newKey := (prevKey != nil && !it.UnsafeKey().Key.Equal(prevKey)) || prevKey == nil
prevKey = it.UnsafeKey().Clone().Key

if newKey {
// All value ts should have been drained at this point, otherwise there is
// a mismatch between the streamed and ingested data.
require.Equal(t, 0, len(valueTimestampTuples))
valueTimestampTuples, err = streamValidator.getValuesForKeyBelowTimestamp(
string(it.Key().Key), frontierTimestamp)
string(it.UnsafeKey().Key), frontierTimestamp)
require.NoError(t, err)
}

Expand All @@ -357,10 +357,10 @@ func assertExactlyEqualKVs(
v, err := it.Value()
require.NoError(t, err)
require.Equal(t, roachpb.KeyValue{
Key: it.Key().Key,
Key: it.UnsafeKey().Key,
Value: roachpb.Value{
RawBytes: v,
Timestamp: it.Key().Timestamp,
Timestamp: it.UnsafeKey().Timestamp,
},
}, latestVersionInChain)
matchingKVs++
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -385,19 +385,19 @@ func assertEqualKVs(

// We only want to process MVCC KVs with a ts less than or equal to the max
// resolved ts for this partition.
if partitionTimestamp.Less(it.Key().Timestamp) {
if partitionTimestamp.Less(it.UnsafeKey().Timestamp) {
continue
}

newKey := (prevKey != nil && !it.Key().Key.Equal(prevKey)) || prevKey == nil
prevKey = it.Key().Key
newKey := (prevKey != nil && !it.UnsafeKey().Key.Equal(prevKey)) || prevKey == nil
prevKey = it.UnsafeKey().Clone().Key

if newKey {
// All value ts should have been drained at this point, otherwise there is
// a mismatch between the streamed and ingested data.
require.Equal(t, 0, len(valueTimestampTuples))
valueTimestampTuples, err = streamValidator.getValuesForKeyBelowTimestamp(
string(it.Key().Key), partitionTimestamp)
string(it.UnsafeKey().Key), partitionTimestamp)
require.NoError(t, err)
}

Expand All @@ -408,10 +408,10 @@ func assertEqualKVs(
v, err := it.Value()
require.NoError(t, err)
require.Equal(t, roachpb.KeyValue{
Key: it.Key().Key,
Key: it.UnsafeKey().Key,
Value: roachpb.Value{
RawBytes: v,
Timestamp: it.Key().Timestamp,
Timestamp: it.UnsafeKey().Timestamp,
},
}, latestVersionInChain)
// Truncate the latest version which we just checked against in preparation
Expand Down
13 changes: 7 additions & 6 deletions pkg/jobs/jobs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2025,19 +2025,20 @@ func TestShowAutomaticJobs(t *testing.T) {
}

for _, in := range rows {
// system.jobs is part proper SQL columns, part protobuf, so we can't use the
// row struct directly.
inPayload, err := protoutil.Marshal(&jobspb.Payload{
rawPayload := &jobspb.Payload{
UsernameProto: username.RootUserName().EncodeProto(),
Details: jobspb.WrapPayloadDetails(in.details),
})
}
// system.jobs is part proper SQL columns, part protobuf, so we can't use the
// row struct directly.
inPayload, err := protoutil.Marshal(rawPayload)
if err != nil {
t.Fatal(err)
}

sqlDB.Exec(t,
`INSERT INTO system.jobs (id, status, payload) VALUES ($1, $2, $3)`,
in.id, in.status, inPayload,
`INSERT INTO system.jobs (id, status, payload, job_type) VALUES ($1, $2, $3, $4)`,
in.id, in.status, inPayload, rawPayload.Type().String(),
)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvnemesis/validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -631,7 +631,7 @@ func (v *validator) processOp(op Operation) {
}
}

key := iter.Key().Key
key := iter.UnsafeKey().Clone().Key
rawValue, err := iter.Value()
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvnemesis/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ func (w *Watcher) handleSSTable(ctx context.Context, data []byte) error {
}

// Add point keys.
key := iter.Key()
key := iter.UnsafeKey().Clone()
rawValue, err := iter.Value()
if err != nil {
return err
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ var LeaseIOOverloadThresholdEnforcement = settings.RegisterEnumSetting(
"non io-overloaded stores, this is a superset of block_transfer_to",
"block_transfer_to",
map[int64]string{
int64(IOOverloadThresholdIgnore): "ingore",
int64(IOOverloadThresholdIgnore): "ignore",
int64(IOOverloadThresholdBlockTransfers): "block_transfer_to",
int64(IOOverloadThresholdShed): "shed",
},
Expand Down
40 changes: 20 additions & 20 deletions pkg/kv/kvserver/batch_spanset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,20 +171,20 @@ func TestSpanSetBatchBoundaries(t *testing.T) {
if ok, err := iter.Valid(); !ok || err != nil {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), insideKey) {
t.Fatalf("expected key %s, got %s", insideKey, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), insideKey) {
t.Fatalf("expected key %s, got %s", insideKey, iter.UnsafeKey())
}
iter.Next()
if ok, err := iter.Valid(); !ok || err != nil {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), insideKey2) {
t.Fatalf("expected key %s, got %s", insideKey2, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), insideKey2) {
t.Fatalf("expected key %s, got %s", insideKey2, iter.UnsafeKey())
}
// Scan out of bounds.
iter.Next()
if ok, err := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
} else if err != nil {
// Scanning out of bounds sets Valid() to false but is not an error.
t.Errorf("unexpected error on iterator: %+v", err)
Expand All @@ -210,20 +210,20 @@ func TestSpanSetBatchBoundaries(t *testing.T) {
if ok, err := iter.Valid(); !ok || err != nil {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), insideKey2) {
t.Fatalf("expected key %s, got %s", insideKey2, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), insideKey2) {
t.Fatalf("expected key %s, got %s", insideKey2, iter.UnsafeKey())
}
iter.Prev()
if ok, err := iter.Valid(); !ok || err != nil {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), insideKey) {
t.Fatalf("expected key %s, got %s", insideKey, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), insideKey) {
t.Fatalf("expected key %s, got %s", insideKey, iter.UnsafeKey())
}
// Scan out of bounds.
iter.Prev()
if ok, err := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
} else if err != nil {
t.Errorf("unexpected error on iterator: %+v", err)
}
Expand All @@ -236,7 +236,7 @@ func TestSpanSetBatchBoundaries(t *testing.T) {
// SeekLT to the lower bound is invalid.
iter.SeekLT(insideKey)
if ok, err := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
} else if !isReadSpanErr(err) {
t.Fatalf("SeekLT: unexpected error %v", err)
}
Expand Down Expand Up @@ -390,16 +390,16 @@ func TestSpanSetIteratorTimestamps(t *testing.T) {
if ok, err := iter.Valid(); !ok {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), k1) {
t.Fatalf("expected key %s, got %s", k1, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), k1) {
t.Fatalf("expected key %s, got %s", k1, iter.UnsafeKey())
}

iter.Next()
if ok, err := iter.Valid(); !ok {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), k2) {
t.Fatalf("expected key %s, got %s", k2, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), k2) {
t.Fatalf("expected key %s, got %s", k2, iter.UnsafeKey())
}
}()

Expand All @@ -410,15 +410,15 @@ func TestSpanSetIteratorTimestamps(t *testing.T) {

iter.SeekGE(k1)
if ok, _ := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
}

iter.SeekGE(k2)
if ok, err := iter.Valid(); !ok {
t.Fatalf("expected valid iterator, err=%v", err)
}
if !reflect.DeepEqual(iter.Key(), k2) {
t.Fatalf("expected key %s, got %s", k2, iter.Key())
if !reflect.DeepEqual(iter.UnsafeKey(), k2) {
t.Fatalf("expected key %s, got %s", k2, iter.UnsafeKey())
}
}()

Expand All @@ -430,12 +430,12 @@ func TestSpanSetIteratorTimestamps(t *testing.T) {

iter.SeekGE(k1)
if ok, _ := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
}

iter.SeekGE(k2)
if ok, _ := iter.Valid(); ok {
t.Fatalf("expected invalid iterator; found valid at key %s", iter.Key())
t.Fatalf("expected invalid iterator; found valid at key %s", iter.UnsafeKey())
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -388,7 +388,7 @@ func EvalAddSSTable(
if ok, err := existingIter.Valid(); err != nil {
return result.Result{}, errors.Wrap(err, "error while searching for non-empty span start")
} else if ok {
reply.FollowingLikelyNonEmptySpanStart = existingIter.Key().Key
reply.FollowingLikelyNonEmptySpanStart = existingIter.UnsafeKey().Key.Clone()
}
}

Expand Down
24 changes: 9 additions & 15 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,9 +393,9 @@ func TestStoreRangeSplitIntents(t *testing.T) {
break
}

if bytes.HasPrefix([]byte(iter.Key().Key), txnPrefix(roachpb.KeyMin)) ||
bytes.HasPrefix([]byte(iter.Key().Key), txnPrefix(splitKey)) {
t.Errorf("unexpected system key: %s; txn record should have been cleaned up", iter.Key())
if bytes.HasPrefix([]byte(iter.UnsafeKey().Key), txnPrefix(roachpb.KeyMin)) ||
bytes.HasPrefix([]byte(iter.UnsafeKey().Key), txnPrefix(splitKey)) {
t.Errorf("unexpected system key: %s; txn record should have been cleaned up", iter.UnsafeKey())
}
}
}
Expand Down Expand Up @@ -2776,7 +2776,6 @@ func TestTxnWaitQueueDependencyCycleWithRangeSplit(t *testing.T) {
func TestStoreCapacityAfterSplit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.WithIssue(t, 92677)
ctx := context.Background()
manualClock := hlc.NewHybridManualClock()
tc := testcluster.StartTestCluster(t, 2,
Expand All @@ -2800,16 +2799,6 @@ func TestStoreCapacityAfterSplit(t *testing.T) {
key := tc.ScratchRange(t)
desc := tc.AddVotersOrFatal(t, key, tc.Target(1))
tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1))
testutils.SucceedsSoon(t, func() error {
repl, err := s.GetReplica(desc.RangeID)
if err != nil {
return err
}
if !repl.OwnsValidLease(ctx, tc.Servers[1].Clock().NowAsClockTimestamp()) {
return errors.New("s2 does not own valid lease for this range")
}
return nil
})

tc.IncrClockForLeaseUpgrade(t, manualClock)
tc.WaitForLeaseUpgrade(ctx, t, desc)
Expand Down Expand Up @@ -2851,6 +2840,11 @@ func TestStoreCapacityAfterSplit(t *testing.T) {
return nil
})

// Bump the clock again, right before calling capacity. We know that the
// writes have succeeded and should be reflected in Capacity, however the
// MinStatsDuration will cause nothing to be returned unless the last lease
// transfer is at least MinStatsDuration ago.
manualClock.Increment(int64(replicastats.MinStatsDuration))
cap, err = s.Capacity(ctx, false /* useCached */)
if err != nil {
t.Fatal(err)
Expand All @@ -2865,7 +2859,7 @@ func TestStoreCapacityAfterSplit(t *testing.T) {
// NB: The writes per second may be within some error bound below the
// minExpected due to timing and floating point calculation. An error of
// 0.01 (WPS) is added to avoid flaking the test.
if minExpected, a := 1/float64(replicastats.MinStatsDuration/time.Second), cap.WritesPerSecond; minExpected > a+0.01 {
if minExpected, a := 1/(float64(2*replicastats.MinStatsDuration/time.Second)), cap.WritesPerSecond; minExpected > a+0.01 {
t.Errorf("expected cap.WritesPerSecond >= %f, got %f", minExpected, a)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/gc/gc_old_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,13 +187,13 @@ func runGCOld(
// Stop iterating if our context has expired.
return Info{}, err
}
iterKey := iter.Key()
iterKey := iter.UnsafeKey().Clone()
if !iterKey.IsValue() || !iterKey.Key.Equal(expBaseKey) {
// Moving to the next key (& values).
processKeysAndValues()
expBaseKey = iterKey.Key
if !iterKey.IsValue() {
keys = []storage.MVCCKey{iter.Key()}
keys = []storage.MVCCKey{iter.UnsafeKey().Clone()}
v, err := iter.Value()
if err != nil {
return Info{}, err
Expand All @@ -212,7 +212,7 @@ func runGCOld(
if err != nil {
return Info{}, err
}
keys = append(keys, iter.Key())
keys = append(keys, iter.UnsafeKey().Clone())
vals = append(vals, v)
}
// Handle last collected set of keys/vals.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/gc_random_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,7 @@ func getExpectationsGenerator(
}
p, r := it.HasPointAndRange()
if p {
k := it.Key()
k := it.UnsafeKey().Clone()
v, err := it.Value()
require.NoError(t, err)
if len(baseKey) == 0 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1534,7 +1534,7 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) []
v = prefix + string(b)
}
result = append(result, tableCell{
key: it.Key(),
key: it.UnsafeKey().Clone(),
value: v,
})
prefix = ""
Expand Down
Loading

0 comments on commit a36d88b

Please sign in to comment.