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

sink(ticdc): optimize buffer sink flush from O(N^2) to O(N) (#3899) #3948

Closed
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
214 changes: 214 additions & 0 deletions cdc/sink/buffer_sink.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,214 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package sink

import (
"context"
"sort"
"sync"
"sync/atomic"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/ticdc/cdc/model"
"github.com/pingcap/ticdc/pkg/util"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)

const maxFlushBatchSize = 512

// bufferSink buffers emitted events and checkpoints and flush asynchronously.
// Note that it is a thread-safe Sink implementation.
type bufferSink struct {
Sink
changeFeedCheckpointTs uint64
tableCheckpointTsMap sync.Map
buffer map[model.TableID][]*model.RowChangedEvent
bufferMu sync.Mutex
flushTsChan chan flushMsg
drawbackChan chan drawbackMsg
}

var _ Sink = (*bufferSink)(nil)

func newBufferSink(
backendSink Sink, checkpointTs model.Ts, drawbackChan chan drawbackMsg,
) *bufferSink {
sink := &bufferSink{
Sink: backendSink,
// buffer shares the same flow control with table sink
buffer: make(map[model.TableID][]*model.RowChangedEvent),
changeFeedCheckpointTs: checkpointTs,
flushTsChan: make(chan flushMsg, maxFlushBatchSize),
drawbackChan: drawbackChan,
}
return sink
}

type runState struct {
batch [maxFlushBatchSize]flushMsg

metricFlushDuration prometheus.Observer
metricEmitRowDuration prometheus.Observer
metricTotalRows prometheus.Counter
}

func (b *bufferSink) run(ctx context.Context, errCh chan error) {
changefeedID := util.ChangefeedIDFromCtx(ctx)
advertiseAddr := util.CaptureAddrFromCtx(ctx)
state := runState{
metricFlushDuration: flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "Flush"),
metricEmitRowDuration: flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "EmitRow"),
metricTotalRows: bufferSinkTotalRowsCountCounter.WithLabelValues(advertiseAddr, changefeedID),
}
defer func() {
flushRowChangedDuration.DeleteLabelValues(advertiseAddr, changefeedID, "Flush")
flushRowChangedDuration.DeleteLabelValues(advertiseAddr, changefeedID, "EmitRow")
bufferSinkTotalRowsCountCounter.DeleteLabelValues(advertiseAddr, changefeedID)
}()

for {
keepRun, err := b.runOnce(ctx, &state)
if err != nil && errors.Cause(err) != context.Canceled {
errCh <- err
return
}
if !keepRun {
return
}
}
}

func (b *bufferSink) runOnce(ctx context.Context, state *runState) (bool, error) {
batchSize, batch := 0, state.batch
push := func(event flushMsg) {
batch[batchSize] = event
batchSize++
}
select {
case <-ctx.Done():
return false, ctx.Err()
case drawback := <-b.drawbackChan:
b.bufferMu.Lock()
delete(b.buffer, drawback.tableID)
b.bufferMu.Unlock()
close(drawback.callback)
case event := <-b.flushTsChan:
push(event)
RecvBatch:
for batchSize < maxFlushBatchSize {
select {
case event := <-b.flushTsChan:
push(event)
default:
break RecvBatch
}
}
}

b.bufferMu.Lock()
startEmit := time.Now()
// find all rows before resolvedTs and emit to backend sink
for i := 0; i < batchSize; i++ {
tableID, resolvedTs := batch[i].tableID, batch[i].resolvedTs
rows := b.buffer[tableID]
i := sort.Search(len(rows), func(i int) bool {
return rows[i].CommitTs > resolvedTs
})
if i == 0 {
continue
}
state.metricTotalRows.Add(float64(i))

err := b.Sink.EmitRowChangedEvents(ctx, rows[:i]...)
if err != nil {
b.bufferMu.Unlock()
return false, errors.Trace(err)
}

// put remaining rows back to buffer
// append to a new, fixed slice to avoid lazy GC
b.buffer[tableID] = append(make([]*model.RowChangedEvent, 0, len(rows[i:])), rows[i:]...)
}
b.bufferMu.Unlock()
state.metricEmitRowDuration.Observe(time.Since(startEmit).Seconds())

startFlush := time.Now()
for i := 0; i < batchSize; i++ {
tableID, resolvedTs := batch[i].tableID, batch[i].resolvedTs
checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, tableID, resolvedTs)
if err != nil {
return false, errors.Trace(err)
}
b.tableCheckpointTsMap.Store(tableID, checkpointTs)
}
now := time.Now()
state.metricFlushDuration.Observe(now.Sub(startFlush).Seconds())
if now.Sub(startEmit) > time.Second {
log.Warn("flush row changed events too slow",
zap.Int("batchSize", batchSize),
zap.Duration("duration", now.Sub(startEmit)),
util.ZapFieldChangefeed(ctx))
}

return true, nil
}

func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error {
select {
case <-ctx.Done():
return ctx.Err()
default:
if len(rows) == 0 {
return nil
}
tableID := rows[0].Table.TableID
b.bufferMu.Lock()
b.buffer[tableID] = append(b.buffer[tableID], rows...)
b.bufferMu.Unlock()
}
return nil
}

func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) {
select {
case <-ctx.Done():
return b.getTableCheckpointTs(tableID), ctx.Err()
case b.flushTsChan <- flushMsg{
tableID: tableID,
resolvedTs: resolvedTs,
}:
}
return b.getTableCheckpointTs(tableID), nil
}

type flushMsg struct {
tableID model.TableID
resolvedTs uint64
}

func (b *bufferSink) getTableCheckpointTs(tableID model.TableID) uint64 {
checkPoints, ok := b.tableCheckpointTsMap.Load(tableID)
if ok {
return checkPoints.(uint64)
}
return atomic.LoadUint64(&b.changeFeedCheckpointTs)
}

// UpdateChangeFeedCheckpointTs update the changeFeedCheckpointTs every processor tick
func (b *bufferSink) UpdateChangeFeedCheckpointTs(checkpointTs uint64) {
atomic.StoreUint64(&b.changeFeedCheckpointTs, checkpointTs)
}
152 changes: 152 additions & 0 deletions cdc/sink/buffer_sink_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package sink

import (
"context"
"fmt"
"math"
"testing"
"time"

"github.com/pingcap/ticdc/cdc/model"
"github.com/stretchr/testify/require"
)

func TestTableIsNotFlushed(t *testing.T) {
t.Parallel()

b := bufferSink{changeFeedCheckpointTs: 1}
require.Equal(t, uint64(1), b.getTableCheckpointTs(2))
b.UpdateChangeFeedCheckpointTs(3)
require.Equal(t, uint64(3), b.getTableCheckpointTs(2))
}

func TestFlushTable(t *testing.T) {
t.Parallel()

ctx, cancel := context.WithCancel(context.TODO())
defer cancel()
b := newBufferSink(newBlackHoleSink(ctx, make(map[string]string)), 5, make(chan drawbackMsg))
go b.run(ctx, make(chan error))

require.Equal(t, uint64(5), b.getTableCheckpointTs(2))
require.Nil(t, b.EmitRowChangedEvents(ctx))
tbl1 := &model.TableName{TableID: 1}
tbl2 := &model.TableName{TableID: 2}
tbl3 := &model.TableName{TableID: 3}
tbl4 := &model.TableName{TableID: 4}
require.Nil(t, b.EmitRowChangedEvents(ctx, []*model.RowChangedEvent{
{CommitTs: 6, Table: tbl1},
{CommitTs: 6, Table: tbl2},
{CommitTs: 6, Table: tbl3},
{CommitTs: 6, Table: tbl4},
{CommitTs: 10, Table: tbl1},
{CommitTs: 10, Table: tbl2},
{CommitTs: 10, Table: tbl3},
{CommitTs: 10, Table: tbl4},
}...))
checkpoint, err := b.FlushRowChangedEvents(ctx, 1, 7)
require.True(t, checkpoint <= 7)
require.Nil(t, err)
checkpoint, err = b.FlushRowChangedEvents(ctx, 2, 6)
require.True(t, checkpoint <= 6)
require.Nil(t, err)
checkpoint, err = b.FlushRowChangedEvents(ctx, 3, 8)
require.True(t, checkpoint <= 8)
require.Nil(t, err)
time.Sleep(200 * time.Millisecond)
require.Equal(t, uint64(7), b.getTableCheckpointTs(1))
require.Equal(t, uint64(6), b.getTableCheckpointTs(2))
require.Equal(t, uint64(8), b.getTableCheckpointTs(3))
require.Equal(t, uint64(5), b.getTableCheckpointTs(4))
b.UpdateChangeFeedCheckpointTs(6)
require.Equal(t, uint64(7), b.getTableCheckpointTs(1))
require.Equal(t, uint64(6), b.getTableCheckpointTs(2))
require.Equal(t, uint64(8), b.getTableCheckpointTs(3))
require.Equal(t, uint64(6), b.getTableCheckpointTs(4))
}

func TestFlushFailed(t *testing.T) {
t.Parallel()

ctx, cancel := context.WithCancel(context.TODO())
b := newBufferSink(newBlackHoleSink(ctx, make(map[string]string)), 5, make(chan drawbackMsg))
go b.run(ctx, make(chan error))

checkpoint, err := b.FlushRowChangedEvents(ctx, 3, 8)
require.True(t, checkpoint <= 8)
require.Nil(t, err)
time.Sleep(200 * time.Millisecond)
require.Equal(t, uint64(8), b.getTableCheckpointTs(3))
cancel()
checkpoint, _ = b.FlushRowChangedEvents(ctx, 3, 18)
require.Equal(t, uint64(8), checkpoint)
checkpoint, _ = b.FlushRowChangedEvents(ctx, 1, 18)
require.Equal(t, uint64(5), checkpoint)
time.Sleep(200 * time.Millisecond)
require.Equal(t, uint64(8), b.getTableCheckpointTs(3))
require.Equal(t, uint64(5), b.getTableCheckpointTs(1))
}

type benchSink struct {
Sink
}

func (b *benchSink) EmitRowChangedEvents(
ctx context.Context, rows ...*model.RowChangedEvent,
) error {
return nil
}

func (b *benchSink) FlushRowChangedEvents(
ctx context.Context, tableID model.TableID, resolvedTs uint64,
) (uint64, error) {
return 0, nil
}

func BenchmarkRun(b *testing.B) {
ctx, cancel := context.WithCancel(context.TODO())
defer cancel()

state := runState{
metricFlushDuration: flushRowChangedDuration.WithLabelValues(b.Name(), b.Name(), "Flush"),
metricEmitRowDuration: flushRowChangedDuration.WithLabelValues(b.Name(), b.Name(), "EmitRow"),
metricTotalRows: bufferSinkTotalRowsCountCounter.WithLabelValues(b.Name(), b.Name()),
}

for exp := 0; exp < 9; exp++ {
count := int(math.Pow(4, float64(exp)))
s := newBufferSink(&benchSink{}, 5, make(chan drawbackMsg))
s.flushTsChan = make(chan flushMsg, count)
for i := 0; i < count; i++ {
s.buffer[int64(i)] = []*model.RowChangedEvent{{CommitTs: 5}}
}
b.ResetTimer()

b.Run(fmt.Sprintf("%d table(s)", count), func(b *testing.B) {
for i := 0; i < b.N; i++ {
for j := 0; j < count; j++ {
s.flushTsChan <- flushMsg{tableID: int64(0)}
}
for len(s.flushTsChan) != 0 {
keepRun, err := s.runOnce(ctx, &state)
if err != nil || !keepRun {
b.Fatal(keepRun, err)
}
}
}
})
}
}
Loading