Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

changefeedccl: Make kvevent.Event memory efficient #87718

Merged
merged 1 commit into from
Sep 9, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,7 +354,7 @@ func TestChangefeedSendError(t *testing.T) {

// Allow triggering a single sendError
sendErrorCh := make(chan error, 1)
knobs.FeedKnobs.OnRangeFeedValue = func(_ roachpb.KeyValue) error {
knobs.FeedKnobs.OnRangeFeedValue = func() error {
select {
case err := <-sendErrorCh:
return err
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/changefeedccl/event_processing.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdceval"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdcevent"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
Expand Down Expand Up @@ -146,8 +145,7 @@ func (c *kvEventToRowConsumer) ConsumeEvent(ctx context.Context, ev kvevent.Even
if !c.details.Opts.GetFilters().WithDiff {
return cdcevent.Row{}, nil
}
prevKV := roachpb.KeyValue{Key: ev.KV().Key, Value: ev.PrevValue()}
return c.decoder.DecodeKV(ctx, prevKV, prevSchemaTimestamp)
return c.decoder.DecodeKV(ctx, ev.PrevKeyValue(), prevSchemaTimestamp)
}()
if err != nil {
// Column families are stored contiguously, so we'll get
Expand Down
20 changes: 10 additions & 10 deletions pkg/ccl/changefeedccl/kvevent/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/require"
Expand All @@ -38,12 +37,9 @@ func BenchmarkMemBuffer(b *testing.B) {
p := make([]kvevent.Event, 32<<10)
for i := range p {
if rng.Int31()%20 == 0 {
p[i] = kvevent.MakeResolvedEvent(generateSpan(rng), hlc.Timestamp{}, jobspb.ResolvedSpan_NONE)
p[i] = kvevent.MakeResolvedEvent(generateRangeFeedCheckpoint(rng), jobspb.ResolvedSpan_NONE)
} else {
p[i] = kvevent.MakeKVEvent(
makeKV(rng, valSize),
roachpb.Value{RawBytes: randutil.RandBytes(rng, rng.Intn(valSize))},
hlc.Timestamp{})
p[i] = kvevent.MakeKVEvent(makeRangeFeedEvent(rng, valSize, valSize))
}
}
return p
Expand Down Expand Up @@ -116,7 +112,7 @@ func BenchmarkMemBuffer(b *testing.B) {
_ = wg.Wait() // Ignore error -- this group returns context cancellation.
}

func generateSpan(rng *rand.Rand) roachpb.Span {
func generateRangeFeedCheckpoint(rng *rand.Rand) *roachpb.RangeFeedEvent {
start := rng.Intn(2 << 20)
end := start + rng.Intn(2<<20)
startDatum := tree.NewDInt(tree.DInt(start))
Expand All @@ -141,8 +137,12 @@ func generateSpan(rng *rand.Rand) roachpb.Span {
panic(err)
}

return roachpb.Span{
Key: startKey,
EndKey: endKey,
return &roachpb.RangeFeedEvent{
Checkpoint: &roachpb.RangeFeedCheckpoint{
Span: roachpb.Span{
Key: startKey,
EndKey: endKey,
},
},
}
}
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/kvevent/blocking_buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ func (b *blockingBuffer) pop() (e Event, ok bool, err error) {
// If the batching event consumer does not have periodic flush configured,
// we may never be able to make forward progress.
// So, we issue the flush request to the consumer to ensure that we release some memory.
e = Event{flush: true}
e = Event{et: TypeFlush}
ok = true
}

Expand Down Expand Up @@ -203,7 +203,7 @@ func (b *blockingBuffer) Add(ctx context.Context, e Event) error {
}

// Acquire the quota first.
alloc := int64(changefeedbase.EventMemoryMultiplier.Get(b.sv) * float64(e.approxSize))
alloc := int64(changefeedbase.EventMemoryMultiplier.Get(b.sv) * float64(e.ApproximateSize()))
if l := changefeedbase.PerChangefeedMemLimit.Get(b.sv); alloc > l {
return errors.Newf("event size %d exceeds per changefeed limit %d", alloc, l)
}
Expand Down
26 changes: 18 additions & 8 deletions pkg/ccl/changefeedccl/kvevent/blocking_buffer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (
"github.com/stretchr/testify/require"
)

func makeKV(rnd *rand.Rand, valSize int) roachpb.KeyValue {
func makeRangeFeedEvent(rnd *rand.Rand, valSize int, prevValSize int) *roachpb.RangeFeedEvent {
const tableID = 42

key, err := keyside.Encode(
Expand All @@ -44,13 +44,23 @@ func makeKV(rnd *rand.Rand, valSize int) roachpb.KeyValue {
panic(err)
}

return roachpb.KeyValue{
Key: key,
Value: roachpb.Value{
RawBytes: randutil.RandBytes(rnd, valSize),
Timestamp: hlc.Timestamp{WallTime: 1},
e := roachpb.RangeFeedEvent{
Val: &roachpb.RangeFeedValue{
Key: key,
Value: roachpb.Value{
RawBytes: randutil.RandBytes(rnd, valSize),
Timestamp: hlc.Timestamp{WallTime: 1},
},
},
}

if prevValSize > 0 {
e.Val.PrevValue = roachpb.Value{
RawBytes: randutil.RandBytes(rnd, prevValSize),
Timestamp: hlc.Timestamp{WallTime: 1},
}
}
return &e
}

func getBoundAccountWithBudget(budget int64) (account mon.BoundAccount, cleanup func()) {
Expand Down Expand Up @@ -95,7 +105,7 @@ func TestBlockingBuffer(t *testing.T) {
wg.GoCtx(func(ctx context.Context) error {
rnd, _ := randutil.NewTestRand()
for {
err := buf.Add(ctx, kvevent.MakeKVEvent(makeKV(rnd, 256), roachpb.Value{}, hlc.Timestamp{}))
err := buf.Add(ctx, kvevent.MakeKVEvent(makeRangeFeedEvent(rnd, 256, 0)))
if err != nil {
return err
}
Expand Down Expand Up @@ -138,7 +148,7 @@ func TestBlockingBufferNotifiesConsumerWhenOutOfMemory(t *testing.T) {
wg.GoCtx(func(ctx context.Context) error {
rnd, _ := randutil.NewTestRand()
for {
err := buf.Add(ctx, kvevent.MakeKVEvent(makeKV(rnd, 256), roachpb.Value{}, hlc.Timestamp{}))
err := buf.Add(ctx, kvevent.MakeKVEvent(makeRangeFeedEvent(rnd, 256, 0)))
if err != nil {
return err
}
Expand Down
11 changes: 6 additions & 5 deletions pkg/ccl/changefeedccl/kvevent/chunked_event_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ package kvevent
import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/assert"
)
Expand All @@ -30,21 +31,21 @@ func TestBufferEntryQueue(t *testing.T) {

// Add events to fill 5 chunks and assert they are consumed in fifo order.
eventCount := bufferEventChunkArrSize * 5
lastPop := -1
lastPush := -1
var lastPop int64 = -1
var lastPush int64 = -1

for eventCount > 0 {
op := rand.Intn(2)
if op == 0 {
q.enqueue(Event{approxSize: lastPush + 1})
q.enqueue(Event{backfillTimestamp: hlc.Timestamp{WallTime: lastPush + 1}})
lastPush++
} else {
e, ok := q.dequeue()
if !ok {
assert.Equal(t, lastPop, lastPush)
assert.True(t, q.empty())
} else {
assert.Equal(t, e.approxSize, lastPop+1)
assert.Equal(t, lastPop+1, e.backfillTimestamp.WallTime)
lastPop++
eventCount--
}
Expand All @@ -54,7 +55,7 @@ func TestBufferEntryQueue(t *testing.T) {
// Verify that purging works.
eventCount = bufferEventChunkArrSize * 2.5
for eventCount > 0 {
q.enqueue(Event{approxSize: lastPush + 1})
q.enqueue(Event{})
eventCount--
}
q.purge()
Expand Down
Loading