Skip to content
Open
Show file tree
Hide file tree
Changes from 5 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 @@ -231,8 +231,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
}
}

rowsCount := uint64(event.Len())
rowCallback := toRowCallback(event.PostTxnFlushed, rowsCount)
rowsCount := event.Len()
rowCallback := toRowCallback(event.PostTxnFlushed, uint64(rowsCount))
events := make([]*commonEvent.MQRowEvent, 0, rowsCount)

Comment on lines +226 to 229
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

🧩 Analysis chain

🏁 Script executed:

find . -type f -name "*.go" | xargs grep -l "type DMLEvent struct" | head -5

Repository: pingcap/ticdc

Length of output: 88


🏁 Script executed:

ast-grep --pattern $'func ($_ *DMLEvent) Len() $_ {
  $$$
}'

Repository: pingcap/ticdc

Length of output: 210


🏁 Script executed:

sed -n '230,240p' downstreamadapter/sink/kafka/sink.go

Repository: pingcap/ticdc

Length of output: 292


Fix compilation error: make() requires int type for capacity, but event.Len() returns int32.

Line 234 assigns rowsCount := event.Len(), making rowsCount type int32. Line 236 then uses rowsCount as the capacity argument to make([]*commonEvent.MQRowEvent, 0, rowsCount), which requires int. Add an explicit conversion:

rowsCount := int(event.Len())
🤖 Prompt for AI Agents
Verify each finding against the current code and only fix it if needed.

In `@downstreamadapter/sink/kafka/sink.go` around lines 234 - 237, The variable
rowsCount is inferred as int32 from event.Len(), but make(..., 0, rowsCount)
requires an int; change the assignment to convert the length to int (e.g., set
rowsCount using int(event.Len())) so the capacity passed to
make([]*commonEvent.MQRowEvent, 0, rowsCount) is the correct type and update any
dependent call sites like toRowCallback(event.PostTxnFlushed, uint64(rowsCount))
as needed.

for {
row, ok := event.GetNextRow()
Expand All @@ -246,7 +247,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 @@ -263,9 +264,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 @@ -341,6 +342,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 @@ -341,8 +341,9 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error {
}
}

rowsCount := uint64(event.Len())
rowCallback := toRowCallback(event.PostTxnFlushed, rowsCount)
rowsCount := event.Len()
rowCallback := toRowCallback(event.PostTxnFlushed, uint64(rowsCount))
events := make([]*commonEvent.MQRowEvent, 0, rowsCount)

for {
row, ok := event.GetNextRow()
Expand All @@ -356,7 +357,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 @@ -373,9 +374,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 @@ -444,6 +445,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 @@ -139,16 +139,17 @@ func (s *Sink) AddDMLEvent(event *commonEvent.DMLEvent) {
}
}
}
rowsCount := uint64(event.Len())
rowCallback := toRowCallback(event.PostTxnFlushed, rowsCount)
rowsCount := event.Len()
rowCallback := toRowCallback(event.PostTxnFlushed, uint64(rowsCount))
events := make([]writer.RedoEvent, 0, 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 @@ -157,6 +158,7 @@ func (s *Sink) AddDMLEvent(event *commonEvent.DMLEvent) {
Callback: rowCallback,
})
}
s.logBuffer.Push(events...)
}

func (s *Sink) IsNormal() bool {
Expand Down Expand Up @@ -201,20 +203,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
61 changes: 61 additions & 0 deletions downstreamadapter/sink/redo/sink_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,9 @@ import (
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 +331,62 @@ func runBenchTest(b *testing.B, storage string, useFileBackend bool) {

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

type mockBatchWriter struct {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use gomock to generate

mu sync.Mutex
batchLens []int
}

func (m *mockBatchWriter) WriteEvents(_ context.Context, events ...writer.RedoEvent) error {
m.mu.Lock()
defer m.mu.Unlock()
m.batchLens = append(m.batchLens, len(events))
for _, event := range events {
event.PostFlush()
}
return nil
}

func (m *mockBatchWriter) Run(ctx context.Context) error {
<-ctx.Done()
return ctx.Err()
}

func (m *mockBatchWriter) Close() error {
return nil
}

func (m *mockBatchWriter) SetTableSchemaStore(_ *commonEvent.TableSchemaStore) {}

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

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

mockWriter := &mockBatchWriter{}
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)

mockWriter.mu.Lock()
defer mockWriter.mu.Unlock()
require.Equal(t, []int{redo.DefaultFlushBatchSize, redo.DefaultFlushBatchSize, 17}, mockWriter.batchLens)
}
Loading
Loading