From f2d69b086b0078aac2ba8a77390dcb24d0e9c424 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 27 Dec 2023 10:24:46 +0800 Subject: [PATCH 01/20] simplr protocol send bootstrap event periodically --- cdc/sink/dmlsink/mq/kafka_dml_sink.go | 2 +- cdc/sink/dmlsink/mq/mq_dml_sink.go | 6 ++- cdc/sink/dmlsink/mq/pulsar_dml_sink.go | 2 +- cdc/sink/dmlsink/mq/worker_test.go | 6 ++- pkg/config/sink.go | 3 ++ pkg/filter/sql_event_filter.go | 3 +- pkg/sink/codec/encoder.go | 7 ++-- pkg/sink/codec/encoder_group.go | 55 +++++++++++++++++++++++--- pkg/sink/codec/simple/encoder.go | 1 + 9 files changed, 69 insertions(+), 16 deletions(-) diff --git a/cdc/sink/dmlsink/mq/kafka_dml_sink.go b/cdc/sink/dmlsink/mq/kafka_dml_sink.go index 86821ac25ce..70b12c79a81 100644 --- a/cdc/sink/dmlsink/mq/kafka_dml_sink.go +++ b/cdc/sink/dmlsink/mq/kafka_dml_sink.go @@ -122,7 +122,7 @@ func NewKafkaDMLSink( metricsCollector := factory.MetricsCollector(tiflowutil.RoleProcessor, adminClient) dmlProducer := producerCreator(ctx, changefeedID, asyncProducer, metricsCollector, errCh, failpointCh) concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) - encoderGroup := codec.NewEncoderGroup(encoderBuilder, concurrency, changefeedID) + encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, concurrency, changefeedID) s := newDMLSink(ctx, changefeedID, dmlProducer, adminClient, topicManager, eventRouter, trans, encoderGroup, protocol, scheme, errCh) log.Info("DML sink producer created", diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink.go b/cdc/sink/dmlsink/mq/mq_dml_sink.go index abbe1325305..aaf7a5611ef 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink.go @@ -165,7 +165,6 @@ func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa txn.Callback() continue } - rowCallback := toRowCallback(txn.Callback, uint64(len(txn.Event.Rows))) for _, row := range txn.Event.Rows { topic := s.alive.eventRouter.GetTopicForRowChange(row) @@ -197,7 +196,10 @@ func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa // So it is safe to send the event to a unbounded channel here. s.alive.worker.msgChan.In() <- mqEvent{ key: codec.TopicPartitionKey{ - Topic: topic, Partition: index, PartitionKey: key, + Topic: topic, + Partition: index, + PartitionKey: key, + TotalPartition: partitionNum, }, rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: row, diff --git a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go index ea77163106d..1eec9864240 100644 --- a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go +++ b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go @@ -122,7 +122,7 @@ func NewPulsarDMLSink( } concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) - encoderGroup := codec.NewEncoderGroup(encoderBuilder, concurrency, changefeedID) + encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, concurrency, changefeedID) s := newDMLSink(ctx, changefeedID, p, nil, topicManager, eventRouter, trans, encoderGroup, protocol, scheme, errCh) diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index f4140c20c28..c1e9a6e98ab 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -44,7 +44,8 @@ func newBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlproduc require.NoError(t, err) encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) - encoderGroup := codec.NewEncoderGroup(encoderBuilder, encoderConcurrency, id) + cfg := config.GetDefaultReplicaConfig() + encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, encoderConcurrency, id) return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } @@ -58,7 +59,8 @@ func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlpro require.NoError(t, err) encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) - encoderGroup := codec.NewEncoderGroup(encoderBuilder, encoderConcurrency, id) + cfg := config.GetDefaultReplicaConfig() + encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, encoderConcurrency, id) return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } diff --git a/pkg/config/sink.go b/pkg/config/sink.go index aa755a65d8f..8571280539f 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -165,6 +165,9 @@ type SinkConfig struct { // AdvanceTimeoutInSec is a duration in second. If a table sink progress hasn't been // advanced for this given duration, the sink will be canceled and re-established. AdvanceTimeoutInSec *uint `toml:"advance-timeout-in-sec" json:"advance-timeout-in-sec,omitempty"` + + SendBootstrapIntervalInSec *uint `toml:"send-bootstrap-interval-in-sec" json:"send-bootstrap-interval-in-sec,omitempty"` + SendBootstrapInMsgCount *int32 `toml:"send-bootstrap-in-msg-count" json:"send-bootstrap-in-msg-count,omitempty"` } // MaskSensitiveData masks sensitive data in SinkConfig diff --git a/pkg/filter/sql_event_filter.go b/pkg/filter/sql_event_filter.go index 90194d8003c..e5954836568 100644 --- a/pkg/filter/sql_event_filter.go +++ b/pkg/filter/sql_event_filter.go @@ -175,7 +175,8 @@ func (f *sqlEventFilter) shouldSkipDDL( } rules := f.getRules(schema, table) - for _, rule := range rules { + for i, rule := range rules { + log.Info("fizz sql event filter handle ddl event", zap.Any("idx", i), zap.Any("rule", rule)) action, err := rule.bf.Filter( binlogFilterSchemaPlaceholder, binlogFilterTablePlaceholder, diff --git a/pkg/sink/codec/encoder.go b/pkg/sink/codec/encoder.go index c6250c08610..c5c94b61d77 100644 --- a/pkg/sink/codec/encoder.go +++ b/pkg/sink/codec/encoder.go @@ -86,7 +86,8 @@ func IsColumnValueEqual(preValue, updatedValue interface{}) bool { // TopicPartitionKey contains the topic and partition key of the message. type TopicPartitionKey struct { - Topic string - Partition int32 - PartitionKey string + Topic string + Partition int32 + PartitionKey string + TotalPartition int32 } diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index 6989c1d696f..0f5c76954c7 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -56,10 +57,14 @@ type encoderGroup struct { index uint64 outputCh chan *future + + bootstrapWorker *bootstrapWorker } // NewEncoderGroup creates a new EncoderGroup instance -func NewEncoderGroup(builder RowEventEncoderBuilder, +func NewEncoderGroup( + cfg *config.SinkConfig, + builder RowEventEncoderBuilder, concurrency int, changefeedID model.ChangeFeedID, ) *encoderGroup { if concurrency <= 0 { @@ -70,15 +75,38 @@ func NewEncoderGroup(builder RowEventEncoderBuilder, for i := 0; i < concurrency; i++ { inputCh[i] = make(chan *future, defaultInputChanSize) } + outCh := make(chan *future, defaultInputChanSize) + + var bootstrapWorker *bootstrapWorker + if *cfg.Protocol == config.ProtocolSimple.String() { + log.Info("fizz:bootstrap worker is enable for simple protocol, create it!") + sendBootstrapIntervalInSec := util.GetOrZero(cfg.SendBootstrapIntervalInSec) + if sendBootstrapIntervalInSec <= 0 { + sendBootstrapIntervalInSec = uint(defaultSendBootstrapInterval.Seconds()) + } + msgCount := util.GetOrZero(cfg.SendBootstrapInMsgCount) + if msgCount <= 0 { + msgCount = defaultSendBootstrapInMsgCount + } + interval := time.Duration(sendBootstrapIntervalInSec) * time.Second + bootstrapWorker = newBootstrapWorker( + outCh, + builder, + interval, + msgCount, + defaultMaxInactiveDuration, + ) + } return &encoderGroup{ changefeedID: changefeedID, - builder: builder, - concurrency: concurrency, - inputCh: inputCh, - index: 0, - outputCh: make(chan *future, defaultInputChanSize*concurrency), + builder: builder, + concurrency: concurrency, + inputCh: inputCh, + index: 0, + outputCh: outCh, + bootstrapWorker: bootstrapWorker, } } @@ -96,6 +124,13 @@ func (g *encoderGroup) Run(ctx context.Context) error { return g.runEncoder(ctx, idx) }) } + + if g.bootstrapWorker != nil { + eg.Go(func() error { + return g.bootstrapWorker.run(ctx) + }) + } + return eg.Wait() } @@ -130,6 +165,14 @@ func (g *encoderGroup) AddEvents( key TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent, ) error { + // bootstrapWorker only not nil when the protocol is simple + if g.bootstrapWorker != nil { + err := g.bootstrapWorker.addEvent(ctx, key, events[0].Event) + if err != nil { + return errors.Trace(err) + } + } + future := newFuture(key, events...) index := atomic.AddUint64(&g.index, 1) % uint64(g.concurrency) select { diff --git a/pkg/sink/codec/simple/encoder.go b/pkg/sink/codec/simple/encoder.go index d4abbc6704a..9414e1bfc2d 100644 --- a/pkg/sink/codec/simple/encoder.go +++ b/pkg/sink/codec/simple/encoder.go @@ -148,6 +148,7 @@ func (e *encoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { // EncodeDDLEvent implement the DDLEventBatchEncoder interface func (e *encoder) EncodeDDLEvent(event *model.DDLEvent) (*common.Message, error) { m := newDDLMessage(event) + log.Info("fizz:encode ddl event", zap.Any("event", event)) value, err := json.Marshal(m) if err != nil { return nil, cerror.WrapError(cerror.ErrEncodeFailed, err) From 851343d773f08d58368e65448e0810aec3fa9375 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 27 Dec 2023 10:25:07 +0800 Subject: [PATCH 02/20] simple protocol send bootstrap event periodically --- pkg/sink/codec/bootstrap.go | 261 ++++++++++++++++++++++++++++++++++++ 1 file changed, 261 insertions(+) create mode 100644 pkg/sink/codec/bootstrap.go diff --git a/pkg/sink/codec/bootstrap.go b/pkg/sink/codec/bootstrap.go new file mode 100644 index 00000000000..2d5fc0c6ae8 --- /dev/null +++ b/pkg/sink/codec/bootstrap.go @@ -0,0 +1,261 @@ +// Copyright 2022 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 codec + +import ( + "context" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "go.uber.org/zap" +) + +const ( + bootstrapWorkerTickerInterval = 10 * time.Second + bootstrapWorkerGCInterval = 30 * time.Second + + defaultSendBootstrapInterval = 1 * time.Minute + defaultSendBootstrapInMsgCount = 1000 + defaultMaxInactiveDuration = 5 * time.Minute +) + +type bootstrapWorker struct { + activeTables sync.Map + builder RowEventEncoderBuilder + sendBootstrapInterval time.Duration + sendBootstrapInMsgCount int32 + // maxInactiveDuration is the max duration that a table can be inactive + maxInactiveDuration time.Duration + outCh chan<- *future +} + +// newBootstrapWorker creates a new bootstrapGenerator instance +func newBootstrapWorker( + outCh chan<- *future, + builder RowEventEncoderBuilder, + sendBootstrapInterval time.Duration, + sendBootstrapInMsgCount int32, + maxInactiveDuration time.Duration, +) *bootstrapWorker { + return &bootstrapWorker{ + outCh: outCh, + builder: builder, + activeTables: sync.Map{}, + sendBootstrapInterval: sendBootstrapInterval, + sendBootstrapInMsgCount: sendBootstrapInMsgCount, + maxInactiveDuration: maxInactiveDuration, + } +} + +func (b *bootstrapWorker) run(ctx context.Context) error { + sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) + defer sendTicker.Stop() + gcTicker := time.NewTicker(bootstrapWorkerGCInterval) + defer gcTicker.Stop() + errCh := make(chan error, 1) + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-sendTicker.C: + b.activeTables.Range(func(key, value interface{}) bool { + table := value.(*tableStatus) + err := b.sendBootstrapMsg(ctx, table) + if err != nil { + errCh <- err + return false + } + return true + }) + case <-gcTicker.C: + b.gcInactiveTables() + case err := <-errCh: + return err + } + } +} + +func (b *bootstrapWorker) addEvent( + ctx context.Context, + key TopicPartitionKey, + row *model.RowChangedEvent, +) error { + table, ok := b.activeTables.Load(row.Table.TableID) + if !ok { + tb := newTableStatus(key, row) + // Send bootstrap message immediately when a new table is added + err := b.sendBootstrapMsg(ctx, tb) + if err != nil { + return errors.Trace(err) + } + log.Info("fizz: a new table is added to the bootstrap worker, send bootstrap message immediately", + zap.String("topic", tb.topic), + zap.Int64("tableID", int64(tb.id))) + b.activeTables.Store(tb.id, tb) + } else { + // If the table is already in the activeTables, update its status. + table.(*tableStatus).update(key, row) + } + return nil +} + +// sendBootstrapMsg sends a bootstrap message if the table meets the condition +// 1. The time since last bootstrap message sent is larger than sendBootstrapInterval +// 2. The received row event count since last bootstrap message sent is larger than sendBootstrapInMsgCount +// Note: It is a blocking method, it will block if the outCh is full. +func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatus) error { + if !table.shouldSendBootstrapMsg( + b.sendBootstrapInterval, + b.sendBootstrapInMsgCount) { + return nil + } + tableInfo := table.tableInfo.Load().(*model.TableInfo) + events, err := b.generateEvents(ctx, table.topic, table.totalPartition.Load(), tableInfo) + if err != nil { + return errors.Trace(err) + } + for _, event := range events { + select { + case <-ctx.Done(): + return ctx.Err() + case b.outCh <- event: + } + } + table.reset() + return nil +} + +func (b *bootstrapWorker) generateEvents( + ctx context.Context, + topic string, + totalPartition int32, + tableInfo *model.TableInfo, +) ([]*future, error) { + events := make([]*future, 0, int(totalPartition)) + // Bootstrap messages of a table should be sent to all partitions. + for i := 0; i < int(totalPartition); i++ { + encoder := b.builder.Build() + msg, err := encoder.EncodeDDLEvent(&model.DDLEvent{ + StartTs: 0, + CommitTs: 0, + TableInfo: tableInfo, + IsBootstrap: true, + }) + if err != nil { + return nil, errors.Trace(err) + } + + key := TopicPartitionKey{ + Topic: topic, + Partition: int32(i), + } + + future := &future{ + Key: key, + Messages: []*common.Message{msg}, + done: make(chan struct{}), + } + close(future.done) + } + return events, nil +} + +func (b *bootstrapWorker) gcInactiveTables() { + b.activeTables.Range(func(key, value interface{}) bool { + table := value.(*tableStatus) + if table.isInactive(b.maxInactiveDuration) { + b.activeTables.Delete(key) + } + return true + }) +} + +// tableStatus is used to record the status of a table +type tableStatus struct { + // id is the table's ID, it will not change + id model.TableID + // topic is the table's topic, it will not change + topic string + // All fields below are concurrently accessed, please use atomic operations. + // totalPartition is the total number of partitions of the table's topic + totalPartition atomic.Int32 + // counter is the number of row event sent since last bootstrap message sent + // It is used to check if the bootstrap message should be sent + counter atomic.Int32 + // lastMsgReceivedTime is the last time the row event is received + // It is used to check if the table is inactive + lastMsgReceivedTime atomic.Value + // lastSendTime is the last time the bootstrap message is sent + // It is used to check if the bootstrap message should be sent + lastSendTime atomic.Value + // version is the table version + // It is used to check if the table schema is changed since last bootstrap message sent + version atomic.Uint64 + // tableInfo is the tableInfo of the table + // It is used to generate bootstrap message + tableInfo atomic.Value +} + +func newTableStatus(key TopicPartitionKey, row *model.RowChangedEvent) *tableStatus { + res := &tableStatus{ + id: row.Table.TableID, + topic: key.Topic, + } + res.counter.Add(1) + res.totalPartition.Store(key.TotalPartition) + res.lastMsgReceivedTime.Store(time.Unix(0, 0)) + res.lastSendTime.Store(time.Unix(0, 0)) + res.version.Store(row.TableInfo.UpdateTS) + res.tableInfo.Store(row.TableInfo) + return res +} + +func (t *tableStatus) shouldSendBootstrapMsg( + sendBootstrapInterval time.Duration, + sendBootstrapMsgCountInterval int32, +) bool { + lastSendTime := t.lastSendTime.Load().(time.Time) + return time.Since(lastSendTime) > sendBootstrapInterval || + t.counter.Load() > sendBootstrapMsgCountInterval +} + +func (t *tableStatus) update(key TopicPartitionKey, row *model.RowChangedEvent) { + t.counter.Add(1) + t.lastMsgReceivedTime.Store(time.Now()) + + if t.totalPartition.Load() != key.TotalPartition { + t.totalPartition.Store(key.TotalPartition) + } + + if t.version.Load() != row.TableInfo.UpdateTS { + t.version.Store(row.TableInfo.UpdateTS) + t.tableInfo.Store(row.TableInfo) + } +} + +func (t *tableStatus) isInactive(maxInactiveDuration time.Duration) bool { + lastMsgReceivedTime := t.lastMsgReceivedTime.Load().(time.Time) + return time.Since(lastMsgReceivedTime) > maxInactiveDuration +} + +func (t *tableStatus) reset() { + t.lastSendTime.Store(time.Now()) + t.counter.Store(0) +} From 230ce0859a68c02c2b63a0b0634f64a177b8aa60 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 27 Dec 2023 17:24:15 +0800 Subject: [PATCH 03/20] add unit test --- cdc/sink/dmlsink/mq/worker.go | 6 + pkg/sink/codec/bootstrap.go | 261 -------------------------------- pkg/sink/codec/encoder.go | 42 +++++ pkg/sink/codec/encoder_group.go | 8 +- 4 files changed, 52 insertions(+), 265 deletions(-) delete mode 100644 pkg/sink/codec/bootstrap.go diff --git a/cdc/sink/dmlsink/mq/worker.go b/cdc/sink/dmlsink/mq/worker.go index 93f22f211dc..41a769f7862 100644 --- a/cdc/sink/dmlsink/mq/worker.go +++ b/cdc/sink/dmlsink/mq/worker.go @@ -297,7 +297,13 @@ func (w *worker) sendMessages(ctx context.Context) error { if err = future.Ready(ctx); err != nil { return errors.Trace(err) } + for _, message := range future.Messages { + log.Info("fizz: MQ sink worker is sending message", + zap.Any("key", future.Key), + zap.Any("messageKey", message.Key), + zap.Any("messageValue", message.Value)) + start := time.Now() if err = w.statistics.RecordBatchExecution(func() (int, int64, error) { message.SetPartitionKey(future.Key.PartitionKey) diff --git a/pkg/sink/codec/bootstrap.go b/pkg/sink/codec/bootstrap.go deleted file mode 100644 index 2d5fc0c6ae8..00000000000 --- a/pkg/sink/codec/bootstrap.go +++ /dev/null @@ -1,261 +0,0 @@ -// Copyright 2022 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 codec - -import ( - "context" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/sink/codec/common" - "go.uber.org/zap" -) - -const ( - bootstrapWorkerTickerInterval = 10 * time.Second - bootstrapWorkerGCInterval = 30 * time.Second - - defaultSendBootstrapInterval = 1 * time.Minute - defaultSendBootstrapInMsgCount = 1000 - defaultMaxInactiveDuration = 5 * time.Minute -) - -type bootstrapWorker struct { - activeTables sync.Map - builder RowEventEncoderBuilder - sendBootstrapInterval time.Duration - sendBootstrapInMsgCount int32 - // maxInactiveDuration is the max duration that a table can be inactive - maxInactiveDuration time.Duration - outCh chan<- *future -} - -// newBootstrapWorker creates a new bootstrapGenerator instance -func newBootstrapWorker( - outCh chan<- *future, - builder RowEventEncoderBuilder, - sendBootstrapInterval time.Duration, - sendBootstrapInMsgCount int32, - maxInactiveDuration time.Duration, -) *bootstrapWorker { - return &bootstrapWorker{ - outCh: outCh, - builder: builder, - activeTables: sync.Map{}, - sendBootstrapInterval: sendBootstrapInterval, - sendBootstrapInMsgCount: sendBootstrapInMsgCount, - maxInactiveDuration: maxInactiveDuration, - } -} - -func (b *bootstrapWorker) run(ctx context.Context) error { - sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) - defer sendTicker.Stop() - gcTicker := time.NewTicker(bootstrapWorkerGCInterval) - defer gcTicker.Stop() - errCh := make(chan error, 1) - - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-sendTicker.C: - b.activeTables.Range(func(key, value interface{}) bool { - table := value.(*tableStatus) - err := b.sendBootstrapMsg(ctx, table) - if err != nil { - errCh <- err - return false - } - return true - }) - case <-gcTicker.C: - b.gcInactiveTables() - case err := <-errCh: - return err - } - } -} - -func (b *bootstrapWorker) addEvent( - ctx context.Context, - key TopicPartitionKey, - row *model.RowChangedEvent, -) error { - table, ok := b.activeTables.Load(row.Table.TableID) - if !ok { - tb := newTableStatus(key, row) - // Send bootstrap message immediately when a new table is added - err := b.sendBootstrapMsg(ctx, tb) - if err != nil { - return errors.Trace(err) - } - log.Info("fizz: a new table is added to the bootstrap worker, send bootstrap message immediately", - zap.String("topic", tb.topic), - zap.Int64("tableID", int64(tb.id))) - b.activeTables.Store(tb.id, tb) - } else { - // If the table is already in the activeTables, update its status. - table.(*tableStatus).update(key, row) - } - return nil -} - -// sendBootstrapMsg sends a bootstrap message if the table meets the condition -// 1. The time since last bootstrap message sent is larger than sendBootstrapInterval -// 2. The received row event count since last bootstrap message sent is larger than sendBootstrapInMsgCount -// Note: It is a blocking method, it will block if the outCh is full. -func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatus) error { - if !table.shouldSendBootstrapMsg( - b.sendBootstrapInterval, - b.sendBootstrapInMsgCount) { - return nil - } - tableInfo := table.tableInfo.Load().(*model.TableInfo) - events, err := b.generateEvents(ctx, table.topic, table.totalPartition.Load(), tableInfo) - if err != nil { - return errors.Trace(err) - } - for _, event := range events { - select { - case <-ctx.Done(): - return ctx.Err() - case b.outCh <- event: - } - } - table.reset() - return nil -} - -func (b *bootstrapWorker) generateEvents( - ctx context.Context, - topic string, - totalPartition int32, - tableInfo *model.TableInfo, -) ([]*future, error) { - events := make([]*future, 0, int(totalPartition)) - // Bootstrap messages of a table should be sent to all partitions. - for i := 0; i < int(totalPartition); i++ { - encoder := b.builder.Build() - msg, err := encoder.EncodeDDLEvent(&model.DDLEvent{ - StartTs: 0, - CommitTs: 0, - TableInfo: tableInfo, - IsBootstrap: true, - }) - if err != nil { - return nil, errors.Trace(err) - } - - key := TopicPartitionKey{ - Topic: topic, - Partition: int32(i), - } - - future := &future{ - Key: key, - Messages: []*common.Message{msg}, - done: make(chan struct{}), - } - close(future.done) - } - return events, nil -} - -func (b *bootstrapWorker) gcInactiveTables() { - b.activeTables.Range(func(key, value interface{}) bool { - table := value.(*tableStatus) - if table.isInactive(b.maxInactiveDuration) { - b.activeTables.Delete(key) - } - return true - }) -} - -// tableStatus is used to record the status of a table -type tableStatus struct { - // id is the table's ID, it will not change - id model.TableID - // topic is the table's topic, it will not change - topic string - // All fields below are concurrently accessed, please use atomic operations. - // totalPartition is the total number of partitions of the table's topic - totalPartition atomic.Int32 - // counter is the number of row event sent since last bootstrap message sent - // It is used to check if the bootstrap message should be sent - counter atomic.Int32 - // lastMsgReceivedTime is the last time the row event is received - // It is used to check if the table is inactive - lastMsgReceivedTime atomic.Value - // lastSendTime is the last time the bootstrap message is sent - // It is used to check if the bootstrap message should be sent - lastSendTime atomic.Value - // version is the table version - // It is used to check if the table schema is changed since last bootstrap message sent - version atomic.Uint64 - // tableInfo is the tableInfo of the table - // It is used to generate bootstrap message - tableInfo atomic.Value -} - -func newTableStatus(key TopicPartitionKey, row *model.RowChangedEvent) *tableStatus { - res := &tableStatus{ - id: row.Table.TableID, - topic: key.Topic, - } - res.counter.Add(1) - res.totalPartition.Store(key.TotalPartition) - res.lastMsgReceivedTime.Store(time.Unix(0, 0)) - res.lastSendTime.Store(time.Unix(0, 0)) - res.version.Store(row.TableInfo.UpdateTS) - res.tableInfo.Store(row.TableInfo) - return res -} - -func (t *tableStatus) shouldSendBootstrapMsg( - sendBootstrapInterval time.Duration, - sendBootstrapMsgCountInterval int32, -) bool { - lastSendTime := t.lastSendTime.Load().(time.Time) - return time.Since(lastSendTime) > sendBootstrapInterval || - t.counter.Load() > sendBootstrapMsgCountInterval -} - -func (t *tableStatus) update(key TopicPartitionKey, row *model.RowChangedEvent) { - t.counter.Add(1) - t.lastMsgReceivedTime.Store(time.Now()) - - if t.totalPartition.Load() != key.TotalPartition { - t.totalPartition.Store(key.TotalPartition) - } - - if t.version.Load() != row.TableInfo.UpdateTS { - t.version.Store(row.TableInfo.UpdateTS) - t.tableInfo.Store(row.TableInfo) - } -} - -func (t *tableStatus) isInactive(maxInactiveDuration time.Duration) bool { - lastMsgReceivedTime := t.lastMsgReceivedTime.Load().(time.Time) - return time.Since(lastMsgReceivedTime) > maxInactiveDuration -} - -func (t *tableStatus) reset() { - t.lastSendTime.Store(time.Now()) - t.counter.Store(0) -} diff --git a/pkg/sink/codec/encoder.go b/pkg/sink/codec/encoder.go index c5c94b61d77..503c2191c8f 100644 --- a/pkg/sink/codec/encoder.go +++ b/pkg/sink/codec/encoder.go @@ -91,3 +91,45 @@ type TopicPartitionKey struct { PartitionKey string TotalPartition int32 } + +// MockRowEventEncoderBuilder is a mock implementation of RowEventEncoderBuilder +type MockRowEventEncoderBuilder struct{} + +// Build implement the RowEventEncoderBuilder interface +func (m *MockRowEventEncoderBuilder) Build() RowEventEncoder { + return &MockRowEventEncoder{} +} + +// CleanMetrics implement the RowEventEncoderBuilder interface +func (m *MockRowEventEncoderBuilder) CleanMetrics() { + // Clean up metrics if needed +} + +// MockRowEventEncoder is a mock implementation of RowEventEncoder +type MockRowEventEncoder struct{} + +// EncodeCheckpointEvent implement the DDLEventBatchEncoder interface +func (m *MockRowEventEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { + // Implement the encoding logic for checkpoint event + return nil, nil +} + +// EncodeDDLEvent implement the DDLEventBatchEncoder interface +func (m *MockRowEventEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { + // Implement the encoding logic for DDL event + return nil, nil +} + +// AppendRowChangedEvent implement the RowEventEncoder interface +func (m *MockRowEventEncoder) AppendRowChangedEvent( + ctx context.Context, tableID string, event *model.RowChangedEvent, callback func(), +) error { + // Implement the logic for appending row changed event + return nil +} + +// Build implement the RowEventEncoder interface +func (m *MockRowEventEncoder) Build() []*common.Message { + // Implement the logic for building the batch and returning the bytes of key and value + return nil +} diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index 0f5c76954c7..c4de0f4739e 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -65,7 +65,8 @@ type encoderGroup struct { func NewEncoderGroup( cfg *config.SinkConfig, builder RowEventEncoderBuilder, - concurrency int, changefeedID model.ChangeFeedID, + concurrency int, + changefeedID model.ChangeFeedID, ) *encoderGroup { if concurrency <= 0 { concurrency = config.DefaultEncoderGroupConcurrency @@ -75,7 +76,7 @@ func NewEncoderGroup( for i := 0; i < concurrency; i++ { inputCh[i] = make(chan *future, defaultInputChanSize) } - outCh := make(chan *future, defaultInputChanSize) + outCh := make(chan *future, defaultInputChanSize*concurrency) var bootstrapWorker *bootstrapWorker if *cfg.Protocol == config.ProtocolSimple.String() { @@ -99,8 +100,7 @@ func NewEncoderGroup( } return &encoderGroup{ - changefeedID: changefeedID, - + changefeedID: changefeedID, builder: builder, concurrency: concurrency, inputCh: inputCh, From dbc944481453322344cdd4ca308599d258063bc3 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Wed, 27 Dec 2023 17:25:13 +0800 Subject: [PATCH 04/20] add unit test --- pkg/sink/codec/bootstraper.go | 269 +++++++++++++++++++++++++++++ pkg/sink/codec/bootstraper_test.go | 134 ++++++++++++++ 2 files changed, 403 insertions(+) create mode 100644 pkg/sink/codec/bootstraper.go create mode 100644 pkg/sink/codec/bootstraper_test.go diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go new file mode 100644 index 00000000000..ef82cfe52b2 --- /dev/null +++ b/pkg/sink/codec/bootstraper.go @@ -0,0 +1,269 @@ +// Copyright 2022 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 codec + +import ( + "context" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + "go.uber.org/zap" +) + +const ( + bootstrapWorkerTickerInterval = 10 * time.Second + bootstrapWorkerGCInterval = 30 * time.Second + + defaultSendBootstrapInterval = 1 * time.Minute + defaultSendBootstrapInMsgCount = 1000 + defaultMaxInactiveDuration = 5 * time.Minute +) + +type bootstrapWorker struct { + activeTables sync.Map + builder RowEventEncoderBuilder + sendBootstrapInterval time.Duration + sendBootstrapInMsgCount int32 + // maxInactiveDuration is the max duration that a table can be inactive + maxInactiveDuration time.Duration + outCh chan<- *future +} + +// newBootstrapWorker creates a new bootstrapGenerator instance +func newBootstrapWorker( + outCh chan<- *future, + builder RowEventEncoderBuilder, + sendBootstrapInterval time.Duration, + sendBootstrapInMsgCount int32, + maxInactiveDuration time.Duration, +) *bootstrapWorker { + return &bootstrapWorker{ + outCh: outCh, + builder: builder, + activeTables: sync.Map{}, + sendBootstrapInterval: sendBootstrapInterval, + sendBootstrapInMsgCount: sendBootstrapInMsgCount, + maxInactiveDuration: maxInactiveDuration, + } +} + +func (b *bootstrapWorker) run(ctx context.Context) error { + log.Info("fizz: bootstrap worker is started") + sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) + defer sendTicker.Stop() + gcTicker := time.NewTicker(bootstrapWorkerGCInterval) + defer gcTicker.Stop() + errCh := make(chan error, 1) + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-sendTicker.C: + b.activeTables.Range(func(key, value interface{}) bool { + table := value.(*tableStatus) + err := b.sendBootstrapMsg(ctx, table) + if err != nil { + errCh <- err + return false + } + return true + }) + case <-gcTicker.C: + b.gcInactiveTables() + case err := <-errCh: + return err + } + } +} + +func (b *bootstrapWorker) addEvent( + ctx context.Context, + key TopicPartitionKey, + row *model.RowChangedEvent, +) error { + table, ok := b.activeTables.Load(row.Table.TableID) + if !ok { + tb := newTableStatus(key, row) + b.activeTables.Store(tb.id, tb) + // Send bootstrap message immediately when a new table is added + err := b.sendBootstrapMsg(ctx, tb) + if err != nil { + return errors.Trace(err) + } + log.Info("fizz: a new table is added to the bootstrap worker, send bootstrap message immediately", + zap.String("topic", tb.topic), + zap.Int64("tableID", int64(tb.id))) + } else { + // If the table is already in the activeTables, update its status. + table.(*tableStatus).update(key, row) + } + return nil +} + +// sendBootstrapMsg sends a bootstrap message if the table meets the condition +// 1. The time since last bootstrap message sent is larger than sendBootstrapInterval +// 2. The received row event count since last bootstrap message sent is larger than sendBootstrapInMsgCount +// Note: It is a blocking method, it will block if the outCh is full. +func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatus) error { + if !table.shouldSendBootstrapMsg( + b.sendBootstrapInterval, + b.sendBootstrapInMsgCount) { + return nil + } + table.reset() + log.Info("fizz: bootstrap worker is sending bootstrap message", zap.Any("table", table.id)) + tableInfo := table.tableInfo.Load().(*model.TableInfo) + events, err := b.generateEvents(ctx, table.topic, table.totalPartition.Load(), tableInfo) + if err != nil { + return errors.Trace(err) + } + for _, event := range events { + select { + case <-ctx.Done(): + return ctx.Err() + case b.outCh <- event: + log.Info("fizz: bootstrap message is sent", + zap.Any("msgKey", event.Key), + zap.Any("table", table.id)) + } + } + log.Info("fizz: bootstrap message are sent", zap.Any("table", table.id)) + return nil +} + +func (b *bootstrapWorker) generateEvents( + ctx context.Context, + topic string, + totalPartition int32, + tableInfo *model.TableInfo, +) ([]*future, error) { + res := make([]*future, 0, int(totalPartition)) + // Bootstrap messages of a table should be sent to all partitions. + for i := 0; i < int(totalPartition); i++ { + encoder := b.builder.Build() + msg, err := encoder.EncodeDDLEvent(&model.DDLEvent{ + StartTs: 0, + CommitTs: 0, + TableInfo: tableInfo, + IsBootstrap: true, + }) + if err != nil { + return nil, errors.Trace(err) + } + + key := TopicPartitionKey{ + Topic: topic, + Partition: int32(i), + TotalPartition: totalPartition, + } + + future := &future{ + Key: key, + done: make(chan struct{}), + } + future.Messages = append(future.Messages, msg) + close(future.done) + res = append(res, future) + } + return res, nil +} + +func (b *bootstrapWorker) gcInactiveTables() { + b.activeTables.Range(func(key, value interface{}) bool { + table := value.(*tableStatus) + if !table.isActive(b.maxInactiveDuration) { + log.Info("fizz: a table is removed from the bootstrap worker", zap.Any("table", table.id)) + b.activeTables.Delete(key) + } + return true + }) +} + +// tableStatus is used to record the status of a table +type tableStatus struct { + // id is the table's ID, it will not change + id int64 + // topic is the table's topic, it will not change + topic string + // All fields below are concurrently accessed, please use atomic operations. + // totalPartition is the total number of partitions of the table's topic + totalPartition atomic.Int32 + // counter is the number of row event sent since last bootstrap message sent + // It is used to check if the bootstrap message should be sent + counter atomic.Int32 + // lastMsgReceivedTime is the last time the row event is received + // It is used to check if the table is inactive + lastMsgReceivedTime atomic.Value + // lastSendTime is the last time the bootstrap message is sent + // It is used to check if the bootstrap message should be sent + lastSendTime atomic.Value + // version is the table version + // It is used to check if the table schema is changed since last bootstrap message sent + version atomic.Uint64 + // tableInfo is the tableInfo of the table + // It is used to generate bootstrap message + tableInfo atomic.Value +} + +func newTableStatus(key TopicPartitionKey, row *model.RowChangedEvent) *tableStatus { + res := &tableStatus{ + id: row.Table.TableID, + topic: key.Topic, + } + res.counter.Add(1) + res.totalPartition.Store(key.TotalPartition) + res.lastMsgReceivedTime.Store(time.Now()) + res.lastSendTime.Store(time.Unix(0, 0)) + res.version.Store(row.TableInfo.UpdateTS) + res.tableInfo.Store(row.TableInfo) + return res +} + +func (t *tableStatus) shouldSendBootstrapMsg( + sendBootstrapInterval time.Duration, + sendBootstrapMsgCountInterval int32, +) bool { + lastSendTime := t.lastSendTime.Load().(time.Time) + return time.Since(lastSendTime) >= sendBootstrapInterval || + t.counter.Load() >= sendBootstrapMsgCountInterval +} + +func (t *tableStatus) update(key TopicPartitionKey, row *model.RowChangedEvent) { + t.counter.Add(1) + t.lastMsgReceivedTime.Store(time.Now()) + + if t.totalPartition.Load() != key.TotalPartition { + t.totalPartition.Store(key.TotalPartition) + } + + if t.version.Load() != row.TableInfo.UpdateTS { + t.version.Store(row.TableInfo.UpdateTS) + t.tableInfo.Store(row.TableInfo) + } +} + +func (t *tableStatus) isActive(maxInactiveDuration time.Duration) bool { + lastMsgReceivedTime := t.lastMsgReceivedTime.Load().(time.Time) + return time.Since(lastMsgReceivedTime) < maxInactiveDuration +} + +func (t *tableStatus) reset() { + t.lastSendTime.Store(time.Now()) + t.counter.Store(0) +} diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go new file mode 100644 index 00000000000..ff511da6f85 --- /dev/null +++ b/pkg/sink/codec/bootstraper_test.go @@ -0,0 +1,134 @@ +// Copyright 2022 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 codec + +import ( + "context" + "testing" + "time" + + timodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tiflow/cdc/model" + "github.com/stretchr/testify/require" +) + +func getMockTableStatus() (TopicPartitionKey, *model.RowChangedEvent, *tableStatus) { + tableInfo := &model.TableInfo{ + TableInfo: &timodel.TableInfo{ + UpdateTS: 1, + }, + } + table := &model.TableName{ + Schema: "test", + Table: "t1", + TableID: 1, + } + key := TopicPartitionKey{ + Topic: "test.t1", + Partition: 1, + TotalPartition: 3, + } + row := &model.RowChangedEvent{ + TableInfo: tableInfo, + Table: table, + } + // case 1: A new added table should send bootstrap message immediately + tb1 := newTableStatus(key, row) + return key, row, tb1 +} + +func TestShouldSendBootstrapMsg(t *testing.T) { + t.Parallel() + + _, _, tb1 := getMockTableStatus() + + // case 1: A new added table should send bootstrap message immediately + require.True(t, tb1. + shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount)) + + // case 2: A table which has sent bootstrap message should not send bootstrap message + tb1.lastSendTime.Store(time.Now()) + require.False(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount)) + + // case 3: When the table receive message more than sendBootstrapInMsgCount, + // it should send bootstrap message + tb1.counter.Add(int32(defaultSendBootstrapInMsgCount)) + require.True(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount)) + + // case 4: When the table does not send bootstrap message for a sendBootstrapInterval time, + // it should send bootstrap message + tb1.lastSendTime.Store(time.Now().Add(-defaultSendBootstrapInterval)) + require.True(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount)) +} + +func TestIsActive(t *testing.T) { + t.Parallel() + key, row, tb1 := getMockTableStatus() + // case 1: A new added table should be active + require.True(t, tb1.isActive(defaultMaxInactiveDuration)) + + // case 2: A table which does not receive message for a long time should be inactive + tb1.lastMsgReceivedTime.Store(time.Now().Add(-defaultMaxInactiveDuration)) + require.False(t, tb1.isActive(defaultMaxInactiveDuration)) + + // case 3: A table which receive message recently should be active + // Note: A table's update method will be call any time it receive message + // So use update method to simulate the table receive message + tb1.update(key, row) + require.True(t, tb1.isActive(defaultMaxInactiveDuration)) +} + +func TestBootstrapWorker(t *testing.T) { + t.Parallel() + // new builder + builder := &MockRowEventEncoderBuilder{} + + outCh := make(chan *future, defaultInputChanSize) + worker := newBootstrapWorker(outCh, + builder, + defaultSendBootstrapInterval, + defaultSendBootstrapInMsgCount, + defaultMaxInactiveDuration) + + // Start the worker in a separate goroutine + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + _ = worker.run(ctx) + }() + + // case 1: A new added table should send bootstrap message immediately + // The messages number should be equal to the total partition number + // Event if we send the same table twice, it should only send bootstrap message once + key, row, _ := getMockTableStatus() + err := worker.addEvent(ctx, key, row) + require.NoError(t, err) + err = worker.addEvent(ctx, key, row) + require.NoError(t, err) + var msgCount int32 + sctx, sancel := context.WithTimeout(context.Background(), 3*time.Second) + defer sancel() + for { + select { + case future := <-outCh: + require.NotNil(t, future) + require.Equal(t, key.Topic, future.Key.Topic) + require.Equal(t, key.TotalPartition, future.Key.TotalPartition) + msgCount++ + case <-sctx.Done(): + require.Equal(t, key.TotalPartition, msgCount) + return + } + } +} From 22332ed672231d375507b25a9156e7755ec7bd0b Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 12:14:00 +0800 Subject: [PATCH 05/20] add configuration and integration test case --- cdc/api/v2/model.go | 17 ++++ cdc/api/v2/model_test.go | 2 + cdc/sink/dmlsink/mq/worker.go | 5 -- pkg/cmd/util/helper_test.go | 4 + pkg/config/config_test_data.go | 12 ++- pkg/config/replica_config.go | 2 + pkg/config/sink.go | 7 +- pkg/filter/sql_event_filter.go | 3 +- pkg/sink/codec/bootstraper.go | 26 +++--- pkg/sink/codec/bootstraper_test.go | 15 ++-- pkg/sink/codec/encoder_group.go | 8 +- pkg/sink/codec/simple/decoder.go | 81 ++++++++++++++----- pkg/sink/codec/simple/encoder.go | 1 - tests/integration_tests/api_v2/cases.go | 22 ++--- tests/integration_tests/api_v2/model.go | 24 +++--- .../kafka_simple_basic/conf/changefeed.toml | 4 + .../kafka_simple_basic/data/data.sql | 46 +---------- .../kafka_simple_basic/run.sh | 11 ++- 18 files changed, 169 insertions(+), 121 deletions(-) diff --git a/cdc/api/v2/model.go b/cdc/api/v2/model.go index 5495a0de177..9150a0ecd78 100644 --- a/cdc/api/v2/model.go +++ b/cdc/api/v2/model.go @@ -487,6 +487,13 @@ func (c *ReplicaConfig) toInternalReplicaConfigWithOriginConfig( res.Sink.AdvanceTimeoutInSec = util.AddressOf(*c.Sink.AdvanceTimeoutInSec) } + if c.Sink.SendBootstrapIntervalInSec != nil { + res.Sink.SendBootstrapIntervalInSec = util.AddressOf(*c.Sink.SendBootstrapIntervalInSec) + } + + if c.Sink.SendBootstrapInMsgCount != nil { + res.Sink.SendBootstrapInMsgCount = util.AddressOf(*c.Sink.SendBootstrapInMsgCount) + } } if c.Mounter != nil { res.Mounter = &config.MounterConfig{ @@ -776,6 +783,14 @@ func ToAPIReplicaConfig(c *config.ReplicaConfig) *ReplicaConfig { if cloned.Sink.AdvanceTimeoutInSec != nil { res.Sink.AdvanceTimeoutInSec = util.AddressOf(*cloned.Sink.AdvanceTimeoutInSec) } + + if cloned.Sink.SendBootstrapIntervalInSec != nil { + res.Sink.SendBootstrapIntervalInSec = util.AddressOf(*cloned.Sink.SendBootstrapIntervalInSec) + } + + if cloned.Sink.SendBootstrapInMsgCount != nil { + res.Sink.SendBootstrapInMsgCount = util.AddressOf(*cloned.Sink.SendBootstrapInMsgCount) + } } if cloned.Consistent != nil { res.Consistent = &ConsistentConfig{ @@ -950,6 +965,8 @@ type SinkConfig struct { MySQLConfig *MySQLConfig `json:"mysql_config,omitempty"` CloudStorageConfig *CloudStorageConfig `json:"cloud_storage_config,omitempty"` AdvanceTimeoutInSec *uint `json:"advance_timeout,omitempty"` + SendBootstrapIntervalInSec *int64 `json:"send_bootstrap_interval_in_sec,omitempty"` + SendBootstrapInMsgCount *int32 `json:"send_bootstrap_in_msg_count,omitempty"` } // CSVConfig denotes the csv config diff --git a/cdc/api/v2/model_test.go b/cdc/api/v2/model_test.go index e8d89c0e944..8b4b1c90b2e 100644 --- a/cdc/api/v2/model_test.go +++ b/cdc/api/v2/model_test.go @@ -59,6 +59,8 @@ var defaultAPIConfig = &ReplicaConfig{ DeleteOnlyOutputHandleKeyColumns: util.AddressOf(false), ContentCompatible: util.AddressOf(false), AdvanceTimeoutInSec: util.AddressOf(uint(150)), + SendBootstrapIntervalInSec: util.AddressOf(int64(120)), + SendBootstrapInMsgCount: util.AddressOf(int32(10000)), }, Consistent: &ConsistentConfig{ Level: "none", diff --git a/cdc/sink/dmlsink/mq/worker.go b/cdc/sink/dmlsink/mq/worker.go index 41a769f7862..0df7b1b19f5 100644 --- a/cdc/sink/dmlsink/mq/worker.go +++ b/cdc/sink/dmlsink/mq/worker.go @@ -299,11 +299,6 @@ func (w *worker) sendMessages(ctx context.Context) error { } for _, message := range future.Messages { - log.Info("fizz: MQ sink worker is sending message", - zap.Any("key", future.Key), - zap.Any("messageKey", message.Key), - zap.Any("messageValue", message.Value)) - start := time.Now() if err = w.statistics.RecordBatchExecution(func() (int, int64, error) { message.SetPartitionKey(future.Key.PartitionKey) diff --git a/pkg/cmd/util/helper_test.go b/pkg/cmd/util/helper_test.go index d016e56f5dc..7ff95fa0676 100644 --- a/pkg/cmd/util/helper_test.go +++ b/pkg/cmd/util/helper_test.go @@ -215,6 +215,8 @@ func TestAndWriteExampleReplicaTOML(t *testing.T) { ContentCompatible: util.AddressOf(false), Protocol: util.AddressOf("open-protocol"), AdvanceTimeoutInSec: util.AddressOf(uint(150)), + SendBootstrapIntervalInSec: util.AddressOf(int64(120)), + SendBootstrapInMsgCount: util.AddressOf(int32(10000)), }, cfg.Sink) } @@ -249,6 +251,8 @@ func TestAndWriteStorageSinkTOML(t *testing.T) { DeleteOnlyOutputHandleKeyColumns: util.AddressOf(false), ContentCompatible: util.AddressOf(false), AdvanceTimeoutInSec: util.AddressOf(uint(150)), + SendBootstrapIntervalInSec: util.AddressOf(int64(120)), + SendBootstrapInMsgCount: util.AddressOf(int32(10000)), }, cfg.Sink) } diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 6c639c5302e..f40aed6ce39 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -65,7 +65,9 @@ const ( "large-message-handle-compression": "", "claim-check-storage-uri": "" }, - "advance-timeout-in-sec": 150 + "advance-timeout-in-sec": 150, + "send-bootstrap-interval-in-sec": 120, + "send-bootstrap-in-msg-count": 10000 }, "consistent": { "level": "none", @@ -318,7 +320,9 @@ const ( "file-size": 1024, "output-column-id":false }, - "advance-timeout-in-sec": 150 + "advance-timeout-in-sec": 150, + "send-bootstrap-interval-in-sec": 120, + "send-bootstrap-in-msg-count": 10000 }, "consistent": { "level": "none", @@ -479,7 +483,9 @@ const ( "file-size": 1024, "output-column-id":false }, - "advance-timeout-in-sec": 150 + "advance-timeout-in-sec": 150, + "send-bootstrap-interval-in-sec": 120, + "send-bootstrap-in-msg-count": 10000 }, "consistent": { "level": "none", diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index adbafb6b44a..4b0c282eeba 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -78,6 +78,8 @@ var defaultReplicaConfig = &ReplicaConfig{ ContentCompatible: util.AddressOf(false), TiDBSourceID: 1, AdvanceTimeoutInSec: util.AddressOf(DefaultAdvanceTimeoutInSec), + SendBootstrapIntervalInSec: util.AddressOf(DefaultSendBootstrapIntervalInSec), + SendBootstrapInMsgCount: util.AddressOf(DefaultSendBootstrapInMsgCount), }, Consistent: &ConsistentConfig{ Level: "none", diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 9ec59e414f1..5871e71267a 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -81,6 +81,11 @@ const ( // DefaultEncoderGroupConcurrency is the default concurrency of encoder group. DefaultEncoderGroupConcurrency = 32 + + // DefaultSendBootstrapIntervalInSec is the default interval to send bootstrap message. + DefaultSendBootstrapIntervalInSec = int64(120) + // DefaultSendBootstrapInMsgCount is the default number of messages to send bootstrap message. + DefaultSendBootstrapInMsgCount = int32(10000) ) // AtomicityLevel represents the atomicity level of a changefeed. @@ -171,7 +176,7 @@ type SinkConfig struct { // Simple Protocol only config. // SendBootstrapIntervalInSec is the interval in seconds to send bootstrap message. - SendBootstrapIntervalInSec *uint `toml:"send-bootstrap-interval-in-sec" json:"send-bootstrap-interval-in-sec,omitempty"` + SendBootstrapIntervalInSec *int64 `toml:"send-bootstrap-interval-in-sec" json:"send-bootstrap-interval-in-sec,omitempty"` // SendBootstrapInMsgCount is the number of messages to send bootstrap message. SendBootstrapInMsgCount *int32 `toml:"send-bootstrap-in-msg-count" json:"send-bootstrap-in-msg-count,omitempty"` } diff --git a/pkg/filter/sql_event_filter.go b/pkg/filter/sql_event_filter.go index e5954836568..90194d8003c 100644 --- a/pkg/filter/sql_event_filter.go +++ b/pkg/filter/sql_event_filter.go @@ -175,8 +175,7 @@ func (f *sqlEventFilter) shouldSkipDDL( } rules := f.getRules(schema, table) - for i, rule := range rules { - log.Info("fizz sql event filter handle ddl event", zap.Any("idx", i), zap.Any("rule", rule)) + for _, rule := range rules { action, err := rule.bf.Filter( binlogFilterSchemaPlaceholder, binlogFilterTablePlaceholder, diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index ef82cfe52b2..e9fb5b6a421 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -26,15 +26,14 @@ import ( ) const ( - bootstrapWorkerTickerInterval = 10 * time.Second + bootstrapWorkerTickerInterval = 5 * time.Second bootstrapWorkerGCInterval = 30 * time.Second - defaultSendBootstrapInterval = 1 * time.Minute - defaultSendBootstrapInMsgCount = 1000 - defaultMaxInactiveDuration = 5 * time.Minute + defaultMaxInactiveDuration = 30 * time.Minute ) type bootstrapWorker struct { + changefeedID model.ChangeFeedID activeTables sync.Map builder RowEventEncoderBuilder sendBootstrapInterval time.Duration @@ -46,6 +45,7 @@ type bootstrapWorker struct { // newBootstrapWorker creates a new bootstrapGenerator instance func newBootstrapWorker( + changefeedID model.ChangeFeedID, outCh chan<- *future, builder RowEventEncoderBuilder, sendBootstrapInterval time.Duration, @@ -53,6 +53,7 @@ func newBootstrapWorker( maxInactiveDuration time.Duration, ) *bootstrapWorker { return &bootstrapWorker{ + changefeedID: changefeedID, outCh: outCh, builder: builder, activeTables: sync.Map{}, @@ -63,7 +64,6 @@ func newBootstrapWorker( } func (b *bootstrapWorker) run(ctx context.Context) error { - log.Info("fizz: bootstrap worker is started") sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) defer sendTicker.Stop() gcTicker := time.NewTicker(bootstrapWorkerGCInterval) @@ -106,9 +106,6 @@ func (b *bootstrapWorker) addEvent( if err != nil { return errors.Trace(err) } - log.Info("fizz: a new table is added to the bootstrap worker, send bootstrap message immediately", - zap.String("topic", tb.topic), - zap.Int64("tableID", int64(tb.id))) } else { // If the table is already in the activeTables, update its status. table.(*tableStatus).update(key, row) @@ -127,9 +124,8 @@ func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStat return nil } table.reset() - log.Info("fizz: bootstrap worker is sending bootstrap message", zap.Any("table", table.id)) tableInfo := table.tableInfo.Load().(*model.TableInfo) - events, err := b.generateEvents(ctx, table.topic, table.totalPartition.Load(), tableInfo) + events, err := b.generateEvents(table.topic, table.totalPartition.Load(), tableInfo) if err != nil { return errors.Trace(err) } @@ -138,17 +134,12 @@ func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStat case <-ctx.Done(): return ctx.Err() case b.outCh <- event: - log.Info("fizz: bootstrap message is sent", - zap.Any("msgKey", event.Key), - zap.Any("table", table.id)) } } - log.Info("fizz: bootstrap message are sent", zap.Any("table", table.id)) return nil } func (b *bootstrapWorker) generateEvents( - ctx context.Context, topic string, totalPartition int32, tableInfo *model.TableInfo, @@ -188,7 +179,10 @@ func (b *bootstrapWorker) gcInactiveTables() { b.activeTables.Range(func(key, value interface{}) bool { table := value.(*tableStatus) if !table.isActive(b.maxInactiveDuration) { - log.Info("fizz: a table is removed from the bootstrap worker", zap.Any("table", table.id)) + log.Info("A table is removed from the bootstrap worker", + zap.Int64("tableID", table.id), + zap.String("topic", table.topic), + zap.Stringer("changefeed", b.changefeedID)) b.activeTables.Delete(key) } return true diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index ff511da6f85..4e2024954e3 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -20,6 +20,7 @@ import ( timodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" ) @@ -50,6 +51,8 @@ func getMockTableStatus() (TopicPartitionKey, *model.RowChangedEvent, *tableStat func TestShouldSendBootstrapMsg(t *testing.T) { t.Parallel() + defaultSendBootstrapInterval := time.Duration(config.DefaultSendBootstrapIntervalInSec) * time.Second + defaultSendBootstrapInMsgCount := config.DefaultSendBootstrapInMsgCount _, _, tb1 := getMockTableStatus() @@ -63,7 +66,7 @@ func TestShouldSendBootstrapMsg(t *testing.T) { // case 3: When the table receive message more than sendBootstrapInMsgCount, // it should send bootstrap message - tb1.counter.Add(int32(defaultSendBootstrapInMsgCount)) + tb1.counter.Add(defaultSendBootstrapInMsgCount) require.True(t, tb1.shouldSendBootstrapMsg(defaultSendBootstrapInterval, defaultSendBootstrapInMsgCount)) // case 4: When the table does not send bootstrap message for a sendBootstrapInterval time, @@ -92,13 +95,15 @@ func TestIsActive(t *testing.T) { func TestBootstrapWorker(t *testing.T) { t.Parallel() // new builder + cfID := model.DefaultChangeFeedID("test") builder := &MockRowEventEncoderBuilder{} - outCh := make(chan *future, defaultInputChanSize) - worker := newBootstrapWorker(outCh, + worker := newBootstrapWorker( + cfID, + outCh, builder, - defaultSendBootstrapInterval, - defaultSendBootstrapInMsgCount, + time.Duration(config.DefaultSendBootstrapIntervalInSec)*time.Second, + config.DefaultSendBootstrapInMsgCount, defaultMaxInactiveDuration) // Start the worker in a separate goroutine diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index c4de0f4739e..19da3c7390d 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -80,17 +80,19 @@ func NewEncoderGroup( var bootstrapWorker *bootstrapWorker if *cfg.Protocol == config.ProtocolSimple.String() { - log.Info("fizz:bootstrap worker is enable for simple protocol, create it!") + log.Info("Sending bootstrap event is enable for simple protocol, creating bootstrap worker...", + zap.Stringer("changefeed", changefeedID)) sendBootstrapIntervalInSec := util.GetOrZero(cfg.SendBootstrapIntervalInSec) if sendBootstrapIntervalInSec <= 0 { - sendBootstrapIntervalInSec = uint(defaultSendBootstrapInterval.Seconds()) + sendBootstrapIntervalInSec = config.DefaultSendBootstrapIntervalInSec } msgCount := util.GetOrZero(cfg.SendBootstrapInMsgCount) if msgCount <= 0 { - msgCount = defaultSendBootstrapInMsgCount + msgCount = config.DefaultSendBootstrapInMsgCount } interval := time.Duration(sendBootstrapIntervalInSec) * time.Second bootstrapWorker = newBootstrapWorker( + changefeedID, outCh, builder, interval, diff --git a/pkg/sink/codec/simple/decoder.go b/pkg/sink/codec/simple/decoder.go index c5a9226fb7a..0e00903ec06 100644 --- a/pkg/sink/codec/simple/decoder.go +++ b/pkg/sink/codec/simple/decoder.go @@ -18,6 +18,7 @@ import ( "database/sql" "encoding/json" "path/filepath" + "time" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" @@ -291,18 +292,21 @@ func (d *decoder) NextDDLEvent() (*model.DDLEvent, error) { } // TableInfoProvider is used to store and read table info +// It works like a schema cache when consuming simple protocol messages +// It will store multiple versions of table info for a table +// The table info which has the exact (schema, table, version) will be returned when reading type TableInfoProvider interface { Write(info *model.TableInfo) Read(schema, table string, version uint64) *model.TableInfo } type memoryTableInfoProvider struct { - memo map[cacheKey]*model.TableInfo + memo map[tableSchemaKey]*model.TableInfo } func newMemoryTableInfoProvider() *memoryTableInfoProvider { return &memoryTableInfoProvider{ - memo: make(map[cacheKey]*model.TableInfo), + memo: make(map[tableSchemaKey]*model.TableInfo), } } @@ -310,41 +314,76 @@ func (m *memoryTableInfoProvider) Write(info *model.TableInfo) { if info == nil { return } - key := cacheKey{ - schema: info.TableName.Schema, - table: info.TableName.Table, + key := tableSchemaKey{ + schema: info.TableName.Schema, + table: info.TableName.Table, + version: info.UpdateTS, } - entry, ok := m.memo[key] - if ok && entry.UpdateTS >= info.UpdateTS { - log.Warn("table info not stored, since the updateTs is stale", + _, ok := m.memo[key] + if ok { + log.Warn("table info not stored, since it already exists", zap.String("schema", info.TableName.Schema), zap.String("table", info.TableName.Table), - zap.Uint64("oldUpdateTs", entry.UpdateTS), - zap.Uint64("updateTs", info.UpdateTS)) + zap.Uint64("version", info.UpdateTS)) return } + m.memo[key] = info log.Info("table info stored", zap.String("schema", info.TableName.Schema), zap.String("table", info.TableName.Table), - zap.Uint64("updateTs", info.UpdateTS)) + zap.Uint64("version", info.UpdateTS)) } +// Read returns the table info with the exact (schema, table, version) +// Note: It's a blocking call, it will wait until the table info is stored func (m *memoryTableInfoProvider) Read(schema, table string, version uint64) *model.TableInfo { - key := cacheKey{ - schema: schema, - table: table, + key := tableSchemaKey{ + schema: schema, + table: table, + version: version, } - entry, ok := m.memo[key] - if ok && entry.UpdateTS == version { - return entry + // Note(dongmen): Since the decoder is only use in unit test for now, + // we don't need to consider the performance + // Just use a ticker to check if the table info is stored every second. + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + for { + entry, ok := m.memo[key] + if ok { + log.Info("table info read", + zap.String("schema", schema), + zap.String("table", table), + zap.Uint64("version", version)) + return entry + } + select { + case <-ticker.C: + entry, ok := m.memo[key] + if ok { + log.Info("table info read", + zap.String("schema", schema), + zap.String("table", table), + zap.Uint64("version", version)) + return entry + } + case <-ctx.Done(): + log.Panic("table info read timeout", + zap.String("schema", schema), + zap.String("table", table), + zap.Uint64("version", version)) + return nil + } } - return nil } -type cacheKey struct { - schema string - table string +type tableSchemaKey struct { + schema string + table string + version uint64 } diff --git a/pkg/sink/codec/simple/encoder.go b/pkg/sink/codec/simple/encoder.go index 9414e1bfc2d..d4abbc6704a 100644 --- a/pkg/sink/codec/simple/encoder.go +++ b/pkg/sink/codec/simple/encoder.go @@ -148,7 +148,6 @@ func (e *encoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { // EncodeDDLEvent implement the DDLEventBatchEncoder interface func (e *encoder) EncodeDDLEvent(event *model.DDLEvent) (*common.Message, error) { m := newDDLMessage(event) - log.Info("fizz:encode ddl event", zap.Any("event", event)) value, err := json.Marshal(m) if err != nil { return nil, cerror.WrapError(cerror.ErrEncodeFailed, err) diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index cdcacb2e04e..c2a2b548c03 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -76,10 +76,12 @@ var customReplicaConfig = &ReplicaConfig{ Delimiter: config.Comma, NullString: config.NULL, }, - DateSeparator: "day", - EncoderConcurrency: util.AddressOf(32), - EnablePartitionSeparator: util.AddressOf(true), - ContentCompatible: util.AddressOf(true), + DateSeparator: "day", + EncoderConcurrency: util.AddressOf(32), + EnablePartitionSeparator: util.AddressOf(true), + ContentCompatible: util.AddressOf(true), + SendBootstrapIntervalInSec: util.AddressOf(int64(120)), + SendBootstrapInMsgCount: util.AddressOf(int32(10000)), }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, @@ -122,11 +124,13 @@ var defaultReplicaConfig = &ReplicaConfig{ Delimiter: config.Comma, NullString: config.NULL, }, - Terminator: "\r\n", - DateSeparator: "day", - EncoderConcurrency: util.AddressOf(32), - EnablePartitionSeparator: util.AddressOf(true), - ContentCompatible: util.AddressOf(false), + Terminator: "\r\n", + DateSeparator: "day", + EncoderConcurrency: util.AddressOf(32), + EnablePartitionSeparator: util.AddressOf(true), + ContentCompatible: util.AddressOf(false), + SendBootstrapIntervalInSec: util.AddressOf(int64(120)), + SendBootstrapInMsgCount: util.AddressOf(int32(10000)), }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, diff --git a/tests/integration_tests/api_v2/model.go b/tests/integration_tests/api_v2/model.go index f35e80227c0..1c4bc83555f 100644 --- a/tests/integration_tests/api_v2/model.go +++ b/tests/integration_tests/api_v2/model.go @@ -233,17 +233,19 @@ type Table struct { // SinkConfig represents sink config for a changefeed // This is a duplicate of config.SinkConfig type SinkConfig struct { - Protocol string `json:"protocol,omitempty"` - SchemaRegistry string `json:"schema_registry,omitempty"` - CSVConfig *CSVConfig `json:"csv,omitempty"` - DispatchRules []*DispatchRule `json:"dispatchers,omitempty"` - ColumnSelectors []*ColumnSelector `json:"column_selectors,omitempty"` - TxnAtomicity string `json:"transaction_atomicity"` - EncoderConcurrency *int `json:"encoder_concurrency,omitempty"` - Terminator string `json:"terminator"` - DateSeparator string `json:"date_separator,omitempty"` - EnablePartitionSeparator *bool `json:"enable_partition_separator,omitempty"` - ContentCompatible *bool `json:"content_compatible"` + Protocol string `json:"protocol,omitempty"` + SchemaRegistry string `json:"schema_registry,omitempty"` + CSVConfig *CSVConfig `json:"csv,omitempty"` + DispatchRules []*DispatchRule `json:"dispatchers,omitempty"` + ColumnSelectors []*ColumnSelector `json:"column_selectors,omitempty"` + TxnAtomicity string `json:"transaction_atomicity"` + EncoderConcurrency *int `json:"encoder_concurrency,omitempty"` + Terminator string `json:"terminator"` + DateSeparator string `json:"date_separator,omitempty"` + EnablePartitionSeparator *bool `json:"enable_partition_separator,omitempty"` + ContentCompatible *bool `json:"content_compatible"` + SendBootstrapIntervalInSec *int64 `json:"send_bootstrap_interval_in_sec,omitempty"` + SendBootstrapInMsgCount *int32 `json:"send_bootstrap_in_msg_count,omitempty"` } // CSVConfig denotes the csv config diff --git a/tests/integration_tests/kafka_simple_basic/conf/changefeed.toml b/tests/integration_tests/kafka_simple_basic/conf/changefeed.toml index 519968536bb..8d766ab7c00 100644 --- a/tests/integration_tests/kafka_simple_basic/conf/changefeed.toml +++ b/tests/integration_tests/kafka_simple_basic/conf/changefeed.toml @@ -1,3 +1,7 @@ [integrity] integrity-check-level = "correctness" corruption-handle-level = "error" + +[sink] +send-bootstrap-interval-in-sec = 5 +send-bootstrap-in-msg-count = 100 \ No newline at end of file diff --git a/tests/integration_tests/kafka_simple_basic/data/data.sql b/tests/integration_tests/kafka_simple_basic/data/data.sql index 98154560a93..db1ba1ca616 100644 --- a/tests/integration_tests/kafka_simple_basic/data/data.sql +++ b/tests/integration_tests/kafka_simple_basic/data/data.sql @@ -2,17 +2,7 @@ drop database if exists test; create database test; use test; -create table tp_int -( - id int auto_increment, - c_tinyint tinyint null, - c_smallint smallint null, - c_mediumint mediumint null, - c_int int null, - c_bigint bigint null, - constraint pk - primary key (id) -); + insert into tp_int() values (); @@ -31,17 +21,7 @@ values (-128, -32768, -8388608, -2147483648, -9223372036854775808); update tp_int set c_int = 0, c_tinyint = 0 where c_smallint = 2; delete from tp_int where c_int = 0; --- unsigned int -create table tp_unsigned_int ( - id int auto_increment, - c_unsigned_tinyint tinyint unsigned null, - c_unsigned_smallint smallint unsigned null, - c_unsigned_mediumint mediumint unsigned null, - c_unsigned_int int unsigned null, - c_unsigned_bigint bigint unsigned null, - constraint pk - primary key (id) -); + insert into tp_unsigned_int() values (); @@ -67,17 +47,7 @@ values (128, 32768, 8388608, 2147483648, 9223372036854775808); update tp_unsigned_int set c_unsigned_int = 0, c_unsigned_tinyint = 0 where c_unsigned_smallint = 65535; delete from tp_unsigned_int where c_unsigned_int = 0; --- real -create table tp_real -( - id int auto_increment, - c_float float null, - c_double double null, - c_decimal decimal null, - c_decimal_2 decimal(10, 4) null, - constraint pk - primary key (id) -); + insert into tp_real() values (); @@ -90,16 +60,6 @@ values (-2.7182818284, -3.1415926, -8000, -179394.233); update tp_real set c_double = 2.333 where c_double = 2020.0303; --- unsigned real -create table tp_unsigned_real ( - id int auto_increment, - c_unsigned_float float unsigned null, - c_unsigned_double double unsigned null, - c_unsigned_decimal decimal unsigned null, - c_unsigned_decimal_2 decimal(10, 4) unsigned null, - constraint pk - primary key (id) -); insert into tp_unsigned_real() values (); diff --git a/tests/integration_tests/kafka_simple_basic/run.sh b/tests/integration_tests/kafka_simple_basic/run.sh index c2b419ce19c..26a840bf4aa 100644 --- a/tests/integration_tests/kafka_simple_basic/run.sh +++ b/tests/integration_tests/kafka_simple_basic/run.sh @@ -33,7 +33,7 @@ function run() { SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple" fi - run_cdc_cli changefeed create --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" + run_cdc_cli changefeed create --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" -c "simple-basic" sleep 5 # wait for changefeed to start # determine the sink uri and run corresponding consumer # currently only kafka and pulsar are supported @@ -41,6 +41,15 @@ function run() { run_kafka_consumer $WORK_DIR $SINK_URI fi + # pre execute some ddls + run_sql_file $CUR/data/pre_ddl.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists test.finish_mark_for_ddl ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + + # pause and resume changefeed makse sure changefeed sneding bootstrap events + # when it is resumed, so the data after pause can be decoded correctly + run_cdc_cli changefeed pause -c "simple-basic" + run_cdc_cli changefeed resume -c "simple-basic" + run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first From 8c9b7743001f44e0259b8606c852c8a2c2e75c25 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 12:14:11 +0800 Subject: [PATCH 06/20] add configuration and integration test case --- .../kafka_simple_basic/data/pre_ddl.sql | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 tests/integration_tests/kafka_simple_basic/data/pre_ddl.sql diff --git a/tests/integration_tests/kafka_simple_basic/data/pre_ddl.sql b/tests/integration_tests/kafka_simple_basic/data/pre_ddl.sql new file mode 100644 index 00000000000..f76108c33f2 --- /dev/null +++ b/tests/integration_tests/kafka_simple_basic/data/pre_ddl.sql @@ -0,0 +1,61 @@ +drop database if exists test; +create database test; +use test; + +create table tp_int +( + id int auto_increment, + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + constraint pk + primary key (id) +); + + +-- unsigned int +create table tp_unsigned_int ( + id int auto_increment, + c_unsigned_tinyint tinyint unsigned null, + c_unsigned_smallint smallint unsigned null, + c_unsigned_mediumint mediumint unsigned null, + c_unsigned_int int unsigned null, + c_unsigned_bigint bigint unsigned null, + constraint pk + primary key (id) +); + + +-- real +create table tp_real +( + id int auto_increment, + c_float float null, + c_double double null, + c_decimal decimal null, + c_decimal_2 decimal(10, 4) null, + constraint pk + primary key (id) +); + + + +-- unsigned real +create table tp_unsigned_real ( + id int auto_increment, + c_unsigned_float float unsigned null, + c_unsigned_double double unsigned null, + c_unsigned_decimal decimal unsigned null, + c_unsigned_decimal_2 decimal(10, 4) unsigned null, + constraint pk + primary key (id) +); + +create table finish_mark_for_ddl +( + id int PRIMARY KEY +); + + From c21354a29272a510a98a2857b7b9d03a2fe2cc96 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 13:53:40 +0800 Subject: [PATCH 07/20] fix failed test cases --- pkg/orchestrator/reactor_state_test.go | 6 ++++++ tests/integration_tests/kafka_simple_basic/data/data.sql | 4 ---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index 97eb8e8aaaa..7b48ca5cc32 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -127,6 +127,8 @@ func TestChangefeedStateUpdate(t *testing.T) { OnlyOutputUpdatedColumns: config.GetDefaultReplicaConfig().Sink.OnlyOutputUpdatedColumns, DeleteOnlyOutputHandleKeyColumns: config.GetDefaultReplicaConfig().Sink.DeleteOnlyOutputHandleKeyColumns, ContentCompatible: config.GetDefaultReplicaConfig().Sink.ContentCompatible, + SendBootstrapIntervalInSec: config.GetDefaultReplicaConfig().Sink.SendBootstrapIntervalInSec, + SendBootstrapInMsgCount: config.GetDefaultReplicaConfig().Sink.SendBootstrapInMsgCount, }, Consistent: config.GetDefaultReplicaConfig().Consistent, Integrity: config.GetDefaultReplicaConfig().Integrity, @@ -191,6 +193,8 @@ func TestChangefeedStateUpdate(t *testing.T) { OnlyOutputUpdatedColumns: config.GetDefaultReplicaConfig().Sink.OnlyOutputUpdatedColumns, DeleteOnlyOutputHandleKeyColumns: config.GetDefaultReplicaConfig().Sink.DeleteOnlyOutputHandleKeyColumns, ContentCompatible: config.GetDefaultReplicaConfig().Sink.ContentCompatible, + SendBootstrapIntervalInSec: config.GetDefaultReplicaConfig().Sink.SendBootstrapIntervalInSec, + SendBootstrapInMsgCount: config.GetDefaultReplicaConfig().Sink.SendBootstrapInMsgCount, }, Scheduler: config.GetDefaultReplicaConfig().Scheduler, Integrity: config.GetDefaultReplicaConfig().Integrity, @@ -261,6 +265,8 @@ func TestChangefeedStateUpdate(t *testing.T) { OnlyOutputUpdatedColumns: config.GetDefaultReplicaConfig().Sink.OnlyOutputUpdatedColumns, DeleteOnlyOutputHandleKeyColumns: config.GetDefaultReplicaConfig().Sink.DeleteOnlyOutputHandleKeyColumns, ContentCompatible: config.GetDefaultReplicaConfig().Sink.ContentCompatible, + SendBootstrapIntervalInSec: config.GetDefaultReplicaConfig().Sink.SendBootstrapIntervalInSec, + SendBootstrapInMsgCount: config.GetDefaultReplicaConfig().Sink.SendBootstrapInMsgCount, }, Consistent: config.GetDefaultReplicaConfig().Consistent, Scheduler: config.GetDefaultReplicaConfig().Scheduler, diff --git a/tests/integration_tests/kafka_simple_basic/data/data.sql b/tests/integration_tests/kafka_simple_basic/data/data.sql index db1ba1ca616..0a2d9b9b20d 100644 --- a/tests/integration_tests/kafka_simple_basic/data/data.sql +++ b/tests/integration_tests/kafka_simple_basic/data/data.sql @@ -1,9 +1,5 @@ -drop database if exists test; -create database test; use test; - - insert into tp_int() values (); From 8770b8a25229511072a49cf7ef1ae1430353dd35 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 14:28:37 +0800 Subject: [PATCH 08/20] fix failed test cases 2 --- pkg/sink/codec/encoder_group.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index 19da3c7390d..e10e33c684b 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -79,7 +79,8 @@ func NewEncoderGroup( outCh := make(chan *future, defaultInputChanSize*concurrency) var bootstrapWorker *bootstrapWorker - if *cfg.Protocol == config.ProtocolSimple.String() { + protocol := util.GetOrZero(cfg.Protocol) + if protocol == config.ProtocolSimple.String() { log.Info("Sending bootstrap event is enable for simple protocol, creating bootstrap worker...", zap.Stringer("changefeed", changefeedID)) sendBootstrapIntervalInSec := util.GetOrZero(cfg.SendBootstrapIntervalInSec) From f3312e7017e9f6bf23fb6a0df36e4d4850437d2d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 16:11:19 +0800 Subject: [PATCH 09/20] fix failed integration test case --- .../data/data.sql | 4 +- .../kafka_simple_handle_key_only/data/ddl.sql | 98 +++++++++---------- .../kafka_simple_handle_key_only/run.sh | 2 +- 3 files changed, 52 insertions(+), 52 deletions(-) diff --git a/tests/integration_tests/kafka_simple_handle_key_only/data/data.sql b/tests/integration_tests/kafka_simple_handle_key_only/data/data.sql index 88cb3d0c997..2730d96d710 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/data/data.sql +++ b/tests/integration_tests/kafka_simple_handle_key_only/data/data.sql @@ -6,7 +6,7 @@ insert into t values ( 2020.0202, 2020.0303, 2020.0404, 2021.1208, 3.1415, 2.7182, 8000, 179394.233, '2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020', - '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', 'b', 'b,c', b'1000001', '{ @@ -27,7 +27,7 @@ insert into t values ( 2020.0202, 2020.0303, 2020.0404, 2021.1208, 3.1415, 2.7182, 8000, 179394.233, '2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020', - '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', 'b', 'b,c', b'1000001', '{ diff --git a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql index cd062fe8ce4..fd7b20d6a0c 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql +++ b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql @@ -3,53 +3,53 @@ create database test; use test; create table t ( - id int primary key auto_increment, - - c_tinyint tinyint null, - c_smallint smallint null, - c_mediumint mediumint null, - c_int int null, - c_bigint bigint null, - - c_unsigned_tinyint tinyint unsigned null, - c_unsigned_smallint smallint unsigned null, - c_unsigned_mediumint mediumint unsigned null, - c_unsigned_int int unsigned null, - c_unsigned_bigint bigint unsigned null, - - c_float float null, - c_double double null, - c_decimal decimal null, - c_decimal_2 decimal(10, 4) null, - - c_unsigned_float float unsigned null, - c_unsigned_double double unsigned null, - c_unsigned_decimal decimal unsigned null, - c_unsigned_decimal_2 decimal(10, 4) unsigned null, - - c_date date null, - c_datetime datetime null, - c_timestamp timestamp null, - c_time time null, - c_year year null, - - c_tinytext tinytext null, - c_text text null, - c_mediumtext mediumtext null, - c_longtext longtext null, - - c_tinyblob tinyblob null, - c_blob blob null, - c_mediumblob mediumblob null, - c_longblob longblob null, - - c_char char(16) null, - c_varchar varchar(16) null, - c_binary binary(16) null, - c_varbinary varbinary(16) null, - - c_enum enum ('a','b','c') null, - c_set set ('a','b','c') null, - c_bit bit(64) null, - c_json json null + id0 int primary key auto_increment, + + c_tinyint1 tinyint null, + c_smallint2 smallint null, + c_mediumint3 mediumint null, + c_int4 int null, + c_bigint5 bigint null, + + c_unsigned_tinyint6 tinyint unsigned null, + c_unsigned_smallint7 smallint unsigned null, + c_unsigned_mediumint8 mediumint unsigned null, + c_unsigned_int9 int unsigned null, + c_unsigned_bigint10 bigint unsigned null, + + c_float11 float null, + c_double12 double null, + c_decimal13 decimal null, + c_decimal_214 decimal(10, 4) null, + + c_unsigned_float15 float unsigned null, + c_unsigned_double16 double unsigned null, + c_unsigned_decimal17 decimal unsigned null, + c_unsigned_decimal_218 decimal(10, 4) unsigned null, + + c_date19 date null, + c_datetime20 datetime null, + c_timestamp21 timestamp null, + c_time22 time null, + c_year23 year null, + + c_tinytext24 tinytext null, + c_text25 text null, + c_mediumtext26 mediumtext null, + c_longtext27 longtext null, + + c_tinyblob28 tinyblob null, + c_blob29 blob null, + c_mediumblob30 mediumblob null, + c_longblob31 longblob null, + + c_char32 char(16) null, + c_varchar33 varchar(16) null, + c_binary34 binary(16) null, + c_varbinary35 varbinary(16) null, + + c_enum36 enum ('a','b','c') null, + c_set37 set ('a','b','c') null, + c_bit38 bit(64) null, + c_json39 json null ); diff --git a/tests/integration_tests/kafka_simple_handle_key_only/run.sh b/tests/integration_tests/kafka_simple_handle_key_only/run.sh index dbad2e0701c..78503cf04a8 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/run.sh +++ b/tests/integration_tests/kafka_simple_handle_key_only/run.sh @@ -35,7 +35,7 @@ function run() { run_cdc_cli changefeed pause -c ${changefeed_id} - SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple&max-message-bytes=500" + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple&max-message-bytes=2048" run_cdc_cli changefeed update -c ${changefeed_id} --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" --no-confirm run_cdc_cli changefeed resume -c ${changefeed_id} cdc_kafka_consumer --upstream-uri $SINK_URI --downstream-uri="mysql://root@127.0.0.1:3306/?safe-mode=true&batch-dml-enable=false" --upstream-tidb-dsn="root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/?" --config="$CUR/conf/changefeed.toml" 2>&1 & From 5966125e5c6dd56a386d61c2be5503bc8effc79c Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 17:01:49 +0800 Subject: [PATCH 10/20] fix failed integration test case 2 --- .../kafka_simple_handle_key_only/data/ddl.sql | 104 +++++++++--------- 1 file changed, 50 insertions(+), 54 deletions(-) diff --git a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql index fd7b20d6a0c..7a7c9ebc1db 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql +++ b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql @@ -1,55 +1,51 @@ -drop database if exists test; -create database test; -use test; - create table t ( - id0 int primary key auto_increment, - - c_tinyint1 tinyint null, - c_smallint2 smallint null, - c_mediumint3 mediumint null, - c_int4 int null, - c_bigint5 bigint null, - - c_unsigned_tinyint6 tinyint unsigned null, - c_unsigned_smallint7 smallint unsigned null, - c_unsigned_mediumint8 mediumint unsigned null, - c_unsigned_int9 int unsigned null, - c_unsigned_bigint10 bigint unsigned null, - - c_float11 float null, - c_double12 double null, - c_decimal13 decimal null, - c_decimal_214 decimal(10, 4) null, - - c_unsigned_float15 float unsigned null, - c_unsigned_double16 double unsigned null, - c_unsigned_decimal17 decimal unsigned null, - c_unsigned_decimal_218 decimal(10, 4) unsigned null, - - c_date19 date null, - c_datetime20 datetime null, - c_timestamp21 timestamp null, - c_time22 time null, - c_year23 year null, - - c_tinytext24 tinytext null, - c_text25 text null, - c_mediumtext26 mediumtext null, - c_longtext27 longtext null, - - c_tinyblob28 tinyblob null, - c_blob29 blob null, - c_mediumblob30 mediumblob null, - c_longblob31 longblob null, - - c_char32 char(16) null, - c_varchar33 varchar(16) null, - c_binary34 binary(16) null, - c_varbinary35 varbinary(16) null, - - c_enum36 enum ('a','b','c') null, - c_set37 set ('a','b','c') null, - c_bit38 bit(64) null, - c_json39 json null -); + id int primary key auto_increment, + + c_tinyint tinyint null, + c_smallint smallint null, + c_mediumint mediumint null, + c_int int null, + c_bigint bigint null, + + c_unsigned_tinyint tinyint unsigned null, + c_unsigned_smallint smallint unsigned null, + c_unsigned_mediumint mediumint unsigned null, + c_unsigned_int int unsigned null, + c_unsigned_bigint bigint unsigned null, + + c_float float null, + c_double double null, + c_decimal decimal null, + c_decimal_2 decimal(10, 4) null, + + c_unsigned_float float unsigned null, + c_unsigned_double double unsigned null, + c_unsigned_decimal decimal unsigned null, + c_unsigned_decimal_2 decimal(10, 4) unsigned null, + + c_date date null, + c_datetime datetime null, + c_timestamp timestamp null, + c_time time null, + c_year year null, + + c_tinytext tinytext null, + c_text text null, + c_mediumtext mediumtext null, + c_longtext longtext null, + + c_tinyblob tinyblob null, + c_blob blob null, + c_mediumblob mediumblob null, + c_longblob longblob null, + + c_char char(16) null, + c_varchar varchar(16) null, + c_binary binary(16) null, + c_varbinary varbinary(16) null, + + c_enum enum ('a','b','c') null, + c_set set ('a','b','c') null, + c_bit bit(64) null, + c_json json null +); \ No newline at end of file From 90925730d1d78926b893fa8bc35caeed84391b95 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 17:02:17 +0800 Subject: [PATCH 11/20] fix failed integration test case 3 --- .../kafka_simple_handle_key_only/data/ddl.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql index 7a7c9ebc1db..7e8cafe3cc2 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql +++ b/tests/integration_tests/kafka_simple_handle_key_only/data/ddl.sql @@ -1,3 +1,7 @@ +drop database if exists test; +create database test; +use test; + create table t ( id int primary key auto_increment, From a477728257757558026b5af320388216b3ee224f Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 28 Dec 2023 17:03:36 +0800 Subject: [PATCH 12/20] fix failed integration test case 4 --- .../integration_tests/kafka_simple_claim_check/data/data.sql | 4 ++-- tests/integration_tests/kafka_simple_claim_check/run.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration_tests/kafka_simple_claim_check/data/data.sql b/tests/integration_tests/kafka_simple_claim_check/data/data.sql index 88cb3d0c997..2730d96d710 100644 --- a/tests/integration_tests/kafka_simple_claim_check/data/data.sql +++ b/tests/integration_tests/kafka_simple_claim_check/data/data.sql @@ -6,7 +6,7 @@ insert into t values ( 2020.0202, 2020.0303, 2020.0404, 2021.1208, 3.1415, 2.7182, 8000, 179394.233, '2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020', - '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', 'b', 'b,c', b'1000001', '{ @@ -27,7 +27,7 @@ insert into t values ( 2020.0202, 2020.0303, 2020.0404, 2021.1208, 3.1415, 2.7182, 8000, 179394.233, '2020-02-20', '2020-02-20 02:20:20', '2020-02-20 02:20:20', '02:20:20', '2020', - '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', + '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', '89504E470D0A1A0A', '89504E470D0A1A0A', x'89504E470D0A1A0A', x'89504E470D0A1A0A', 'b', 'b,c', b'1000001', '{ diff --git a/tests/integration_tests/kafka_simple_claim_check/run.sh b/tests/integration_tests/kafka_simple_claim_check/run.sh index b0a13351981..3aac34a0786 100644 --- a/tests/integration_tests/kafka_simple_claim_check/run.sh +++ b/tests/integration_tests/kafka_simple_claim_check/run.sh @@ -35,7 +35,7 @@ function run() { run_cdc_cli changefeed pause -c ${changefeed_id} - SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple&max-message-bytes=500" + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple&max-message-bytes=2048" run_cdc_cli changefeed update -c ${changefeed_id} --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" --no-confirm run_cdc_cli changefeed resume -c ${changefeed_id} From 9fae03187ee86e3c25aef05c1bf26b74e99a20e1 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 2 Jan 2024 10:02:53 +0800 Subject: [PATCH 13/20] Resolve comments --- cdc/model/sink.go | 8 ++++ cdc/sink/dmlsink/mq/mq_dml_sink.go | 2 +- cdc/sink/dmlsink/mq/worker.go | 7 ++-- cdc/sink/dmlsink/mq/worker_test.go | 24 ++++++------ pkg/config/config_test_data.go | 6 +-- pkg/config/sink.go | 22 +++++++++-- pkg/sink/codec/bootstraper.go | 63 +++++++++++++++++------------- pkg/sink/codec/bootstraper_test.go | 16 ++++---- pkg/sink/codec/encoder.go | 8 ---- pkg/sink/codec/encoder_group.go | 29 ++++---------- 10 files changed, 96 insertions(+), 89 deletions(-) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index bad634448ef..9eade84bccd 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -912,3 +912,11 @@ func (t *SingleTableTxn) Append(row *RowChangedEvent) { func (t *SingleTableTxn) ToWaitFlush() bool { return t.FinishWg != nil } + +// TopicPartitionKey contains the topic and partition key of the message. +type TopicPartitionKey struct { + Topic string + Partition int32 + PartitionKey string + TotalPartition int32 +} diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink.go b/cdc/sink/dmlsink/mq/mq_dml_sink.go index aaf7a5611ef..e6d2105b69f 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink.go @@ -195,7 +195,7 @@ func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa // We already limit the memory usage by MemoryQuota at SinkManager level. // So it is safe to send the event to a unbounded channel here. s.alive.worker.msgChan.In() <- mqEvent{ - key: codec.TopicPartitionKey{ + key: model.TopicPartitionKey{ Topic: topic, Partition: index, PartitionKey: key, diff --git a/cdc/sink/dmlsink/mq/worker.go b/cdc/sink/dmlsink/mq/worker.go index 0df7b1b19f5..08e32ad3cce 100644 --- a/cdc/sink/dmlsink/mq/worker.go +++ b/cdc/sink/dmlsink/mq/worker.go @@ -44,7 +44,7 @@ const ( // mqEvent is the event of the mq worker. // It carries the topic and partition information of the message. type mqEvent struct { - key codec.TopicPartitionKey + key model.TopicPartitionKey rowEvent *dmlsink.RowChangeCallbackableEvent } @@ -252,8 +252,8 @@ func (w *worker) batch( // group groups messages by its key. func (w *worker) group( msgs []mqEvent, -) map[codec.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent { - groupedMsgs := make(map[codec.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent) +) map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent { + groupedMsgs := make(map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent) for _, msg := range msgs { // Skip this event when the table is stopping. if msg.rowEvent.GetTableSinkState() != state.TableSinkSinking { @@ -297,7 +297,6 @@ func (w *worker) sendMessages(ctx context.Context) error { if err = future.Ready(ctx); err != nil { return errors.Trace(err) } - for _, message := range future.Messages { start := time.Now() if err = w.statistics.RecordBatchExecution(func() (int, int64, error) { diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index c1e9a6e98ab..ddee84cd6a7 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -79,7 +79,7 @@ func TestNonBatchEncode_SendMessages(t *testing.T) { worker, p := newNonBatchEncodeWorker(ctx, t) defer worker.close() - key := codec.TopicPartitionKey{ + key := model.TopicPartitionKey{ Topic: "test", Partition: 1, } @@ -138,7 +138,7 @@ func TestBatchEncode_Batch(t *testing.T) { defer cancel() worker, _ := newBatchEncodeWorker(ctx, t) defer worker.close() - key := codec.TopicPartitionKey{ + key := model.TopicPartitionKey{ Topic: "test", Partition: 1, } @@ -170,15 +170,15 @@ func TestBatchEncode_Batch(t *testing.T) { func TestBatchEncode_Group(t *testing.T) { t.Parallel() - key1 := codec.TopicPartitionKey{ + key1 := model.TopicPartitionKey{ Topic: "test", Partition: 1, } - key2 := codec.TopicPartitionKey{ + key2 := model.TopicPartitionKey{ Topic: "test", Partition: 2, } - key3 := codec.TopicPartitionKey{ + key3 := model.TopicPartitionKey{ Topic: "test1", Partition: 2, } @@ -274,11 +274,11 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { tableInfo := model.WrapTableInfo(0, "test", 1, job.BinlogInfo.TableInfo) _, _, colInfo := tableInfo.GetRowColInfos() - key1 := codec.TopicPartitionKey{ + key1 := model.TopicPartitionKey{ Topic: "test", Partition: 1, } - key2 := codec.TopicPartitionKey{ + key2 := model.TopicPartitionKey{ Topic: "test", Partition: 2, } @@ -341,15 +341,15 @@ func TestBatchEncode_GroupWhenTableStopping(t *testing.T) { } func TestBatchEncode_SendMessages(t *testing.T) { - key1 := codec.TopicPartitionKey{ + key1 := model.TopicPartitionKey{ Topic: "test", Partition: 1, } - key2 := codec.TopicPartitionKey{ + key2 := model.TopicPartitionKey{ Topic: "test", Partition: 2, } - key3 := codec.TopicPartitionKey{ + key3 := model.TopicPartitionKey{ Topic: "test1", Partition: 2, } @@ -504,11 +504,11 @@ func TestBatchEncodeWorker_Abort(t *testing.T) { } func TestNonBatchEncode_SendMessagesWhenTableStopping(t *testing.T) { - key1 := codec.TopicPartitionKey{ + key1 := model.TopicPartitionKey{ Topic: "test", Partition: 1, } - key2 := codec.TopicPartitionKey{ + key2 := model.TopicPartitionKey{ Topic: "test", Partition: 2, } diff --git a/pkg/config/config_test_data.go b/pkg/config/config_test_data.go index 6c67f8abf9f..dbff958308c 100644 --- a/pkg/config/config_test_data.go +++ b/pkg/config/config_test_data.go @@ -67,7 +67,7 @@ const ( }, "advance-timeout-in-sec": 150, "send-bootstrap-interval-in-sec": 120, - "send-bootstrap-in-msg-count": 10000 + "send-bootstrap-in-msg-count": 10000, "debezium-disable-schema": false }, "consistent": { @@ -323,7 +323,7 @@ const ( }, "advance-timeout-in-sec": 150, "send-bootstrap-interval-in-sec": 120, - "send-bootstrap-in-msg-count": 10000 + "send-bootstrap-in-msg-count": 10000, "debezium-disable-schema": false }, "consistent": { @@ -487,7 +487,7 @@ const ( }, "advance-timeout-in-sec": 150, "send-bootstrap-interval-in-sec": 120, - "send-bootstrap-in-msg-count": 10000 + "send-bootstrap-in-msg-count": 10000, "debezium-disable-schema": false }, "consistent": { diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 5871e71267a..c9809bbe320 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -171,14 +171,16 @@ type SinkConfig struct { // advanced for this given duration, the sink will be canceled and re-established. AdvanceTimeoutInSec *uint `toml:"advance-timeout-in-sec" json:"advance-timeout-in-sec,omitempty"` - // Debezium only. Whether schema should be excluded in the output. - DebeziumDisableSchema *bool `toml:"debezium-disable-schema" json:"debezium-disable-schema,omitempty"` - - // Simple Protocol only config. + // Simple Protocol only config, use to control the behavior of sending bootstrap message. + // Note: When one of the following conditions is set to negative value, + // bootstrap sending function will be disabled. // SendBootstrapIntervalInSec is the interval in seconds to send bootstrap message. SendBootstrapIntervalInSec *int64 `toml:"send-bootstrap-interval-in-sec" json:"send-bootstrap-interval-in-sec,omitempty"` // SendBootstrapInMsgCount is the number of messages to send bootstrap message. SendBootstrapInMsgCount *int32 `toml:"send-bootstrap-in-msg-count" json:"send-bootstrap-in-msg-count,omitempty"` + + // Debezium only. Whether schema should be excluded in the output. + DebeziumDisableSchema *bool `toml:"debezium-disable-schema" json:"debezium-disable-schema,omitempty"` } // MaskSensitiveData masks sensitive data in SinkConfig @@ -194,6 +196,18 @@ func (s *SinkConfig) MaskSensitiveData() { } } +func (s *SinkConfig) ShouldSendBootstrapMsg() bool { + if s == nil { + return false + } + protocol := util.GetOrZero(s.Protocol) + // Only enable bootstrap sending function for simple protocol + // and when both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0 + return protocol == ProtocolSimple.String() && + util.GetOrZero(s.SendBootstrapIntervalInSec) > 0 && + util.GetOrZero(s.SendBootstrapInMsgCount) > 0 +} + // CSVConfig defines a series of configuration items for csv codec. type CSVConfig struct { // delimiter between fields, it can be 1 character or at most 2 characters diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index e9fb5b6a421..a6e3d281566 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Inc. +// Copyright 2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -35,7 +35,7 @@ const ( type bootstrapWorker struct { changefeedID model.ChangeFeedID activeTables sync.Map - builder RowEventEncoderBuilder + encoder RowEventEncoder sendBootstrapInterval time.Duration sendBootstrapInMsgCount int32 // maxInactiveDuration is the max duration that a table can be inactive @@ -43,21 +43,27 @@ type bootstrapWorker struct { outCh chan<- *future } -// newBootstrapWorker creates a new bootstrapGenerator instance +// newBootstrapWorker creates a new bootstrapWorker instance func newBootstrapWorker( changefeedID model.ChangeFeedID, outCh chan<- *future, - builder RowEventEncoderBuilder, - sendBootstrapInterval time.Duration, + encoder RowEventEncoder, + sendBootstrapInterval int64, sendBootstrapInMsgCount int32, maxInactiveDuration time.Duration, ) *bootstrapWorker { + log.Info("Sending bootstrap event is enable for simple protocol,"+ + "and both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0"+ + "enable bootstrap sending function", + zap.Stringer("changefeed", changefeedID), + zap.Int64("sendBootstrapIntervalInSec", sendBootstrapInterval), + zap.Int32("sendBootstrapInMsgCount", sendBootstrapInMsgCount)) return &bootstrapWorker{ changefeedID: changefeedID, outCh: outCh, - builder: builder, + encoder: encoder, activeTables: sync.Map{}, - sendBootstrapInterval: sendBootstrapInterval, + sendBootstrapInterval: time.Duration(sendBootstrapInterval) * time.Second, sendBootstrapInMsgCount: sendBootstrapInMsgCount, maxInactiveDuration: maxInactiveDuration, } @@ -65,18 +71,20 @@ func newBootstrapWorker( func (b *bootstrapWorker) run(ctx context.Context) error { sendTicker := time.NewTicker(bootstrapWorkerTickerInterval) - defer sendTicker.Stop() gcTicker := time.NewTicker(bootstrapWorkerGCInterval) - defer gcTicker.Stop() - errCh := make(chan error, 1) + defer func() { + gcTicker.Stop() + sendTicker.Stop() + }() + errCh := make(chan error, 1) for { select { case <-ctx.Done(): return ctx.Err() case <-sendTicker.C: b.activeTables.Range(func(key, value interface{}) bool { - table := value.(*tableStatus) + table := value.(*tableStatistic) err := b.sendBootstrapMsg(ctx, table) if err != nil { errCh <- err @@ -94,7 +102,7 @@ func (b *bootstrapWorker) run(ctx context.Context) error { func (b *bootstrapWorker) addEvent( ctx context.Context, - key TopicPartitionKey, + key model.TopicPartitionKey, row *model.RowChangedEvent, ) error { table, ok := b.activeTables.Load(row.Table.TableID) @@ -108,7 +116,7 @@ func (b *bootstrapWorker) addEvent( } } else { // If the table is already in the activeTables, update its status. - table.(*tableStatus).update(key, row) + table.(*tableStatistic).update(key, row) } return nil } @@ -117,7 +125,7 @@ func (b *bootstrapWorker) addEvent( // 1. The time since last bootstrap message sent is larger than sendBootstrapInterval // 2. The received row event count since last bootstrap message sent is larger than sendBootstrapInMsgCount // Note: It is a blocking method, it will block if the outCh is full. -func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatus) error { +func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStatistic) error { if !table.shouldSendBootstrapMsg( b.sendBootstrapInterval, b.sendBootstrapInMsgCount) { @@ -147,8 +155,7 @@ func (b *bootstrapWorker) generateEvents( res := make([]*future, 0, int(totalPartition)) // Bootstrap messages of a table should be sent to all partitions. for i := 0; i < int(totalPartition); i++ { - encoder := b.builder.Build() - msg, err := encoder.EncodeDDLEvent(&model.DDLEvent{ + msg, err := b.encoder.EncodeDDLEvent(&model.DDLEvent{ StartTs: 0, CommitTs: 0, TableInfo: tableInfo, @@ -158,7 +165,7 @@ func (b *bootstrapWorker) generateEvents( return nil, errors.Trace(err) } - key := TopicPartitionKey{ + key := model.TopicPartitionKey{ Topic: topic, Partition: int32(i), TotalPartition: totalPartition, @@ -177,8 +184,8 @@ func (b *bootstrapWorker) generateEvents( func (b *bootstrapWorker) gcInactiveTables() { b.activeTables.Range(func(key, value interface{}) bool { - table := value.(*tableStatus) - if !table.isActive(b.maxInactiveDuration) { + table := value.(*tableStatistic) + if table.isInactive(b.maxInactiveDuration) { log.Info("A table is removed from the bootstrap worker", zap.Int64("tableID", table.id), zap.String("topic", table.topic), @@ -189,8 +196,8 @@ func (b *bootstrapWorker) gcInactiveTables() { }) } -// tableStatus is used to record the status of a table -type tableStatus struct { +// tableStatistic is used to record the statistics of a table +type tableStatistic struct { // id is the table's ID, it will not change id int64 // topic is the table's topic, it will not change @@ -215,8 +222,8 @@ type tableStatus struct { tableInfo atomic.Value } -func newTableStatus(key TopicPartitionKey, row *model.RowChangedEvent) *tableStatus { - res := &tableStatus{ +func newTableStatus(key model.TopicPartitionKey, row *model.RowChangedEvent) *tableStatistic { + res := &tableStatistic{ id: row.Table.TableID, topic: key.Topic, } @@ -229,7 +236,7 @@ func newTableStatus(key TopicPartitionKey, row *model.RowChangedEvent) *tableSta return res } -func (t *tableStatus) shouldSendBootstrapMsg( +func (t *tableStatistic) shouldSendBootstrapMsg( sendBootstrapInterval time.Duration, sendBootstrapMsgCountInterval int32, ) bool { @@ -238,7 +245,7 @@ func (t *tableStatus) shouldSendBootstrapMsg( t.counter.Load() >= sendBootstrapMsgCountInterval } -func (t *tableStatus) update(key TopicPartitionKey, row *model.RowChangedEvent) { +func (t *tableStatistic) update(key model.TopicPartitionKey, row *model.RowChangedEvent) { t.counter.Add(1) t.lastMsgReceivedTime.Store(time.Now()) @@ -252,12 +259,12 @@ func (t *tableStatus) update(key TopicPartitionKey, row *model.RowChangedEvent) } } -func (t *tableStatus) isActive(maxInactiveDuration time.Duration) bool { +func (t *tableStatistic) isInactive(maxInactiveDuration time.Duration) bool { lastMsgReceivedTime := t.lastMsgReceivedTime.Load().(time.Time) - return time.Since(lastMsgReceivedTime) < maxInactiveDuration + return time.Since(lastMsgReceivedTime) > maxInactiveDuration } -func (t *tableStatus) reset() { +func (t *tableStatistic) reset() { t.lastSendTime.Store(time.Now()) t.counter.Store(0) } diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index 4e2024954e3..d29cb3d96cc 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Inc. +// Copyright 2023 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -24,7 +24,7 @@ import ( "github.com/stretchr/testify/require" ) -func getMockTableStatus() (TopicPartitionKey, *model.RowChangedEvent, *tableStatus) { +func getMockTableStatus() (model.TopicPartitionKey, *model.RowChangedEvent, *tableStatistic) { tableInfo := &model.TableInfo{ TableInfo: &timodel.TableInfo{ UpdateTS: 1, @@ -35,7 +35,7 @@ func getMockTableStatus() (TopicPartitionKey, *model.RowChangedEvent, *tableStat Table: "t1", TableID: 1, } - key := TopicPartitionKey{ + key := model.TopicPartitionKey{ Topic: "test.t1", Partition: 1, TotalPartition: 3, @@ -79,17 +79,17 @@ func TestIsActive(t *testing.T) { t.Parallel() key, row, tb1 := getMockTableStatus() // case 1: A new added table should be active - require.True(t, tb1.isActive(defaultMaxInactiveDuration)) + require.True(t, tb1.isInactive(defaultMaxInactiveDuration)) // case 2: A table which does not receive message for a long time should be inactive tb1.lastMsgReceivedTime.Store(time.Now().Add(-defaultMaxInactiveDuration)) - require.False(t, tb1.isActive(defaultMaxInactiveDuration)) + require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) // case 3: A table which receive message recently should be active // Note: A table's update method will be call any time it receive message // So use update method to simulate the table receive message tb1.update(key, row) - require.True(t, tb1.isActive(defaultMaxInactiveDuration)) + require.True(t, tb1.isInactive(defaultMaxInactiveDuration)) } func TestBootstrapWorker(t *testing.T) { @@ -101,8 +101,8 @@ func TestBootstrapWorker(t *testing.T) { worker := newBootstrapWorker( cfID, outCh, - builder, - time.Duration(config.DefaultSendBootstrapIntervalInSec)*time.Second, + builder.Build(), + config.DefaultSendBootstrapIntervalInSec, config.DefaultSendBootstrapInMsgCount, defaultMaxInactiveDuration) diff --git a/pkg/sink/codec/encoder.go b/pkg/sink/codec/encoder.go index 503c2191c8f..7d518d94657 100644 --- a/pkg/sink/codec/encoder.go +++ b/pkg/sink/codec/encoder.go @@ -84,14 +84,6 @@ func IsColumnValueEqual(preValue, updatedValue interface{}) bool { return preValue == updatedValue } -// TopicPartitionKey contains the topic and partition key of the message. -type TopicPartitionKey struct { - Topic string - Partition int32 - PartitionKey string - TotalPartition int32 -} - // MockRowEventEncoderBuilder is a mock implementation of RowEventEncoderBuilder type MockRowEventEncoderBuilder struct{} diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index e10e33c684b..3fa39ef4b33 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" - "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -41,7 +40,7 @@ type EncoderGroup interface { Run(ctx context.Context) error // AddEvents add events into the group and encode them by one of the encoders in the group. // Note: The caller should make sure all events should belong to the same topic and partition. - AddEvents(ctx context.Context, key TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent) error + AddEvents(ctx context.Context, key model.TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent) error // Output returns a channel produce futures Output() <-chan *future } @@ -79,25 +78,13 @@ func NewEncoderGroup( outCh := make(chan *future, defaultInputChanSize*concurrency) var bootstrapWorker *bootstrapWorker - protocol := util.GetOrZero(cfg.Protocol) - if protocol == config.ProtocolSimple.String() { - log.Info("Sending bootstrap event is enable for simple protocol, creating bootstrap worker...", - zap.Stringer("changefeed", changefeedID)) - sendBootstrapIntervalInSec := util.GetOrZero(cfg.SendBootstrapIntervalInSec) - if sendBootstrapIntervalInSec <= 0 { - sendBootstrapIntervalInSec = config.DefaultSendBootstrapIntervalInSec - } - msgCount := util.GetOrZero(cfg.SendBootstrapInMsgCount) - if msgCount <= 0 { - msgCount = config.DefaultSendBootstrapInMsgCount - } - interval := time.Duration(sendBootstrapIntervalInSec) * time.Second + if cfg.ShouldSendBootstrapMsg() { bootstrapWorker = newBootstrapWorker( changefeedID, outCh, - builder, - interval, - msgCount, + builder.Build(), + *cfg.SendBootstrapIntervalInSec, + *cfg.SendBootstrapInMsgCount, defaultMaxInactiveDuration, ) } @@ -165,7 +152,7 @@ func (g *encoderGroup) runEncoder(ctx context.Context, idx int) error { func (g *encoderGroup) AddEvents( ctx context.Context, - key TopicPartitionKey, + key model.TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent, ) error { // bootstrapWorker only not nil when the protocol is simple @@ -206,13 +193,13 @@ func (g *encoderGroup) cleanMetrics() { // future is a wrapper of the result of encoding events // It's used to notify the caller that the result is ready. type future struct { - Key TopicPartitionKey + Key model.TopicPartitionKey events []*dmlsink.RowChangeCallbackableEvent Messages []*common.Message done chan struct{} } -func newFuture(key TopicPartitionKey, +func newFuture(key model.TopicPartitionKey, events ...*dmlsink.RowChangeCallbackableEvent, ) *future { return &future{ From 9a4cf68d2aa7f04a57ce0265f8e42c07b400346d Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 2 Jan 2024 10:34:06 +0800 Subject: [PATCH 14/20] fix lint check --- pkg/config/sink.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/config/sink.go b/pkg/config/sink.go index c9809bbe320..b3910f975b2 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -196,13 +196,15 @@ func (s *SinkConfig) MaskSensitiveData() { } } +// ShouldSendBootstrapMsg returns whether the sink should send bootstrap message. +// Only enable bootstrap sending function for simple protocol +// and when both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0 func (s *SinkConfig) ShouldSendBootstrapMsg() bool { if s == nil { return false } protocol := util.GetOrZero(s.Protocol) - // Only enable bootstrap sending function for simple protocol - // and when both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0 + return protocol == ProtocolSimple.String() && util.GetOrZero(s.SendBootstrapIntervalInSec) > 0 && util.GetOrZero(s.SendBootstrapInMsgCount) > 0 From 7cbf5d937d8dbea05472bfd7ad25fd40f5d6d1f1 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 2 Jan 2024 11:21:43 +0800 Subject: [PATCH 15/20] fix integration test --- cdc/model/sink_gen.go | 178 ++++++++++++++++++++++++ cdc/model/sink_gen_test.go | 113 +++++++++++++++ tests/integration_tests/api_v2/cases.go | 2 +- 3 files changed, 292 insertions(+), 1 deletion(-) diff --git a/cdc/model/sink_gen.go b/cdc/model/sink_gen.go index a89615c5692..8760f795c1e 100644 --- a/cdc/model/sink_gen.go +++ b/cdc/model/sink_gen.go @@ -2251,3 +2251,181 @@ func (z *TableName) Msgsize() (s int) { s = 1 + 8 + msgp.StringPrefixSize + len(z.Schema) + 9 + msgp.StringPrefixSize + len(z.Table) + 7 + msgp.Int64Size + 13 + msgp.BoolSize return } + +// DecodeMsg implements msgp.Decodable +func (z *TopicPartitionKey) DecodeMsg(dc *msgp.Reader) (err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, err = dc.ReadMapHeader() + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, err = dc.ReadMapKeyPtr() + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Topic": + z.Topic, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "Topic") + return + } + case "Partition": + z.Partition, err = dc.ReadInt32() + if err != nil { + err = msgp.WrapError(err, "Partition") + return + } + case "PartitionKey": + z.PartitionKey, err = dc.ReadString() + if err != nil { + err = msgp.WrapError(err, "PartitionKey") + return + } + case "TotalPartition": + z.TotalPartition, err = dc.ReadInt32() + if err != nil { + err = msgp.WrapError(err, "TotalPartition") + return + } + default: + err = dc.Skip() + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + return +} + +// EncodeMsg implements msgp.Encodable +func (z *TopicPartitionKey) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 4 + // write "Topic" + err = en.Append(0x84, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) + if err != nil { + return + } + err = en.WriteString(z.Topic) + if err != nil { + err = msgp.WrapError(err, "Topic") + return + } + // write "Partition" + err = en.Append(0xa9, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) + if err != nil { + return + } + err = en.WriteInt32(z.Partition) + if err != nil { + err = msgp.WrapError(err, "Partition") + return + } + // write "PartitionKey" + err = en.Append(0xac, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79) + if err != nil { + return + } + err = en.WriteString(z.PartitionKey) + if err != nil { + err = msgp.WrapError(err, "PartitionKey") + return + } + // write "TotalPartition" + err = en.Append(0xae, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) + if err != nil { + return + } + err = en.WriteInt32(z.TotalPartition) + if err != nil { + err = msgp.WrapError(err, "TotalPartition") + return + } + return +} + +// MarshalMsg implements msgp.Marshaler +func (z *TopicPartitionKey) MarshalMsg(b []byte) (o []byte, err error) { + o = msgp.Require(b, z.Msgsize()) + // map header, size 4 + // string "Topic" + o = append(o, 0x84, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) + o = msgp.AppendString(o, z.Topic) + // string "Partition" + o = append(o, 0xa9, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) + o = msgp.AppendInt32(o, z.Partition) + // string "PartitionKey" + o = append(o, 0xac, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79) + o = msgp.AppendString(o, z.PartitionKey) + // string "TotalPartition" + o = append(o, 0xae, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) + o = msgp.AppendInt32(o, z.TotalPartition) + return +} + +// UnmarshalMsg implements msgp.Unmarshaler +func (z *TopicPartitionKey) UnmarshalMsg(bts []byte) (o []byte, err error) { + var field []byte + _ = field + var zb0001 uint32 + zb0001, bts, err = msgp.ReadMapHeaderBytes(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + for zb0001 > 0 { + zb0001-- + field, bts, err = msgp.ReadMapKeyZC(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + switch msgp.UnsafeString(field) { + case "Topic": + z.Topic, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "Topic") + return + } + case "Partition": + z.Partition, bts, err = msgp.ReadInt32Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "Partition") + return + } + case "PartitionKey": + z.PartitionKey, bts, err = msgp.ReadStringBytes(bts) + if err != nil { + err = msgp.WrapError(err, "PartitionKey") + return + } + case "TotalPartition": + z.TotalPartition, bts, err = msgp.ReadInt32Bytes(bts) + if err != nil { + err = msgp.WrapError(err, "TotalPartition") + return + } + default: + bts, err = msgp.Skip(bts) + if err != nil { + err = msgp.WrapError(err) + return + } + } + } + o = bts + return +} + +// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message +func (z *TopicPartitionKey) Msgsize() (s int) { + s = 1 + 6 + msgp.StringPrefixSize + len(z.Topic) + 10 + msgp.Int32Size + 13 + msgp.StringPrefixSize + len(z.PartitionKey) + 15 + msgp.Int32Size + return +} diff --git a/cdc/model/sink_gen_test.go b/cdc/model/sink_gen_test.go index 9291f02bde5..43963ef2e0b 100644 --- a/cdc/model/sink_gen_test.go +++ b/cdc/model/sink_gen_test.go @@ -912,3 +912,116 @@ func BenchmarkDecodeTableName(b *testing.B) { } } } + +func TestMarshalUnmarshalTopicPartitionKey(t *testing.T) { + v := TopicPartitionKey{} + bts, err := v.MarshalMsg(nil) + if err != nil { + t.Fatal(err) + } + left, err := v.UnmarshalMsg(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left) + } + + left, err = msgp.Skip(bts) + if err != nil { + t.Fatal(err) + } + if len(left) > 0 { + t.Errorf("%d bytes left over after Skip(): %q", len(left), left) + } +} + +func BenchmarkMarshalMsgTopicPartitionKey(b *testing.B) { + v := TopicPartitionKey{} + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.MarshalMsg(nil) + } +} + +func BenchmarkAppendMsgTopicPartitionKey(b *testing.B) { + v := TopicPartitionKey{} + bts := make([]byte, 0, v.Msgsize()) + bts, _ = v.MarshalMsg(bts[0:0]) + b.SetBytes(int64(len(bts))) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + bts, _ = v.MarshalMsg(bts[0:0]) + } +} + +func BenchmarkUnmarshalTopicPartitionKey(b *testing.B) { + v := TopicPartitionKey{} + bts, _ := v.MarshalMsg(nil) + b.ReportAllocs() + b.SetBytes(int64(len(bts))) + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := v.UnmarshalMsg(bts) + if err != nil { + b.Fatal(err) + } + } +} + +func TestEncodeDecodeTopicPartitionKey(t *testing.T) { + v := TopicPartitionKey{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + + m := v.Msgsize() + if buf.Len() > m { + t.Log("WARNING: TestEncodeDecodeTopicPartitionKey Msgsize() is inaccurate") + } + + vn := TopicPartitionKey{} + err := msgp.Decode(&buf, &vn) + if err != nil { + t.Error(err) + } + + buf.Reset() + msgp.Encode(&buf, &v) + err = msgp.NewReader(&buf).Skip() + if err != nil { + t.Error(err) + } +} + +func BenchmarkEncodeTopicPartitionKey(b *testing.B) { + v := TopicPartitionKey{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + b.SetBytes(int64(buf.Len())) + en := msgp.NewWriter(msgp.Nowhere) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + v.EncodeMsg(en) + } + en.Flush() +} + +func BenchmarkDecodeTopicPartitionKey(b *testing.B) { + v := TopicPartitionKey{} + var buf bytes.Buffer + msgp.Encode(&buf, &v) + b.SetBytes(int64(buf.Len())) + rd := msgp.NewEndlessReader(buf.Bytes(), b) + dc := msgp.NewReader(rd) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + err := v.DecodeMsg(dc) + if err != nil { + b.Fatal(err) + } + } +} diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index 10c45cde0a9..a71e72c6e9b 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -129,9 +129,9 @@ var defaultReplicaConfig = &ReplicaConfig{ EncoderConcurrency: util.AddressOf(32), EnablePartitionSeparator: util.AddressOf(true), ContentCompatible: util.AddressOf(false), - DebeziumDisableSchema: util.AddressOf(false), SendBootstrapIntervalInSec: util.AddressOf(int64(120)), SendBootstrapInMsgCount: util.AddressOf(int32(10000)), + DebeziumDisableSchema: util.AddressOf(false), }, Scheduler: &ChangefeedSchedulerConfig{ EnableTableAcrossNodes: false, From bdede128d65304a5e925cbdd77ec03ca37dd6f1c Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 2 Jan 2024 13:52:34 +0800 Subject: [PATCH 16/20] fix unit test --- pkg/sink/codec/bootstraper_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index d29cb3d96cc..0d1f0016c0a 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -79,17 +79,17 @@ func TestIsActive(t *testing.T) { t.Parallel() key, row, tb1 := getMockTableStatus() // case 1: A new added table should be active - require.True(t, tb1.isInactive(defaultMaxInactiveDuration)) + require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) // case 2: A table which does not receive message for a long time should be inactive tb1.lastMsgReceivedTime.Store(time.Now().Add(-defaultMaxInactiveDuration)) - require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) + require.True(t, tb1.isInactive(defaultMaxInactiveDuration)) // case 3: A table which receive message recently should be active // Note: A table's update method will be call any time it receive message // So use update method to simulate the table receive message tb1.update(key, row) - require.True(t, tb1.isInactive(defaultMaxInactiveDuration)) + require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) } func TestBootstrapWorker(t *testing.T) { From b4e267b8c1d865f0b58cda54f8e71433b16ed49f Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Tue, 2 Jan 2024 15:36:43 +0800 Subject: [PATCH 17/20] Resolve comments --- cdc/model/sink.go | 13 +++++++++++++ pkg/sink/codec/bootstraper.go | 22 +++++++--------------- tests/integration_tests/api_v2/cases.go | 1 + 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index 9eade84bccd..b7d59df01fb 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -747,6 +747,19 @@ func (d *DDLEvent) FromJobWithArgs( } } +// NewBootstrapDDLEvent returns a bootstrap DDL event. +// We set Bootstrap DDL event's startTs and commitTs to 0. +// Because it is generated by the TiCDC, not from the upstream TiDB. +// And they ere useless for a bootstrap DDL event. +func NewBootstrapDDLEvent(tableInfo *TableInfo) *DDLEvent { + return &DDLEvent{ + StartTs: 0, + CommitTs: 0, + TableInfo: tableInfo, + IsBootstrap: true, + } +} + // SingleTableTxn represents a transaction which includes many row events in a single table // //msgp:ignore SingleTableTxn diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index a6e3d281566..ab5b9ee7c3e 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -77,7 +77,7 @@ func (b *bootstrapWorker) run(ctx context.Context) error { sendTicker.Stop() }() - errCh := make(chan error, 1) + var err error for { select { case <-ctx.Done(): @@ -85,17 +85,14 @@ func (b *bootstrapWorker) run(ctx context.Context) error { case <-sendTicker.C: b.activeTables.Range(func(key, value interface{}) bool { table := value.(*tableStatistic) - err := b.sendBootstrapMsg(ctx, table) - if err != nil { - errCh <- err - return false - } - return true + err = b.sendBootstrapMsg(ctx, table) + return err == nil }) + if err != nil { + return errors.Trace(err) + } case <-gcTicker.C: b.gcInactiveTables() - case err := <-errCh: - return err } } } @@ -155,12 +152,7 @@ func (b *bootstrapWorker) generateEvents( res := make([]*future, 0, int(totalPartition)) // Bootstrap messages of a table should be sent to all partitions. for i := 0; i < int(totalPartition); i++ { - msg, err := b.encoder.EncodeDDLEvent(&model.DDLEvent{ - StartTs: 0, - CommitTs: 0, - TableInfo: tableInfo, - IsBootstrap: true, - }) + msg, err := b.encoder.EncodeDDLEvent(model.NewBootstrapDDLEvent(tableInfo)) if err != nil { return nil, errors.Trace(err) } diff --git a/tests/integration_tests/api_v2/cases.go b/tests/integration_tests/api_v2/cases.go index a71e72c6e9b..c0bf4770e4a 100644 --- a/tests/integration_tests/api_v2/cases.go +++ b/tests/integration_tests/api_v2/cases.go @@ -76,6 +76,7 @@ var customReplicaConfig = &ReplicaConfig{ Delimiter: config.Comma, NullString: config.NULL, }, + DateSeparator: "day", EncoderConcurrency: util.AddressOf(32), EnablePartitionSeparator: util.AddressOf(true), ContentCompatible: util.AddressOf(true), From 34409f3e8a3b1b274e503c3374c7b6f0cac2a54a Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 4 Jan 2024 10:15:01 +0800 Subject: [PATCH 18/20] Resolve comments again --- cdc/model/sink.go | 7 ++- cdc/model/sink_gen.go | 41 ++++------------- cdc/sink/dmlsink/mq/kafka_dml_sink.go | 3 +- cdc/sink/dmlsink/mq/mq_dml_sink.go | 7 ++- cdc/sink/dmlsink/mq/pulsar_dml_sink.go | 3 +- cdc/sink/dmlsink/mq/worker_test.go | 6 ++- pkg/sink/codec/bootstraper.go | 64 +++++++++++--------------- pkg/sink/codec/bootstraper_test.go | 13 +++--- pkg/sink/codec/encoder_group.go | 4 +- 9 files changed, 54 insertions(+), 94 deletions(-) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index b7d59df01fb..c37a530babb 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -928,8 +928,7 @@ func (t *SingleTableTxn) ToWaitFlush() bool { // TopicPartitionKey contains the topic and partition key of the message. type TopicPartitionKey struct { - Topic string - Partition int32 - PartitionKey string - TotalPartition int32 + Topic string + Partition int32 + PartitionKey string } diff --git a/cdc/model/sink_gen.go b/cdc/model/sink_gen.go index 8760f795c1e..13ee1f10247 100644 --- a/cdc/model/sink_gen.go +++ b/cdc/model/sink_gen.go @@ -2288,12 +2288,6 @@ func (z *TopicPartitionKey) DecodeMsg(dc *msgp.Reader) (err error) { err = msgp.WrapError(err, "PartitionKey") return } - case "TotalPartition": - z.TotalPartition, err = dc.ReadInt32() - if err != nil { - err = msgp.WrapError(err, "TotalPartition") - return - } default: err = dc.Skip() if err != nil { @@ -2306,10 +2300,10 @@ func (z *TopicPartitionKey) DecodeMsg(dc *msgp.Reader) (err error) { } // EncodeMsg implements msgp.Encodable -func (z *TopicPartitionKey) EncodeMsg(en *msgp.Writer) (err error) { - // map header, size 4 +func (z TopicPartitionKey) EncodeMsg(en *msgp.Writer) (err error) { + // map header, size 3 // write "Topic" - err = en.Append(0x84, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) + err = en.Append(0x83, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) if err != nil { return } @@ -2338,25 +2332,15 @@ func (z *TopicPartitionKey) EncodeMsg(en *msgp.Writer) (err error) { err = msgp.WrapError(err, "PartitionKey") return } - // write "TotalPartition" - err = en.Append(0xae, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) - if err != nil { - return - } - err = en.WriteInt32(z.TotalPartition) - if err != nil { - err = msgp.WrapError(err, "TotalPartition") - return - } return } // MarshalMsg implements msgp.Marshaler -func (z *TopicPartitionKey) MarshalMsg(b []byte) (o []byte, err error) { +func (z TopicPartitionKey) MarshalMsg(b []byte) (o []byte, err error) { o = msgp.Require(b, z.Msgsize()) - // map header, size 4 + // map header, size 3 // string "Topic" - o = append(o, 0x84, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) + o = append(o, 0x83, 0xa5, 0x54, 0x6f, 0x70, 0x69, 0x63) o = msgp.AppendString(o, z.Topic) // string "Partition" o = append(o, 0xa9, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) @@ -2364,9 +2348,6 @@ func (z *TopicPartitionKey) MarshalMsg(b []byte) (o []byte, err error) { // string "PartitionKey" o = append(o, 0xac, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79) o = msgp.AppendString(o, z.PartitionKey) - // string "TotalPartition" - o = append(o, 0xae, 0x54, 0x6f, 0x74, 0x61, 0x6c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e) - o = msgp.AppendInt32(o, z.TotalPartition) return } @@ -2406,12 +2387,6 @@ func (z *TopicPartitionKey) UnmarshalMsg(bts []byte) (o []byte, err error) { err = msgp.WrapError(err, "PartitionKey") return } - case "TotalPartition": - z.TotalPartition, bts, err = msgp.ReadInt32Bytes(bts) - if err != nil { - err = msgp.WrapError(err, "TotalPartition") - return - } default: bts, err = msgp.Skip(bts) if err != nil { @@ -2425,7 +2400,7 @@ func (z *TopicPartitionKey) UnmarshalMsg(bts []byte) (o []byte, err error) { } // Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message -func (z *TopicPartitionKey) Msgsize() (s int) { - s = 1 + 6 + msgp.StringPrefixSize + len(z.Topic) + 10 + msgp.Int32Size + 13 + msgp.StringPrefixSize + len(z.PartitionKey) + 15 + msgp.Int32Size +func (z TopicPartitionKey) Msgsize() (s int) { + s = 1 + 6 + msgp.StringPrefixSize + len(z.Topic) + 10 + msgp.Int32Size + 13 + msgp.StringPrefixSize + len(z.PartitionKey) return } diff --git a/cdc/sink/dmlsink/mq/kafka_dml_sink.go b/cdc/sink/dmlsink/mq/kafka_dml_sink.go index 70b12c79a81..3841c2a7298 100644 --- a/cdc/sink/dmlsink/mq/kafka_dml_sink.go +++ b/cdc/sink/dmlsink/mq/kafka_dml_sink.go @@ -121,8 +121,7 @@ func NewKafkaDMLSink( metricsCollector := factory.MetricsCollector(tiflowutil.RoleProcessor, adminClient) dmlProducer := producerCreator(ctx, changefeedID, asyncProducer, metricsCollector, errCh, failpointCh) - concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) - encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, concurrency, changefeedID) + encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, changefeedID) s := newDMLSink(ctx, changefeedID, dmlProducer, adminClient, topicManager, eventRouter, trans, encoderGroup, protocol, scheme, errCh) log.Info("DML sink producer created", diff --git a/cdc/sink/dmlsink/mq/mq_dml_sink.go b/cdc/sink/dmlsink/mq/mq_dml_sink.go index e6d2105b69f..39bfb1639ef 100644 --- a/cdc/sink/dmlsink/mq/mq_dml_sink.go +++ b/cdc/sink/dmlsink/mq/mq_dml_sink.go @@ -196,10 +196,9 @@ func (s *dmlSink) WriteEvents(txns ...*dmlsink.CallbackableEvent[*model.SingleTa // So it is safe to send the event to a unbounded channel here. s.alive.worker.msgChan.In() <- mqEvent{ key: model.TopicPartitionKey{ - Topic: topic, - Partition: index, - PartitionKey: key, - TotalPartition: partitionNum, + Topic: topic, + Partition: index, + PartitionKey: key, }, rowEvent: &dmlsink.RowChangeCallbackableEvent{ Event: row, diff --git a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go index 1eec9864240..089f521fd37 100644 --- a/cdc/sink/dmlsink/mq/pulsar_dml_sink.go +++ b/cdc/sink/dmlsink/mq/pulsar_dml_sink.go @@ -121,8 +121,7 @@ func NewPulsarDMLSink( return nil, cerror.WrapError(cerror.ErrPulsarInvalidConfig, err) } - concurrency := tiflowutil.GetOrZero(replicaConfig.Sink.EncoderConcurrency) - encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, concurrency, changefeedID) + encoderGroup := codec.NewEncoderGroup(replicaConfig.Sink, encoderBuilder, changefeedID) s := newDMLSink(ctx, changefeedID, p, nil, topicManager, eventRouter, trans, encoderGroup, protocol, scheme, errCh) diff --git a/cdc/sink/dmlsink/mq/worker_test.go b/cdc/sink/dmlsink/mq/worker_test.go index ddee84cd6a7..f9877acd33c 100644 --- a/cdc/sink/dmlsink/mq/worker_test.go +++ b/cdc/sink/dmlsink/mq/worker_test.go @@ -45,7 +45,8 @@ func newBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlproduc encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) cfg := config.GetDefaultReplicaConfig() - encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, encoderConcurrency, id) + cfg.Sink.EncoderConcurrency = &encoderConcurrency + encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, id) return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } @@ -60,7 +61,8 @@ func newNonBatchEncodeWorker(ctx context.Context, t *testing.T) (*worker, dmlpro encoderConcurrency := 4 statistics := metrics.NewStatistics(ctx, id, sink.RowSink) cfg := config.GetDefaultReplicaConfig() - encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, encoderConcurrency, id) + cfg.Sink.EncoderConcurrency = &encoderConcurrency + encoderGroup := codec.NewEncoderGroup(cfg.Sink, encoderBuilder, id) return newWorker(id, config.ProtocolOpen, p, encoderGroup, statistics), p } diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index ab5b9ee7c3e..a8d517ae507 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -30,6 +30,10 @@ const ( bootstrapWorkerGCInterval = 30 * time.Second defaultMaxInactiveDuration = 30 * time.Minute + + // In the current implementation, the bootstrapWorker only sends bootstrap message + // to the first partition of the corresponding topic of the table. + defaultBootstrapPartitionIndex = 0 ) type bootstrapWorker struct { @@ -113,7 +117,7 @@ func (b *bootstrapWorker) addEvent( } } else { // If the table is already in the activeTables, update its status. - table.(*tableStatistic).update(key, row) + table.(*tableStatistic).update(row) } return nil } @@ -130,48 +134,40 @@ func (b *bootstrapWorker) sendBootstrapMsg(ctx context.Context, table *tableStat } table.reset() tableInfo := table.tableInfo.Load().(*model.TableInfo) - events, err := b.generateEvents(table.topic, table.totalPartition.Load(), tableInfo) + event, err := b.generateEvents(table.topic, tableInfo) if err != nil { return errors.Trace(err) } - for _, event := range events { - select { - case <-ctx.Done(): - return ctx.Err() - case b.outCh <- event: - } + select { + case <-ctx.Done(): + return ctx.Err() + case b.outCh <- event: } return nil } func (b *bootstrapWorker) generateEvents( topic string, - totalPartition int32, tableInfo *model.TableInfo, -) ([]*future, error) { - res := make([]*future, 0, int(totalPartition)) +) (*future, error) { // Bootstrap messages of a table should be sent to all partitions. - for i := 0; i < int(totalPartition); i++ { - msg, err := b.encoder.EncodeDDLEvent(model.NewBootstrapDDLEvent(tableInfo)) - if err != nil { - return nil, errors.Trace(err) - } + msg, err := b.encoder.EncodeDDLEvent(model.NewBootstrapDDLEvent(tableInfo)) + if err != nil { + return nil, errors.Trace(err) + } - key := model.TopicPartitionKey{ - Topic: topic, - Partition: int32(i), - TotalPartition: totalPartition, - } + key := model.TopicPartitionKey{ + Topic: topic, + Partition: defaultBootstrapPartitionIndex, + } - future := &future{ - Key: key, - done: make(chan struct{}), - } - future.Messages = append(future.Messages, msg) - close(future.done) - res = append(res, future) + f := &future{ + Key: key, + done: make(chan struct{}), } - return res, nil + f.Messages = append(f.Messages, msg) + close(f.done) + return f, nil } func (b *bootstrapWorker) gcInactiveTables() { @@ -194,9 +190,6 @@ type tableStatistic struct { id int64 // topic is the table's topic, it will not change topic string - // All fields below are concurrently accessed, please use atomic operations. - // totalPartition is the total number of partitions of the table's topic - totalPartition atomic.Int32 // counter is the number of row event sent since last bootstrap message sent // It is used to check if the bootstrap message should be sent counter atomic.Int32 @@ -220,7 +213,6 @@ func newTableStatus(key model.TopicPartitionKey, row *model.RowChangedEvent) *ta topic: key.Topic, } res.counter.Add(1) - res.totalPartition.Store(key.TotalPartition) res.lastMsgReceivedTime.Store(time.Now()) res.lastSendTime.Store(time.Unix(0, 0)) res.version.Store(row.TableInfo.UpdateTS) @@ -237,14 +229,10 @@ func (t *tableStatistic) shouldSendBootstrapMsg( t.counter.Load() >= sendBootstrapMsgCountInterval } -func (t *tableStatistic) update(key model.TopicPartitionKey, row *model.RowChangedEvent) { +func (t *tableStatistic) update(row *model.RowChangedEvent) { t.counter.Add(1) t.lastMsgReceivedTime.Store(time.Now()) - if t.totalPartition.Load() != key.TotalPartition { - t.totalPartition.Store(key.TotalPartition) - } - if t.version.Load() != row.TableInfo.UpdateTS { t.version.Store(row.TableInfo.UpdateTS) t.tableInfo.Store(row.TableInfo) diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index 0d1f0016c0a..59e2cd19664 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -36,9 +36,8 @@ func getMockTableStatus() (model.TopicPartitionKey, *model.RowChangedEvent, *tab TableID: 1, } key := model.TopicPartitionKey{ - Topic: "test.t1", - Partition: 1, - TotalPartition: 3, + Topic: "test.t1", + Partition: 1, } row := &model.RowChangedEvent{ TableInfo: tableInfo, @@ -77,7 +76,7 @@ func TestShouldSendBootstrapMsg(t *testing.T) { func TestIsActive(t *testing.T) { t.Parallel() - key, row, tb1 := getMockTableStatus() + _, row, tb1 := getMockTableStatus() // case 1: A new added table should be active require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) @@ -88,7 +87,7 @@ func TestIsActive(t *testing.T) { // case 3: A table which receive message recently should be active // Note: A table's update method will be call any time it receive message // So use update method to simulate the table receive message - tb1.update(key, row) + tb1.update(row) require.False(t, tb1.isInactive(defaultMaxInactiveDuration)) } @@ -129,10 +128,10 @@ func TestBootstrapWorker(t *testing.T) { case future := <-outCh: require.NotNil(t, future) require.Equal(t, key.Topic, future.Key.Topic) - require.Equal(t, key.TotalPartition, future.Key.TotalPartition) msgCount++ case <-sctx.Done(): - require.Equal(t, key.TotalPartition, msgCount) + // The bootstrap event is only sent to the + require.Equal(t, 1, msgCount) return } } diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index 3fa39ef4b33..443e8cc559a 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -64,13 +65,12 @@ type encoderGroup struct { func NewEncoderGroup( cfg *config.SinkConfig, builder RowEventEncoderBuilder, - concurrency int, changefeedID model.ChangeFeedID, ) *encoderGroup { + concurrency := util.GetOrZero(cfg.EncoderConcurrency) if concurrency <= 0 { concurrency = config.DefaultEncoderGroupConcurrency } - inputCh := make([]chan *future, concurrency) for i := 0; i < concurrency; i++ { inputCh[i] = make(chan *future, defaultInputChanSize) From 19e1dc3667aa0a243755f5cc5f73e9ed007d20f2 Mon Sep 17 00:00:00 2001 From: dongmen <414110582@qq.com> Date: Thu, 4 Jan 2024 11:45:44 +0800 Subject: [PATCH 19/20] fix test --- pkg/sink/codec/bootstraper_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sink/codec/bootstraper_test.go b/pkg/sink/codec/bootstraper_test.go index 59e2cd19664..ec8528ee8c7 100644 --- a/pkg/sink/codec/bootstraper_test.go +++ b/pkg/sink/codec/bootstraper_test.go @@ -131,7 +131,7 @@ func TestBootstrapWorker(t *testing.T) { msgCount++ case <-sctx.Done(): // The bootstrap event is only sent to the - require.Equal(t, 1, msgCount) + require.Equal(t, int32(1), msgCount) return } } From 50a21d69de07ba39987bf032040b420b5590c508 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Thu, 4 Jan 2024 15:30:12 +0800 Subject: [PATCH 20/20] Update pkg/sink/codec/bootstraper.go Co-authored-by: Jianyuan Jiang --- pkg/sink/codec/bootstraper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index a8d517ae507..cdc6bb7e1f0 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -56,7 +56,7 @@ func newBootstrapWorker( sendBootstrapInMsgCount int32, maxInactiveDuration time.Duration, ) *bootstrapWorker { - log.Info("Sending bootstrap event is enable for simple protocol,"+ + log.Info("Sending bootstrap event is enabled for simple protocol,"+ "and both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0"+ "enable bootstrap sending function", zap.Stringer("changefeed", changefeedID),