Skip to content
Open
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
12 changes: 7 additions & 5 deletions downstreamadapter/sink/kafka/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,8 +223,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {

partitionGenerator := s.comp.eventRouter.GetPartitionGenerator(schema, table)
selector := s.comp.columnSelector.Get(schema, table)
rowsCount := uint64(event.Len())
rowCallback := helper.NewTxnPostFlushRowCallback(event, rowsCount)
rowsCount := event.Len()
events := make([]*commonEvent.MQRowEvent, 0, rowsCount)
rowCallback := helper.NewTxnPostFlushRowCallback(event, uint64(rowsCount))

for {
row, ok := event.GetNextRow()
Expand All @@ -238,7 +239,7 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
return errors.Trace(err)
}

mqEvent := &commonEvent.MQRowEvent{
events = append(events, &commonEvent.MQRowEvent{
Key: commonEvent.TopicPartitionKey{
Topic: topic,
Partition: index,
Expand All @@ -255,9 +256,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
ColumnSelector: selector,
Checksum: row.Checksum,
},
}
s.rowChan.Push(mqEvent)
})
}
s.rowChan.Push(events...)
}
}
}
Expand Down Expand Up @@ -333,6 +334,7 @@ func (s *sink) batch(ctx context.Context, buffer []*commonEvent.MQRowEvent) ([]*
zap.String("changefeed", s.changefeedID.Name()))
return nil, nil
}
buffer = buffer[:0]
return msgs, nil
}
}
Expand Down
12 changes: 7 additions & 5 deletions downstreamadapter/sink/pulsar/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -333,8 +333,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {

partitionGenerator := s.comp.eventRouter.GetPartitionGenerator(schema, table)
selector := s.comp.columnSelector.Get(schema, table)
rowsCount := uint64(event.Len())
rowCallback := helper.NewTxnPostFlushRowCallback(event, rowsCount)
rowsCount := event.Len()
events := make([]*commonEvent.MQRowEvent, 0, rowsCount)
rowCallback := helper.NewTxnPostFlushRowCallback(event, uint64(rowsCount))

for {
row, ok := event.GetNextRow()
Expand All @@ -348,7 +349,7 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
return errors.Trace(err)
}

mqEvent := &commonEvent.MQRowEvent{
events = append(events, &commonEvent.MQRowEvent{
Key: commonEvent.TopicPartitionKey{
Topic: topic,
Partition: index,
Expand All @@ -365,9 +366,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
ColumnSelector: selector,
Checksum: row.Checksum,
},
}
s.rowChan.Push(mqEvent)
})
}
s.rowChan.Push(events...)
}
}
}
Expand Down Expand Up @@ -436,6 +437,7 @@ func (s *sink) batch(ctx context.Context, buffer []*commonEvent.MQRowEvent, tick
zap.String("changefeed", s.changefeedID.Name()))
return nil, nil
}
buffer = buffer[:0]
return msgs, nil
}
}
Expand Down
24 changes: 18 additions & 6 deletions downstreamadapter/sink/redo/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,16 +133,17 @@ func (s *Sink) WriteBlockEvent(event commonEvent.BlockEvent) error {
}

func (s *Sink) AddDMLEvent(event *commonEvent.DMLEvent) {
rowsCount := uint64(event.Len())
rowCallback := helper.NewTxnPostFlushRowCallback(event, rowsCount)
rowsCount := event.Len()
events := make([]writer.RedoEvent, 0, rowsCount)
rowCallback := helper.NewTxnPostFlushRowCallback(event, uint64(rowsCount))

for {
row, ok := event.GetNextRow()
if !ok {
event.Rewind()
break
}
s.logBuffer.Push(&commonEvent.RedoRowEvent{
events = append(events, &commonEvent.RedoRowEvent{
StartTs: event.StartTs,
CommitTs: event.CommitTs,
Event: row,
Expand All @@ -151,6 +152,7 @@ func (s *Sink) AddDMLEvent(event *commonEvent.DMLEvent) {
Callback: rowCallback,
})
}
s.logBuffer.Push(events...)
}

func (s *Sink) IsNormal() bool {
Expand Down Expand Up @@ -195,20 +197,30 @@ func (s *Sink) Close(_ bool) {
}

func (s *Sink) sendMessages(ctx context.Context) error {
buffer := make([]writer.RedoEvent, 0, redo.DefaultFlushBatchSize)
for {
e, ok := s.logBuffer.Get()
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
default:
}
events, ok := s.logBuffer.GetMultipleNoGroup(buffer)
if !ok {
return nil
}
if len(events) == 0 {
continue
}
buffer = events[:0]

start := time.Now()
err := s.dmlWriter.WriteEvents(ctx, e)
err := s.dmlWriter.WriteEvents(ctx, events...)
if err != nil {
return err
}

if s.metric != nil {
s.metric.observeRowWrite(1, time.Since(start))
s.metric.observeRowWrite(len(events), time.Since(start))
}
}
}
Expand Down
55 changes: 55 additions & 0 deletions downstreamadapter/sink/redo/sink_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,15 @@ import (
"testing"
"time"

"github.com/golang/mock/gomock"
"github.com/pingcap/ticdc/heartbeatpb"
"github.com/pingcap/ticdc/pkg/common"
commonEvent "github.com/pingcap/ticdc/pkg/common/event"
"github.com/pingcap/ticdc/pkg/config"
"github.com/pingcap/ticdc/pkg/redo"
"github.com/pingcap/ticdc/pkg/redo/writer"
"github.com/pingcap/ticdc/pkg/util"
"github.com/pingcap/ticdc/utils/chann"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
)
Expand Down Expand Up @@ -329,3 +332,55 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) {

require.ErrorIs(b, eg.Wait(), context.Canceled)
}

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

ctx, cancel := context.WithCancel(context.Background())
defer cancel()

ctrl := gomock.NewController(t)
defer ctrl.Finish()

mockWriter := writer.NewMockRedoLogWriter(ctrl)
expectWriteBatch := func(batchSize int) *gomock.Call {
args := make([]interface{}, 0, batchSize+1)
args = append(args, gomock.Any()) // context
for i := 0; i < batchSize; i++ {
args = append(args, gomock.Any())
}
return mockWriter.EXPECT().
WriteEvents(args[0], args[1:]...).
DoAndReturn(func(_ context.Context, events ...writer.RedoEvent) error {
require.Len(t, events, batchSize)
return nil
})
}

gomock.InOrder(
expectWriteBatch(redo.DefaultFlushBatchSize),
expectWriteBatch(redo.DefaultFlushBatchSize),
expectWriteBatch(17),
)

s := &Sink{
dmlWriter: mockWriter,
logBuffer: chann.NewUnlimitedChannelDefault[writer.RedoEvent](),
}

doneCh := make(chan error, 1)
go func() {
doneCh <- s.sendMessages(ctx)
}()

totalEvents := redo.DefaultFlushBatchSize*2 + 17
events := make([]writer.RedoEvent, 0, totalEvents)
for i := 0; i < totalEvents; i++ {
events = append(events, &commonEvent.RedoRowEvent{})
}
s.logBuffer.Push(events...)
s.logBuffer.Close()

err := <-doneCh
require.NoError(t, err)
}
Loading