Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
66362: kv,storage: use a BytesMonitor to track memory allocations for scans r=sumeerbhola a=sumeerbhola

The goal is to track memory allocations for:
- non-local SQL=>KV requests: this can happen with joins,
  multi-tenant clusters, and if ranges move between DistSQL
  planning and execution.
- local SQL=>KV requests for the first request by a fetcher:
  in this case the fetcher reserves a modest 1KB which can
  be significantly exceeded by KV allocations.

Only allocations in pebbleMVCCScanner for kv pairs and intents are
tracked. The memory is released when returning from
executeReadOnlyBatchWithServersideRefreshes since the chain
of returns will end up in gRPC response processing and we can't
hook into where that memory is released. This should still help
for some cases of OOMs, and give some signal of memory overload
that we can use elsewhere (e.g. admission control).

The BytesMonitor is used to construct a BoundAccount that is
wrapped in a narrower ScannerMemoryMonitor that is passed via the
EvalContext interface. The other alternative would be for the
engine to have a BytesMonitor at initialization time that it
can use to construct a BoundAccount for each MVCC scan, and
pass it back via MVCCScanResult. This would mean multiple
BoundAccounts for a batch (since we don't want to release memory
until all the requests in the batch are processed), and would
be harder to extend to track additional request types compared
to embedding in EvalContext.

The rootSQLMonitor is reused for this memory allocation tracking.
This tracking is always done for non-local requests, and for the
first request by a fetcher for a local request. This is to
avoid double-counting, the first request issued by a SQL fetcher
only reserves 1KB, but subsequent ones have already reserved
what was returned in the first response. So there is room to
tighten this if we knew what had been reserved on the local
client (there are complications because the batch may have
been split to send to different nodes, only one of which was
local).

The AdmissionHeader.SourceLocation field is used to mark local
requests and is set in rpc.internalClientAdapter. The first
request is marked using the
AdmissionHeader.NoMemoryReservedAtSource bit.

Informs #19721

Release note (ops change): The memory pool used for SQL is now
also used to cover KV memory used for scans.

67451: colexec: optimize Bytes sorting with abbreviated values r=mgartner a=mgartner

#### colexec: add benchmark for sorting UUIDs

Release note: None

#### colexec: optimize Bytes sorting with abbreviated values

This commit introduces an optimization for sorting `Bytes` vectors.
While sorting, a signification potion of time is spent performing
comparisons between two datums. These comparisons are optimized by
creating an abbreviated `uint64` value for each `[]byte` that represents
up to the first eight bytes in the slice. The abbreviated value is used
for a comparison fast path that avoid comparing all bytes in a slice in
some cases. This technique is used by Postgres[1][2] and Pebble[3].

Abbreviating values to a `uint64` helps optimize comparisons for datums
that are larger than the size of the system's native pointer (64 bits).
Given datums `a` and `b`, the following properties must hold true for
the respective abbreviated values `abbrA` and `abbrB`:

  - `abbrA > abbrB` iff `a > b`
  - `abbrA < abbrB` iff `a < b`
  - If `abbrA == abbrB`, it is unknown if `a` is greater than, less
    than, or equal to `b`. A full comparison of `a` and `b` is required.

Comparing the abbreviated values first improves comparison performance
for two reasons. First, comparing two `uint64`s is fast. It is a single
CPU instruction. Any datum larger than 64 bits requires multiple
instructions for comparison. For example, comparing two `[]byte`s
requires iterating over each byte until non-equal bytes are found.
Second, CPU caches are more efficiently used because the abbreviated
values of a vector are packed into a `[]uint64` of contiguous memory.
This increases the chance that two datums can be compared by only
fetching information from CPU caches rather than main memory.

In the benchmarks below, `rows` is the number of rows being sorted,
`cols` is the number of sort columns, and `constAbbrPct` is the
percentage of rows for which the abbreviated values are the same. The
benchmarks show that in the best case, when the abbreviated values are
all unique, sort throughput has increased by up to ~45%. In the worst
case, where all values share the same abbreviated value, throughput has
decreased by as much as ~8.5%. This is due to the extra overhead of
creating abbreviated values and comparing them, only to have to fall
back to a full comparison every time.

    name                                             old speed      new speed      delta
    SortUUID/rows=2048/cols=1/constAbbrPct=0-16      38.0MB/s ± 2%  55.3MB/s ± 3%  +45.30%
    SortUUID/rows=2048/cols=1/constAbbrPct=50-16     36.2MB/s ± 3%  43.0MB/s ± 4%  +18.82%
    SortUUID/rows=2048/cols=1/constAbbrPct=75-16     36.0MB/s ± 4%  37.1MB/s ± 4%     ~
    SortUUID/rows=2048/cols=1/constAbbrPct=90-16     36.8MB/s ± 1%  36.2MB/s ± 3%     ~
    SortUUID/rows=2048/cols=1/constAbbrPct=100-16    37.0MB/s ± 1%  33.8MB/s ± 5%   -8.66%
    SortUUID/rows=2048/cols=2/constAbbrPct=0-16      60.3MB/s ± 1%  74.0MB/s ± 3%  +22.85%
    SortUUID/rows=2048/cols=2/constAbbrPct=50-16     58.7MB/s ± 1%  63.7MB/s ± 1%   +8.54%
    SortUUID/rows=2048/cols=2/constAbbrPct=75-16     58.3MB/s ± 1%  56.6MB/s ± 6%     ~
    SortUUID/rows=2048/cols=2/constAbbrPct=90-16     58.7MB/s ± 1%  54.2MB/s ± 3%   -7.69%
    SortUUID/rows=2048/cols=2/constAbbrPct=100-16    59.3MB/s ± 1%  56.1MB/s ± 4%   -5.30%
    SortUUID/rows=16384/cols=1/constAbbrPct=0-16     30.7MB/s ± 2%  41.9MB/s ± 3%  +36.24%
    SortUUID/rows=16384/cols=1/constAbbrPct=50-16    30.6MB/s ± 1%  32.0MB/s ± 8%     ~
    SortUUID/rows=16384/cols=1/constAbbrPct=75-16    30.2MB/s ± 2%  30.9MB/s ± 6%     ~
    SortUUID/rows=16384/cols=1/constAbbrPct=90-16    30.4MB/s ± 2%  27.6MB/s ± 3%   -9.29%
    SortUUID/rows=16384/cols=1/constAbbrPct=100-16   30.6MB/s ± 2%  28.5MB/s ± 5%   -6.98%
    SortUUID/rows=16384/cols=2/constAbbrPct=0-16     49.5MB/s ± 2%  59.0MB/s ± 2%  +19.13%
    SortUUID/rows=16384/cols=2/constAbbrPct=50-16    48.6MB/s ± 1%  49.5MB/s ± 3%   +1.84%
    SortUUID/rows=16384/cols=2/constAbbrPct=75-16    47.3MB/s ± 2%  47.3MB/s ± 2%     ~
    SortUUID/rows=16384/cols=2/constAbbrPct=90-16    48.5MB/s ± 2%  45.2MB/s ± 1%   -6.65%
    SortUUID/rows=16384/cols=2/constAbbrPct=100-16   48.9MB/s ± 1%  44.4MB/s ± 2%   -9.13%
    SortUUID/rows=262144/cols=1/constAbbrPct=0-16    25.4MB/s ± 2%  35.7MB/s ± 3%  +40.53%
    SortUUID/rows=262144/cols=1/constAbbrPct=50-16   24.9MB/s ± 3%  28.8MB/s ± 3%  +15.44%
    SortUUID/rows=262144/cols=1/constAbbrPct=75-16   25.4MB/s ± 2%  25.7MB/s ± 5%     ~
    SortUUID/rows=262144/cols=1/constAbbrPct=90-16   25.4MB/s ± 3%  24.7MB/s ± 2%   -3.00%
    SortUUID/rows=262144/cols=1/constAbbrPct=100-16  25.1MB/s ± 3%  23.7MB/s ± 2%   -5.73%
    SortUUID/rows=262144/cols=2/constAbbrPct=0-16    37.5MB/s ± 2%  43.7MB/s ± 5%  +16.65%
    SortUUID/rows=262144/cols=2/constAbbrPct=50-16   37.5MB/s ± 1%  37.8MB/s ± 7%     ~
    SortUUID/rows=262144/cols=2/constAbbrPct=75-16   37.1MB/s ± 4%  36.4MB/s ± 1%     ~
    SortUUID/rows=262144/cols=2/constAbbrPct=90-16   36.9MB/s ± 5%  34.6MB/s ± 7%   -6.26%
    SortUUID/rows=262144/cols=2/constAbbrPct=100-16  37.2MB/s ± 3%  34.0MB/s ± 2%   -8.53%

I experimented with several mitigations to the regressions, but did not
have much luck. Postgres uses HyperLogLog while building abbreviated
values to track their cardinality and abort the optimization if the
cardinality is too low. I attempted this, but the overhead of
HyperLogLog negated the benefits of the optimization entirely.

Using a `map[uint64]struct{}` to track the cardinality of abbreviated
values was the most promising. It was able to reduce the worst
regressions at the expense of reducing the speedup of the best case
benchmarks.

These mechanisms for aborting the optimization when abbreviated value
cardinality is low are essentially a trade-off between maximizing sort
performance when their cardinality is high and minimizing the overhead
of generating and comparing abbreviated values when their cardinality is
low. It's worth future investigation to try to find a low-overhead
method of aborting, and to be more thoughtful about the trade-offs.

[1] https://brandur.org/sortsupport
[2] http://pgeoghegan.blogspot.com/2015/01/abbreviated-keys-exploiting-locality-to.html
[3] https://github.com/cockroachdb/pebble/blob/ea60b4722cca21fa0d3c2749c788e1ac76981f7d/internal/base/comparer.go#L28-L36

Release note (performance improvement): Sort performance has been
improved when sorting columns of type STRING, BYTES, or UUID.


67606: sql: add sql shell telemetry counter r=rafiss,knz,arulajmani a=e-mbrown

Fixes #62208

This change allows us to keep track of
connections made using our internal SQl shell.

Release note: None

68427: jobs: skip TestRegistryLifecycle r=adityamaru a=jbowens

Refs: #68315

Reason: flaky test

Generated by bin/skip-test.

Release justification: non-production code changes

Release note: None

Co-authored-by: sumeerbhola <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: e-mbrown <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
  • Loading branch information
5 people committed Aug 4, 2021
5 parents 8455178 + a5708ab + 1bfa1a6 + 45ca79e + 830a67f commit 1443ced
Show file tree
Hide file tree
Showing 42 changed files with 2,355 additions and 755 deletions.
3 changes: 2 additions & 1 deletion pkg/cli/sql_shell_cmd.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/cli/clisqlshell"
"github.com/cockroachdb/cockroach/pkg/server/pgurl"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
)
Expand Down Expand Up @@ -51,7 +52,7 @@ func runTerm(cmd *cobra.Command, args []string) (resErr error) {
fmt.Print(welcomeMessage)
}

conn, err := makeSQLClient("cockroach sql", useDefaultDb)
conn, err := makeSQLClient(catconstants.InternalSQLAppName, useDefaultDb)
if err != nil {
return err
}
Expand Down
46 changes: 46 additions & 0 deletions pkg/col/coldata/bytes.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package coldata

import (
"encoding/binary"
"fmt"
"strings"
"unsafe"
Expand Down Expand Up @@ -399,6 +400,51 @@ func (b *Bytes) ProportionalSize(n int64) int64 {
return FlatBytesOverhead + int64(len(b.data[b.offsets[0]:b.offsets[n]])) + n*memsize.Int32
}

// Abbreviated returns a uint64 slice where each uint64 represents the first
// eight bytes of each []byte. It is used for byte comparison fast paths.
//
// Given Bytes b, and abbr = b.Abbreviated():
//
// - abbr[i] > abbr[j] iff b.Get(i) > b.Get(j)
// - abbr[i] < abbr[j] iff b.Get(i) < b.Get(j)
// - If abbr[i] == abbr[j], it is unknown if b.Get(i) is greater than, less
// than, or equal to b.Get(j). A full comparison of all bytes in each is
// required.
//
func (b *Bytes) Abbreviated() []uint64 {
r := make([]uint64, b.Len())
for i := range r {
bs := b.Get(i)
r[i] = abbreviate(bs)
}
return r
}

// abbreviate interprets up to the first 8 bytes of the slice as a big-endian
// uint64. If the slice has less than 8 bytes, the value returned is the same as
// if the slice was filled to 8 bytes with zero value bytes. For example:
//
// abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
// => 1
//
// abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, 0x00})
// => 256
//
// abbreviate([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01})
// => 256
//
func abbreviate(bs []byte) uint64 {
if len(bs) >= 8 {
return binary.BigEndian.Uint64(bs)
}
var v uint64
for _, b := range bs {
v <<= 8
v |= uint64(b)
}
return v << uint(8*(8-len(bs)))
}

// Reset resets the underlying Bytes for reuse.
// TODO(asubiotto): Move towards removing Set in favor of AppendVal. At that
// point we can truncate the offsets slice.
Expand Down
29 changes: 29 additions & 0 deletions pkg/col/coldata/bytes_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -452,6 +452,35 @@ func TestBytes(t *testing.T) {
require.Equal(t, 0, b.maxSetLength)
require.NotPanics(t, func() { b.Set(4, []byte("deadbeef")) })
})

t.Run("Abbreviated", func(t *testing.T) {
rng, _ := randutil.NewPseudoRand()

// Create a vector with random bytes values.
b := NewBytes(250)
for i := 0; i < b.Len(); i++ {
size := rng.Intn(32)
b.Set(i, randutil.RandBytes(rng, size))
}

// Ensure that for every i and j:
//
// - abbr[i] < abbr[j] iff b.Get(i) < b.Get(j)
// - abbr[i] > abbr[j] iff b.Get(i) > b.Get(j)
//
abbr := b.Abbreviated()
for i := 0; i < b.Len(); i++ {
for j := 0; j < b.Len(); j++ {
cmp := bytes.Compare(b.Get(i), b.Get(j))
if abbr[i] < abbr[j] && cmp >= 0 {
t.Errorf("abbr value of %v should not be less than %v", b.Get(i), b.Get(j))
}
if abbr[i] > abbr[j] && cmp <= 0 {
t.Errorf("abbr value of %v should not be greater than %v", b.Get(i), b.Get(j))
}
}
}
})
}

func TestProportionalSize(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions pkg/jobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ go_test(
"//pkg/sql/tests",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/ctxgroup",
Expand Down
2 changes: 2 additions & 0 deletions pkg/jobs/jobs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -361,6 +362,7 @@ func (rts *registryTestSuite) check(t *testing.T, expectedStatus jobs.Status) {

func TestRegistryLifecycle(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 68315, "flaky test")
defer log.Scope(t).Close(t)

t.Run("normal success", func(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ go_library(
"//pkg/util/hlc",
"//pkg/util/limit",
"//pkg/util/log",
"//pkg/util/mon",
"//pkg/util/protoutil",
"//pkg/util/tracing",
"//pkg/util/uuid",
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ func Get(
Txn: h.Txn,
FailOnMoreRecent: args.KeyLocking != lock.None,
LocalUncertaintyLimit: cArgs.LocalUncertaintyLimit,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
})
if err != nil {
return result.Result{}, err
Expand Down
12 changes: 8 additions & 4 deletions pkg/kv/kvserver/batcheval/cmd_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,8 @@ func TestGetResumeSpan(t *testing.T) {
defer db.Close()

_, err := Put(ctx, db, CommandArgs{
Header: roachpb.Header{TargetBytes: -1},
EvalCtx: (&MockEvalCtx{}).EvalContext(),
Header: roachpb.Header{TargetBytes: -1},
Args: &roachpb.PutRequest{
RequestHeader: roachpb.RequestHeader{
Key: key,
Expand All @@ -45,7 +46,8 @@ func TestGetResumeSpan(t *testing.T) {

// Case 1: Check that a negative TargetBytes causes a resume span.
_, err = Get(ctx, db, CommandArgs{
Header: roachpb.Header{TargetBytes: -1},
EvalCtx: (&MockEvalCtx{}).EvalContext(),
Header: roachpb.Header{TargetBytes: -1},
Args: &roachpb.GetRequest{
RequestHeader: roachpb.RequestHeader{
Key: key,
Expand All @@ -62,7 +64,8 @@ func TestGetResumeSpan(t *testing.T) {
resp = &roachpb.GetResponse{}
// Case 2: Check that a negative MaxSpanRequestKeys causes a resume span.
_, err = Get(ctx, db, CommandArgs{
Header: roachpb.Header{MaxSpanRequestKeys: -1},
EvalCtx: (&MockEvalCtx{}).EvalContext(),
Header: roachpb.Header{MaxSpanRequestKeys: -1},
Args: &roachpb.GetRequest{
RequestHeader: roachpb.RequestHeader{
Key: key,
Expand All @@ -79,7 +82,8 @@ func TestGetResumeSpan(t *testing.T) {
resp = &roachpb.GetResponse{}
// Case 3: Check that a positive limit causes a normal return.
_, err = Get(ctx, db, CommandArgs{
Header: roachpb.Header{MaxSpanRequestKeys: 10, TargetBytes: 100},
EvalCtx: (&MockEvalCtx{}).EvalContext(),
Header: roachpb.Header{MaxSpanRequestKeys: 10, TargetBytes: 100},
Args: &roachpb.GetRequest{
RequestHeader: roachpb.RequestHeader{
Key: key,
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_reverse_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ func ReverseScan(
TargetBytes: h.TargetBytes,
FailOnMoreRecent: args.KeyLocking != lock.None,
Reverse: true,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
}

switch args.ScanFormat {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ func Scan(
TargetBytes: h.TargetBytes,
FailOnMoreRecent: args.KeyLocking != lock.None,
Reverse: false,
MemoryAccount: cArgs.EvalCtx.GetResponseMemoryAccount(),
}

switch args.ScanFormat {
Expand Down
13 changes: 12 additions & 1 deletion pkg/kv/kvserver/batcheval/eval_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/limit"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"golang.org/x/time/rate"
)
Expand Down Expand Up @@ -126,6 +127,12 @@ type EvalContext interface {
// WatchForMerge arranges to block all requests until the in-progress merge
// completes. Returns an error if no in-progress merge is detected.
WatchForMerge(ctx context.Context) error

// GetResponseMemoryAccount returns a memory account to be used when
// generating BatchResponses. Currently only used for MVCC scans, and only
// non-nil on those paths (a nil account is safe to use since it functions
// as an unlimited account).
GetResponseMemoryAccount() *mon.BoundAccount
}

// MockEvalCtx is a dummy implementation of EvalContext for testing purposes.
Expand All @@ -150,7 +157,7 @@ type MockEvalCtx struct {
// EvalContext returns the MockEvalCtx as an EvalContext. It will reflect future
// modifications to the underlying MockEvalContext.
func (m *MockEvalCtx) EvalContext() EvalContext {
return &mockEvalCtxImpl{m}
return &mockEvalCtxImpl{MockEvalCtx: m}
}

type mockEvalCtxImpl struct {
Expand Down Expand Up @@ -258,3 +265,7 @@ func (m *mockEvalCtxImpl) RevokeLease(_ context.Context, seq roachpb.LeaseSequen
func (m *mockEvalCtxImpl) WatchForMerge(ctx context.Context) error {
panic("unimplemented")
}
func (m *mockEvalCtxImpl) GetResponseMemoryAccount() *mon.BoundAccount {
// No limits.
return nil
}
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -1869,6 +1870,13 @@ func (r *Replica) markSystemConfigGossipFailed() {
r.mu.failureToGossipSystemConfig = true
}

// GetResponseMemoryAccount implements the batcheval.EvalContext interface.
func (r *Replica) GetResponseMemoryAccount() *mon.BoundAccount {
// Return an empty account, which places no limits. Places where a real
// account is needed use a wrapper for Replica as the EvalContext.
return nil
}

func init() {
tracing.RegisterTagRemapping("r", "range")
}
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/replica_eval_context_span.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

Expand Down Expand Up @@ -252,3 +253,8 @@ func (rec *SpanSetReplicaEvalContext) RevokeLease(ctx context.Context, seq roach
func (rec *SpanSetReplicaEvalContext) WatchForMerge(ctx context.Context) error {
return rec.i.WatchForMerge(ctx)
}

// GetResponseMemoryAccount implements the batcheval.EvalContext interface.
func (rec *SpanSetReplicaEvalContext) GetResponseMemoryAccount() *mon.BoundAccount {
return rec.i.GetResponseMemoryAccount()
}
Loading

0 comments on commit 1443ced

Please sign in to comment.