Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
74602: kvserver: add `WholeRowsOfSize` request parameter for scans r=erikgrinaker a=erikgrinaker

**roachpb: rename `TargetBytesAllowEmpty` to `AllowEmpty`**

The `TargetBytesAllowEmpty` parameter currently only applies to
`TargetBytes` limits, but will shortly apply to `MaxSpanRequestKeys` as
well (when the scan API becomes SQL row-aware and may discard the first
row if it's incomplete).

This patch therefore renames the parameter to `AllowEmpty`, to make it
limit-agnostic. There are no behavioral changes.

Release note: None

**storage: add benchmark support for SQL row data**

Release note: None

**kvserver: add `WholeRowsOfSize` request parameter for scans**

This patch adds a `WholeRowsOfSize` parameter to request headers, and a
corresponding `ScanWholeRows` version gate, which prevents scan requests
from including partial SQL rows at the end of the result. See
`api.proto` for details.

This implementation requires the maximum row size to be plumbed down
from SQL. This is a performance optimization which allows use of a
fixed-size ring buffer to keep track of the last KV byte offsets in
`pebbleMVCCScanner`. Two alternative implementations were attempted:
keeping track of all KV byte offsets while scanning, and decoding every
key's row prefix while scanning to track the last row's starting offset.
Both of these alternatives predictably had significant overhead (~10%).

However, this patch comes with a 2-3% performance penalty for scans even
when `WholeRowsOfSize` is disabled, due to code restructuring and
additional checks. Efforts to recover this performance have stalled.

```
name                                                    old time/op    new time/op    delta
MVCCScan_Pebble/rows=1/versions=1/valueSize=64-24         4.25µs ± 1%    4.26µs ± 1%    ~     (p=0.406 n=16+16)
MVCCScan_Pebble/rows=1/versions=10/valueSize=64-24        6.10µs ± 1%    6.13µs ± 1%    ~     (p=0.065 n=16+14)
MVCCScan_Pebble/rows=100/versions=1/valueSize=64-24       36.4µs ± 1%    37.3µs ± 1%  +2.57%  (p=0.000 n=16+15)
MVCCScan_Pebble/rows=100/versions=10/valueSize=64-24       122µs ± 2%     123µs ± 1%    ~     (p=0.068 n=16+13)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64-24     2.29ms ± 1%    2.36ms ± 1%  +3.08%  (p=0.000 n=16+16)
MVCCScan_Pebble/rows=10000/versions=10/valueSize=64-24    9.15ms ± 1%    9.22ms ± 1%  +0.75%  (p=0.000 n=16+16)
```

Enabling `WholeRowsOfSize` has an additional 0-1.5% penalty for large
scans, as seen below. However, notice that single-key scans with
`versions=10` can have penalties as high as 7%. This is because rows may
omit column families that are all `NULL`, so in order to know whether
the single key is a complete row or not the scan must continue to the
next key. This additional scan can be relatively expensive for
single-key scans if there is a lot of MVCC garbage. However, if the last
row contains the final column family when the limit is hit, this penalty
can be avoided: compare how `rows=3/columnFamilies=3` and
`rows=1/columnFamilies=3` change from `versions=1` to `versions=10`.

```
name                                                                             old time/op    new time/op    delta
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=1/versions=1/valueSize=64-24          4.38µs ± 1%    4.40µs ± 1%     ~     (p=0.075 n=16+14)
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=1/versions=10/valueSize=64-24         6.22µs ± 1%    6.18µs ± 1%   -0.66%  (p=0.004 n=14+15)
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=3/versions=1/valueSize=64-24          4.39µs ± 1%    4.33µs ± 1%   -1.20%  (p=0.000 n=16+16)
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=3/versions=10/valueSize=64-24         6.16µs ± 1%    6.54µs ± 1%   +6.12%  (p=0.000 n=15+16)
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=10/versions=1/valueSize=64-24         4.38µs ± 1%    4.35µs ± 1%   -0.64%  (p=0.000 n=16+14)
MVCCScan_PebbleSQLRows/rows=1/columnFamilies=10/versions=10/valueSize=64-24        6.08µs ± 1%    6.50µs ± 1%   +6.88%  (p=0.000 n=16+16)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=1/versions=1/valueSize=64-24          5.35µs ± 2%    5.36µs ± 1%     ~     (p=0.277 n=16+15)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=1/versions=10/valueSize=64-24         9.33µs ± 1%    9.33µs ± 1%     ~     (p=0.904 n=16+16)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=3/versions=1/valueSize=64-24          5.34µs ± 1%    5.46µs ± 1%   +2.14%  (p=0.000 n=14+16)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=3/versions=10/valueSize=64-24         9.32µs ± 2%    9.45µs ± 2%   +1.41%  (p=0.000 n=16+16)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=10/versions=1/valueSize=64-24         5.32µs ± 1%    5.18µs ± 1%   -2.50%  (p=0.000 n=16+15)
MVCCScan_PebbleSQLRows/rows=3/columnFamilies=10/versions=10/valueSize=64-24        9.22µs ± 1%    9.45µs ± 1%   +2.54%  (p=0.000 n=16+15)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=1/versions=1/valueSize=64-24        35.9µs ± 1%    35.7µs ± 1%   -0.70%  (p=0.001 n=15+15)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=1/versions=10/valueSize=64-24        117µs ± 1%     117µs ± 1%     ~     (p=0.323 n=16+16)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=3/versions=1/valueSize=64-24        36.7µs ± 1%    37.0µs ± 1%   +0.82%  (p=0.000 n=15+16)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=3/versions=10/valueSize=64-24        118µs ± 1%     119µs ± 1%   +0.89%  (p=0.000 n=16+16)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=10/versions=1/valueSize=64-24       35.9µs ± 0%    36.3µs ± 2%   +1.26%  (p=0.000 n=13+15)
MVCCScan_PebbleSQLRows/rows=100/columnFamilies=10/versions=10/valueSize=64-24       116µs ± 1%     117µs ± 1%   +0.77%  (p=0.000 n=15+16)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=1/versions=1/valueSize=64-24      2.41ms ± 1%    2.41ms ± 1%     ~     (p=0.094 n=16+16)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=1/versions=10/valueSize=64-24     9.11ms ± 1%    9.10ms ± 1%     ~     (p=0.822 n=14+16)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=3/versions=1/valueSize=64-24      2.52ms ± 1%    2.53ms ± 1%     ~     (p=0.591 n=14+15)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=3/versions=10/valueSize=64-24     9.28ms ± 1%    9.33ms ± 1%   +0.54%  (p=0.010 n=16+16)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=10/versions=1/valueSize=64-24     2.44ms ± 1%    2.46ms ± 1%   +0.96%  (p=0.000 n=15+13)
MVCCScan_PebbleSQLRows/rows=10000/columnFamilies=10/versions=10/valueSize=64-24    9.13ms ± 1%    9.18ms ± 1%   +0.59%  (p=0.006 n=15+16)
```

Resolves #73618.

Release note: None

75115: sql: fix call of `FindIndexWithName` r=postamar a=otan

In e89093f, we added
`tableDesc.FindIndexWithName(name.String())`. However, `name.String()`
can return a `EncodeRestrictedSQLIdent` version of the string.

This fixes the call to use `string(name)`.

I couldn't produce an error with this at the moment, mostly because
there is some other check preventing it from happening from the
combinations I've tried.

Release note: None

75139: kvserver: add `SSTTimestamp` parameter for `AddSSTable` r=dt a=erikgrinaker

This patch adds an `SSTTimestamp` parameter for `AddSSTable`. When set,
the client promises that all MVCC timestamps in the given SST are equal
to `SSTTimestamp`. When used together with `WriteAtRequestTimestamp`,
this can avoid the cost of rewriting the SST timestamps if the
`SSTTimestamp` already equals the request `Timestamp`.

Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Oliver Tan <otan@cockroachlabs.com>
  • Loading branch information
3 people committed Jan 20, 2022
4 parents 5359e54 + 6a7678c + 93b20e7 + d47f932 commit ebda0ec
Show file tree
Hide file tree
Showing 27 changed files with 1,036 additions and 142 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -170,4 +170,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen
trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 21.2-42 set the active cluster version in the format '<major>.<minor>'
version version 21.2-44 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,6 @@
<tr><td><code>trace.jaeger.agent</code></td><td>string</td><td><code></code></td><td>the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.</td></tr>
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-42</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-44</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,9 @@ const (
// EnableSpanConfigStore enables the use of the span configs infrastructure
// in KV.
EnableSpanConfigStore
// ScanWholeRows is the version at which the Header.WholeRowsOfSize parameter
// was introduced, preventing limited scans from returning partial rows.
ScanWholeRows

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -359,6 +362,10 @@ var versionsSingleton = keyedVersions{
Key: EnableSpanConfigStore,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 42},
},
{
Key: ScanWholeRows,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 44},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

21 changes: 21 additions & 0 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package keys
import (
"bytes"
"fmt"
"math"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -776,6 +777,26 @@ func MakeFamilyKey(key []byte, famID uint32) []byte {
return encoding.EncodeUvarintAscending(key, uint64(len(key)-size))
}

// DecodeFamilyKey returns the family ID in the given row key. Returns an error
// if the key does not contain a family ID.
func DecodeFamilyKey(key []byte) (uint32, error) {
n, err := GetRowPrefixLength(key)
if err != nil {
return 0, err
}
if n <= 0 || n >= len(key) {
return 0, errors.Errorf("invalid row prefix, got prefix length %d for key %s", n, key)
}
_, colFamilyID, err := encoding.DecodeUvarintAscending(key[n:])
if err != nil {
return 0, err
}
if colFamilyID > math.MaxUint32 {
return 0, errors.Errorf("column family ID overflow, got %d", colFamilyID)
}
return uint32(colFamilyID), nil
}

// DecodeTableIDIndexID decodes a table id followed by an index id from the
// provided key. The input key must already have its tenant id removed.
func DecodeTableIDIndexID(key []byte) ([]byte, uint32, uint32, error) {
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -1369,11 +1369,8 @@ func (ds *DistSender) divideAndSendBatchToRanges(
// might be passed recursively to further divideAndSendBatchToRanges()
// calls.
if ba.MaxSpanRequestKeys > 0 {
if replyKeys > ba.MaxSpanRequestKeys {
log.Fatalf(ctx, "received %d results, limit was %d", replyKeys, ba.MaxSpanRequestKeys)
}
ba.MaxSpanRequestKeys -= replyKeys
if ba.MaxSpanRequestKeys == 0 {
if ba.MaxSpanRequestKeys <= 0 {
couldHaveSkippedResponses = true
resumeReason = roachpb.RESUME_KEY_LIMIT
return
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvstreamer/streamer.go
Original file line number Diff line number Diff line change
Expand Up @@ -901,7 +901,7 @@ func (w *workerCoordinator) performRequestAsync(
var ba roachpb.BatchRequest
ba.Header.WaitPolicy = w.lockWaitPolicy
ba.Header.TargetBytes = targetBytes
ba.Header.TargetBytesAllowEmpty = !headOfLine
ba.Header.AllowEmpty = !headOfLine
// TODO(yuzefovich): consider setting MaxSpanRequestKeys whenever
// applicable (#67885).
ba.AdmissionHeader = w.requestAdmissionHeader
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -133,13 +133,18 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
"//pkg/sql/rowenc",
"//pkg/sql/sem/tree",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand Down
38 changes: 36 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -56,8 +57,15 @@ func EvalAddSSTable(
// If requested, rewrite the SST's MVCC timestamps to the request timestamp.
// This ensures the writes comply with the timestamp cache and closed
// timestamp, i.e. by not writing to timestamps that have already been
// observed or closed.
if args.WriteAtRequestTimestamp {
// observed or closed. If the race detector is enabled, also assert that
// the provided SST only contains the expected timestamps.
if util.RaceEnabled && !args.SSTTimestamp.IsEmpty() {
if err := assertSSTTimestamp(sst, args.SSTTimestamp); err != nil {
return result.Result{}, err
}
}
if args.WriteAtRequestTimestamp &&
(args.SSTTimestamp.IsEmpty() || h.Timestamp != args.SSTTimestamp) {
sst, err = storage.UpdateSSTTimestamps(sst, h.Timestamp)
if err != nil {
return result.Result{}, errors.Wrap(err, "updating SST timestamps")
Expand Down Expand Up @@ -260,3 +268,29 @@ func EvalAddSSTable(
},
}, nil
}

func assertSSTTimestamp(sst []byte, ts hlc.Timestamp) error {
iter, err := storage.NewMemSSTIterator(sst, true)
if err != nil {
return err
}
defer iter.Close()

iter.SeekGE(storage.MVCCKey{Key: keys.MinKey})
for {
ok, err := iter.Valid()
if err != nil {
return err
}
if !ok {
return nil
}

key := iter.UnsafeKey()
if key.Timestamp != ts {
return errors.AssertionFailedf("incorrect timestamp %s for SST key %s (expected %s)",
key.Timestamp, key.Key, ts)
}
iter.Next()
}
}
57 changes: 42 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -70,15 +71,17 @@ func TestEvalAddSSTable(t *testing.T) {

// These are run with IngestAsWrites both disabled and enabled.
testcases := map[string]struct {
data []mvccKV
sst []mvccKV
atReqTS int64 // WriteAtRequestTimestamp with given timestamp
noConflict bool // DisallowConflicts
noShadow bool // DisallowShadowing
noShadowBelow int64 // DisallowShadowingBelow
expect []mvccKV
expectErr interface{} // error type, substring, or true (any error)
expectStatsEst bool // expect MVCCStats.ContainsEstimates, don't check stats
data []mvccKV
sst []mvccKV
sstTimestamp int64 // SSTTimestamp set to given timestamp
atReqTS int64 // WriteAtRequestTimestamp with given timestamp
noConflict bool // DisallowConflicts
noShadow bool // DisallowShadowing
noShadowBelow int64 // DisallowShadowingBelow
expect []mvccKV
expectErr interface{} // error type, substring, or true (any error)
expectErrUnderRace interface{}
expectStatsEst bool // expect MVCCStats.ContainsEstimates, don't check stats
}{
// Blind writes.
"blind writes below existing": {
Expand Down Expand Up @@ -610,6 +613,25 @@ func TestEvalAddSSTable(t *testing.T) {
sst: []mvccKV{{"a", 7, "a8"}},
expectErr: &roachpb.WriteTooOldError{},
},

// SSTTimestamp
"SSTTimestamp works with WriteAtRequestTimestamp": {
atReqTS: 7,
data: []mvccKV{{"a", 6, "a6"}},
sst: []mvccKV{{"a", 7, "a7"}},
sstTimestamp: 7,
expect: []mvccKV{{"a", 7, "a7"}, {"a", 6, "a6"}},
expectStatsEst: true,
},
"SSTTimestamp doesn't rewrite with incorrect timestamp, but errors under race": {
atReqTS: 8,
data: []mvccKV{{"a", 6, "a6"}},
sst: []mvccKV{{"a", 7, "a7"}},
sstTimestamp: 8,
expect: []mvccKV{{"a", 7, "a7"}, {"a", 6, "a6"}},
expectErrUnderRace: `incorrect timestamp 0.000000007,0 for SST key "a" (expected 0.000000008,0)`,
expectStatsEst: true,
},
}
testutils.RunTrueAndFalse(t, "IngestAsWrites", func(t *testing.T, ingestAsWrites bool) {
for name, tc := range testcases {
Expand Down Expand Up @@ -657,20 +679,25 @@ func TestEvalAddSSTable(t *testing.T) {
DisallowShadowing: tc.noShadow,
DisallowShadowingBelow: hlc.Timestamp{WallTime: tc.noShadowBelow},
WriteAtRequestTimestamp: tc.atReqTS != 0,
SSTTimestamp: hlc.Timestamp{WallTime: tc.sstTimestamp},
IngestAsWrites: ingestAsWrites,
},
}, resp)

if tc.expectErr != nil {
expectErr := tc.expectErr
if expectErr == nil && tc.expectErrUnderRace != nil && util.RaceEnabled {
expectErr = tc.expectErrUnderRace
}
if expectErr != nil {
require.Error(t, err)
if b, ok := tc.expectErr.(bool); ok && b {
if b, ok := expectErr.(bool); ok && b {
// any error is fine
} else if expectMsg, ok := tc.expectErr.(string); ok {
} else if expectMsg, ok := expectErr.(string); ok {
require.Contains(t, err.Error(), expectMsg)
} else if expectErr, ok := tc.expectErr.(error); ok {
require.True(t, errors.HasType(err, expectErr), "expected %T, got %v", expectErr, err)
} else if e, ok := expectErr.(error); ok {
require.True(t, errors.HasType(err, e), "expected %T, got %v", e, err)
} else {
require.Fail(t, "invalid expectErr", "expectErr=%v", tc.expectErr)
require.Fail(t, "invalid expectErr", "expectErr=%v", expectErr)
}
return
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func Get(
// NB: This calculation is different from Scan, since Scan responses include
// the key/value pair while Get only includes the value.
numBytes := int64(len(val.RawBytes))
if h.TargetBytes > 0 && h.TargetBytesAllowEmpty && numBytes > h.TargetBytes {
if h.TargetBytes > 0 && h.AllowEmpty && numBytes > h.TargetBytes {
reply.ResumeSpan = &roachpb.Span{Key: args.Key}
reply.ResumeReason = roachpb.RESUME_BYTE_LIMIT
reply.ResumeNextBytes = numBytes
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,9 +93,9 @@ func TestGetResumeSpan(t *testing.T) {
_, err := Get(ctx, db, CommandArgs{
EvalCtx: (&MockEvalCtx{ClusterSettings: settings}).EvalContext(),
Header: roachpb.Header{
MaxSpanRequestKeys: tc.maxKeys,
TargetBytes: tc.targetBytes,
TargetBytesAllowEmpty: tc.allowEmpty,
MaxSpanRequestKeys: tc.maxKeys,
TargetBytes: tc.targetBytes,
AllowEmpty: tc.allowEmpty,
},
Args: &roachpb.GetRequest{
RequestHeader: roachpb.RequestHeader{Key: key},
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_reverse_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,8 +48,9 @@ func ReverseScan(
MaxKeys: h.MaxSpanRequestKeys,
MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV),
TargetBytes: h.TargetBytes,
TargetBytesAvoidExcess: h.TargetBytesAllowEmpty || avoidExcess, // AllowEmpty takes precedence
TargetBytesAllowEmpty: h.TargetBytesAllowEmpty,
TargetBytesAvoidExcess: h.AllowEmpty || avoidExcess, // AllowEmpty takes precedence
AllowEmpty: h.AllowEmpty,
WholeRowsOfSize: h.WholeRowsOfSize,
FailOnMoreRecent: args.KeyLocking != lock.None,
Reverse: true,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,9 @@ func Scan(
MaxKeys: h.MaxSpanRequestKeys,
MaxIntents: storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV),
TargetBytes: h.TargetBytes,
TargetBytesAvoidExcess: h.TargetBytesAllowEmpty || avoidExcess, // AllowEmpty takes precedence
TargetBytesAllowEmpty: h.TargetBytesAllowEmpty,
TargetBytesAvoidExcess: h.AllowEmpty || avoidExcess, // AllowEmpty takes precedence
AllowEmpty: h.AllowEmpty,
WholeRowsOfSize: h.WholeRowsOfSize,
FailOnMoreRecent: args.KeyLocking != lock.None,
Reverse: false,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
Expand Down
Loading

0 comments on commit ebda0ec

Please sign in to comment.